From 7830462d4b7da3acefff6353419e71cde62d5fee Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Tue, 14 Mar 2017 14:37:47 +0700 Subject: [PATCH 001/563] 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 120274b451646ddd9e1de9e12a4904414f881c7c Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Tue, 14 Mar 2017 16:20:46 +0700 Subject: [PATCH 002/563] 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 bac3424936eacb2381138612ca70276ef5f909d5 Mon Sep 17 00:00:00 2001 From: markrmiller Date: Tue, 14 Mar 2017 06:01:06 -0400 Subject: [PATCH 003/563] 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 91c3f78f8fafbd95cd375bb114e80831ba50d525 Mon Sep 17 00:00:00 2001 From: markrmiller Date: Tue, 14 Mar 2017 06:13:34 -0400 Subject: [PATCH 004/563] 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 ddda27e4deab45b9a6bfec8d61319b00f88e27f6 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Tue, 14 Mar 2017 10:46:25 +0000 Subject: [PATCH 005/563] 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 1750095048a5bee9c8f63d9bf6ad53d4a309aa60 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 14 Mar 2017 11:17:56 +0100 Subject: [PATCH 006/563] 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 5aafea84948e9838dd2cd333c3e873bea176c9e8 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Tue, 14 Mar 2017 13:05:55 +0000 Subject: [PATCH 007/563] 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 9417367b1afd404b944692d9947f78587e2b2d94 Mon Sep 17 00:00:00 2001 From: Chris Hostetter Date: Tue, 14 Mar 2017 09:49:22 -0700 Subject: [PATCH 008/563] 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 09bd8612ce82c2852542127eb24b887c7b332a0d Mon Sep 17 00:00:00 2001 From: Chris Hostetter Date: Tue, 14 Mar 2017 10:23:49 -0700 Subject: [PATCH 009/563] 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 1192d396dd5f1384b6cee290f1b9210c5aad8d9e Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Tue, 14 Mar 2017 19:57:57 +0100 Subject: [PATCH 010/563] 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 124b50581041d6b2b3e68aa37a26e66aceb2abf1 Mon Sep 17 00:00:00 2001 From: Noble Paul Date: Wed, 15 Mar 2017 18:38:48 +1030 Subject: [PATCH 011/563] 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 716d43eca936e3ea5325a0ee0cceb087a4ff32e5 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Wed, 15 Mar 2017 06:03:54 -0400 Subject: [PATCH 012/563] 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 e7b87f5b79f744252bca79356c2bcdeaed503e74 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Wed, 15 Mar 2017 06:08:10 -0400 Subject: [PATCH 013/563] 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 65c695b025ad0efb952494f767c1ec9fa44a4924 Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Wed, 15 Mar 2017 17:25:06 +0530 Subject: [PATCH 014/563] 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 9d56f136505098ea5538c5d6eaaf60536848feb9 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Wed, 15 Mar 2017 10:31:10 +0000 Subject: [PATCH 015/563] 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 2bce98b0c162c5d8a815bc3e2ec32ba6d08c62fa Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Thu, 16 Mar 2017 02:03:09 +0530 Subject: [PATCH 016/563] 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 22f91ba0cafeb04e1568cb8c61b5356ef7e91ade Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Mon, 13 Mar 2017 12:50:17 -0400 Subject: [PATCH 017/563] 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 be9fea1bc509eff9296c89e7270c5e8669b39d0f Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Mon, 13 Mar 2017 17:10:21 -0400 Subject: [PATCH 018/563] 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 b46e09c79f849d9211b3de235788bbf32d7aa84b Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Wed, 15 Mar 2017 18:31:14 -0400 Subject: [PATCH 019/563] 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 d82573704856fea7fe3980e9f4c0b995fafb718f Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Thu, 16 Mar 2017 05:32:27 +0530 Subject: [PATCH 020/563] 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+"$@"} From 17cc3e7dad7eb2f691767fc5a08aac8e6d055bdc Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Thu, 16 Mar 2017 14:10:28 +0000 Subject: [PATCH 021/563] SOLR-10283: Learning to Rank (LTR) SolrFeature to reject searches with missing efi (External Feature Information) used by fq. --- solr/CHANGES.txt | 3 +++ .../apache/solr/ltr/feature/SolrFeature.java | 3 +++ .../featureExamples/external_features.json | 6 +++++ .../solr/ltr/TestSelectiveWeightCreation.java | 8 ++++-- .../ltr/feature/TestExternalFeatures.java | 25 ++++++++++++++++--- 5 files changed, 39 insertions(+), 6 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 5e2a518ddd8..2d4ce6c93f7 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -260,6 +260,9 @@ Bug Fixes * SOLR-10302: Solr's zkcli scripts now able to find the metrics libraries, which it couldn't earlier (kiran, Ishan Chattopadhyaya) +* SOLR-10283: Learning to Rank (LTR) SolrFeature to reject searches with missing efi (External Feature Information) used by fq. + (Christine Poerschke) + Optimizations ---------------------- diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/SolrFeature.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/SolrFeature.java index 13eb96fee2a..4aa872d2e7a 100644 --- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/SolrFeature.java +++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/SolrFeature.java @@ -157,6 +157,9 @@ public class SolrFeature extends Feature { for (String fq : fqs) { if ((fq != null) && (fq.trim().length() != 0)) { fq = macroExpander.expand(fq); + if (fq == null) { + throw new FeatureException(this.getClass().getSimpleName()+" requires efi parameter that was not passed in request."); + } final QParser fqp = QParser.getParser(fq, req); final Query filterQuery = fqp.getQuery(); if (filterQuery != null) { diff --git a/solr/contrib/ltr/src/test-files/featureExamples/external_features.json b/solr/contrib/ltr/src/test-files/featureExamples/external_features.json index d8a9ecae6f9..5c4f12ddaa0 100644 --- a/solr/contrib/ltr/src/test-files/featureExamples/external_features.json +++ b/solr/contrib/ltr/src/test-files/featureExamples/external_features.json @@ -48,4 +48,10 @@ "params" : { "q" : "{!field f=title}${user_query}" } +}, { + "name" : "titlePhrasesMatch", + "class" : "org.apache.solr.ltr.feature.SolrFeature", + "params" : { + "fq" : [ "{!field f=title}${userTitlePhrase1}", "{!field f=title}${userTitlePhrase2}"] + } } ] diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java index 5cfd999b360..7bf8373a56a 100644 --- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java +++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestSelectiveWeightCreation.java @@ -210,18 +210,22 @@ public class TestSelectiveWeightCreation extends TestRerankBase { @Test public void testSelectiveWeightsRequestFeaturesFromDifferentStore() throws Exception { - final String docs0fv = FeatureLoggerTestUtils.toFeatureVector( + final String docs0fv_sparse = FeatureLoggerTestUtils.toFeatureVector( "matchedTitle","1.0", "titlePhraseMatch","0.6103343"); + final String docs0fv_dense = FeatureLoggerTestUtils.toFeatureVector( + "matchedTitle","1.0", "titlePhraseMatch","0.6103343", "titlePhrasesMatch","0.0"); final String docs0fv_fstore4= FeatureLoggerTestUtils.toFeatureVector( "popularity","3.0", "originalScore","1.0"); + final String docs0fv = chooseDefaultFeatureVector(docs0fv_dense, docs0fv_sparse); + // extract all features in externalmodel's store (default store) // rerank using externalmodel (default store) final SolrQuery query = new SolrQuery(); query.setQuery("*:*"); query.add("fl", "*,score,fv:[fv]"); query.add("rows", "5"); - query.add("rq", "{!ltr reRankDocs=10 model=externalmodel efi.user_query=w3}"); + query.add("rq", "{!ltr reRankDocs=10 model=externalmodel efi.user_query=w3 efi.userTitlePhrase1=w2 efi.userTitlePhrase2=w1}"); assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='3'"); assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='4'"); diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java index 4010ee1900c..c6ae30fcd58 100644 --- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java +++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestExternalFeatures.java @@ -67,7 +67,7 @@ public class TestExternalFeatures extends TestRerankBase { query.remove("fl"); query.add("fl", "*,score,[fv]"); - query.add("rq", "{!ltr reRankDocs=10 model=externalmodel efi.user_query=w3}"); + query.add("rq", "{!ltr reRankDocs=10 model=externalmodel efi.user_query=w3 efi.userTitlePhrase1=w4 efi.userTitlePhrase2=w5}"); assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='3'"); assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.7693934"); @@ -77,7 +77,7 @@ public class TestExternalFeatures extends TestRerankBase { // Adding an efi in the transformer should not affect the rq ranking with a // different value for efi of the same parameter query.remove("fl"); - query.add("fl", "*,score,[fv efi.user_query=w2]"); + query.add("fl", "*,score,[fv efi.user_query=w2 efi.userTitlePhrase1=w4 efi.userTitlePhrase2=w5]"); assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='3'"); assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==0.7693934"); @@ -92,11 +92,12 @@ public class TestExternalFeatures extends TestRerankBase { query.add("fl", "*,score,fv:[fv]"); query.add("rows", "1"); // Stopword only query passed in - query.add("rq", "{!ltr reRankDocs=10 model=externalmodel efi.user_query='a'}"); + query.add("rq", "{!ltr reRankDocs=10 model=externalmodel efi.user_query='a' efi.userTitlePhrase1='b' efi.userTitlePhrase2='c'}"); final String docs0fv_dense_csv = FeatureLoggerTestUtils.toFeatureVector( "matchedTitle","0.0", - "titlePhraseMatch","0.0"); + "titlePhraseMatch","0.0", + "titlePhrasesMatch","0.0"); final String docs0fv_sparse_csv = FeatureLoggerTestUtils.toFeatureVector(); final String docs0fv_default_csv = chooseDefaultFeatureVector(docs0fv_dense_csv, docs0fv_sparse_csv); @@ -181,4 +182,20 @@ public class TestExternalFeatures extends TestRerankBase { query.add("fl", "fvalias:[fv store=fstore4]"); assertJQ("/query" + query.toQueryString(), "/error/msg=='Exception from createWeight for ValueFeature [name=popularity, params={value=${myPop}, required=true}] ValueFeatureWeight requires efi parameter that was not passed in request.'"); } + + @Test + public void featureExtraction_valueFeatureRequiredInFq_shouldThrowException() throws Exception { + final String userTitlePhrase1 = "userTitlePhrase1"; + final String userTitlePhrase2 = "userTitlePhrase2"; + final String userTitlePhrasePresent = (random().nextBoolean() ? userTitlePhrase1 : userTitlePhrase2); + + final SolrQuery query = new SolrQuery(); + query.setQuery("*:*"); + query.add("rows", "1"); + query.add("fl", "score,features:[fv efi.user_query=uq "+userTitlePhrasePresent+"=utpp]"); + assertJQ("/query" + query.toQueryString(), "/error/msg=='Exception from createWeight for " + + "SolrFeature [name=titlePhrasesMatch, params={fq=[{!field f=title}${"+userTitlePhrase1+"}, {!field f=title}${"+userTitlePhrase2+"}]}] " + + "SolrFeatureWeight requires efi parameter that was not passed in request.'"); + } + } From 3d81a9c8e72c396503bde41e1b53a090822fbec7 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Thu, 16 Mar 2017 13:54:25 -0400 Subject: [PATCH 022/563] SOLR-10254, 10085: Update CHANGES.txt --- solr/CHANGES.txt | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 2d4ce6c93f7..6ffb93f6053 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -100,6 +100,7 @@ Velocity 1.7 and Velocity Tools 2.0 Apache UIMA 2.3.1 Apache ZooKeeper 3.4.6 Jetty 9.3.14.v20161028 +Apache Calcite 1.11.0 Detailed Change List ---------------------- @@ -111,6 +112,11 @@ Upgrade Notes number of requests. New Codahale Metrics implementation applies exponential decay to this value, which heavily biases the average towards the last 5 minutes. (ab) +* SOLR-8593: Parallel SQL now uses Apache Calcite as it's SQL framework. As part of this change + the default aggregation mode has been changed to facet rather map_reduce. There has also beeen changes + to the SQL aggregate response and some SQL syntax changes. Consult the documentation for full details. + + New Features ---------------------- @@ -187,6 +193,10 @@ New Features * SOLR-10046: Add UninvertDocValuesMergePolicyFactory class. (Keith Laban, Christine Poerschke) +* SOLR-10085: SQL result set fields should be ordered by the field list (Joel Bernstein) + +* SOLR-10254: significantTerms Streaming Expression should work in non-SolrCloud mode (Joel Bernstein) + Bug Fixes ---------------------- From 7a625bbb25c119014e94a952c06d0f789bf04c3c Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Thu, 16 Mar 2017 11:08:50 -0700 Subject: [PATCH 023/563] SOLR-9990: Add PointFields in example/default schemas --- solr/CHANGES.txt | 2 + .../basic_configs/conf/managed-schema | 67 ++++++++++----- .../conf/managed-schema | 64 +++++++++----- .../conf/managed-schema | 85 ++++++++++++++----- 4 files changed, 157 insertions(+), 61 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 6ffb93f6053..d49b439018f 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -365,6 +365,8 @@ Other Changes * SOLR-10247: Support non-numeric metrics and a "compact" format of /admin/metrics output. (ab) +* SOLR-9990: Add PointFields in example/default schemas (Tomás Fernández Löbbe) + ================== 6.4.2 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/server/solr/configsets/basic_configs/conf/managed-schema b/solr/server/solr/configsets/basic_configs/conf/managed-schema index 22553d8bcd1..d7aacc48644 100644 --- a/solr/server/solr/configsets/basic_configs/conf/managed-schema +++ b/solr/server/solr/configsets/basic_configs/conf/managed-schema @@ -71,14 +71,14 @@ indexed: true if this field should be indexed (searchable or sortable) stored: true if this field should be retrievable docValues: true if this field should have doc values. Doc values are - useful for faceting, grouping, sorting and function queries. Although not - required, doc values will make the index faster to load, more - NRT-friendly and more memory-efficient. They however come with some - limitations: they are currently only supported by StrField, UUIDField - and all Trie*Fields, and depending on the field type, they might - require the field to be single-valued, be required or have a default - value (check the documentation of the field type you're interested in - for more information) + useful (required, if you are using *Point fields) for faceting, + grouping, sorting and function queries. Doc values will make the index + faster to load, more NRT-friendly and more memory-efficient. + They however come with some limitations: they are currently only + supported by StrField, UUIDField, all Trie*Fields and *PointFields, + and depending on the field type, they might require the field to be + single-valued, be required or have a default value (check the + documentation of the field type you're interested in for more information) multiValued: true if this field may contain multiple values per document omitNorms: (expert) set to true to omit the norms associated with this field (this disables length normalization and index-time @@ -152,8 +152,20 @@ + + + + + + + + + + + + - + @@ -216,8 +228,8 @@ + --> + + + + + + + + + + + @@ -254,6 +279,8 @@ Smaller precisionStep values (specified in bits) will lead to more tokens indexed per value, slightly larger index size, and faster range queries. A precisionStep of 0 disables indexing at different precision levels. + + Consider using pint/pfloat/plong/pdouble instead of Trie* fields if possible --> @@ -284,15 +311,15 @@ the current day Consult the TrieDateField javadocs for more information. - - Note: For faster range queries, consider the tdate type --> + + + + - - diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema index 558c05e759c..b716f9c6d06 100644 --- a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema +++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema @@ -71,14 +71,14 @@ indexed: true if this field should be indexed (searchable or sortable) stored: true if this field should be retrievable docValues: true if this field should have doc values. Doc values are - useful for faceting, grouping, sorting and function queries. Although not - required, doc values will make the index faster to load, more - NRT-friendly and more memory-efficient. They however come with some - limitations: they are currently only supported by StrField, UUIDField - and all Trie*Fields, and depending on the field type, they might - require the field to be single-valued, be required or have a default - value (check the documentation of the field type you're interested in - for more information) + useful (required, if you are using *Point fields) for faceting, + grouping, sorting and function queries. Doc values will make the index + faster to load, more NRT-friendly and more memory-efficient. + They however come with some limitations: they are currently only + supported by StrField, UUIDField, all Trie*Fields and *PointFields, + and depending on the field type, they might require the field to be + single-valued, be required or have a default value (check the + documentation of the field type you're interested in for more information) multiValued: true if this field may contain multiple values per document omitNorms: (expert) set to true to omit the norms associated with this field (this disables length normalization and index-time @@ -152,8 +152,20 @@ + + + + + + + + + + + + - + @@ -216,8 +228,8 @@ + + + + + + + + + - These fields support doc values, but they require the field to be - single-valued and either be required or have a default value. + @@ -254,6 +279,8 @@ Smaller precisionStep values (specified in bits) will lead to more tokens indexed per value, slightly larger index size, and faster range queries. A precisionStep of 0 disables indexing at different precision levels. + + Consider using pint/pfloat/plong/pdouble instead of Trie* fields if possible --> @@ -284,15 +311,14 @@ the current day Consult the TrieDateField javadocs for more information. - - Note: For faster range queries, consider the tdate type --> + + + + - - - diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema index bd292a07763..9caf3d6e2b0 100644 --- a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema +++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema @@ -72,14 +72,14 @@ indexed: true if this field should be indexed (searchable or sortable) stored: true if this field should be retrievable docValues: true if this field should have doc values. Doc values are - useful for faceting, grouping, sorting and function queries. Although not - required, doc values will make the index faster to load, more - NRT-friendly and more memory-efficient. They however come with some - limitations: they are currently only supported by StrField, UUIDField - and all Trie*Fields, and depending on the field type, they might - require the field to be single-valued, be required or have a default - value (check the documentation of the field type you're interested in - for more information) + useful (required, if you are using *Point fields) for faceting, + grouping, sorting and function queries. Doc values will make the index + faster to load, more NRT-friendly and more memory-efficient. + They however come with some limitations: they are currently only + supported by StrField, UUIDField, all Trie*Fields and *PointFields, + and depending on the field type, they might require the field to be + single-valued, be required or have a default value (check the + documentation of the field type you're interested in for more information) multiValued: true if this field may contain multiple values per document omitNorms: (expert) set to true to omit the norms associated with this field (this disables length normalization and index-time @@ -139,9 +139,9 @@ - - - + + + @@ -164,7 +164,7 @@ - + @@ -216,7 +216,19 @@ - + + + + + + + + + + + + + @@ -282,6 +294,18 @@ + + + + + + + + + + + + @@ -317,8 +341,8 @@ + + + + + + + + + - These fields support doc values, but they require the field to be - single-valued and either be required or have a default value. + @@ -350,6 +387,8 @@ Smaller precisionStep values (specified in bits) will lead to more tokens indexed per value, slightly larger index size, and faster range queries. A precisionStep of 0 disables indexing at different precision levels. + + Consider using pint/pfloat/plong/pdouble instead of Trie* fields if possible --> @@ -375,12 +414,14 @@ the current day Consult the TrieDateField javadocs for more information. - - Note: For faster range queries, consider the tdate type --> - + + + + - + + From f8831ce3c4c608d4662bd2179454f7992de4bea6 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Thu, 16 Mar 2017 14:18:43 -0400 Subject: [PATCH 024/563] Fixed typos in CHANGES.txt --- solr/CHANGES.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index d49b439018f..7ec0c5298e8 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -112,8 +112,8 @@ Upgrade Notes number of requests. New Codahale Metrics implementation applies exponential decay to this value, which heavily biases the average towards the last 5 minutes. (ab) -* SOLR-8593: Parallel SQL now uses Apache Calcite as it's SQL framework. As part of this change - the default aggregation mode has been changed to facet rather map_reduce. There has also beeen changes +* SOLR-8593: Parallel SQL now uses Apache Calcite as its SQL framework. As part of this change + the default aggregation mode has been changed to facet rather than map_reduce. There has also been changes to the SQL aggregate response and some SQL syntax changes. Consult the documentation for full details. From 2502af9f3fa25a1b724400af61bf74102f2475dd Mon Sep 17 00:00:00 2001 From: David Smiley Date: Thu, 16 Mar 2017 14:58:59 -0400 Subject: [PATCH 025/563] SOLR-10286: large fields. And refactored FieldType.checkSchemaField to call a new checkSupportsDocValues() --- solr/CHANGES.txt | 6 + .../apache/solr/schema/ICUCollationField.java | 5 +- .../component/RealTimeGetComponent.java | 36 ++- .../transform/BaseEditorialTransformer.java | 36 +-- .../org/apache/solr/schema/BinaryField.java | 9 + .../org/apache/solr/schema/BoolField.java | 3 - .../apache/solr/schema/CollationField.java | 5 +- .../org/apache/solr/schema/EnumField.java | 7 - .../apache/solr/schema/FieldProperties.java | 39 +-- .../org/apache/solr/schema/FieldType.java | 20 +- .../solr/schema/LatLonPointSpatialField.java | 3 +- .../org/apache/solr/schema/PointField.java | 4 - .../solr/schema/PrimitiveFieldType.java | 4 + .../org/apache/solr/schema/SchemaField.java | 7 +- .../java/org/apache/solr/schema/StrField.java | 4 - .../org/apache/solr/schema/TrieField.java | 3 - .../apache/solr/search/SolrIndexSearcher.java | 268 ++++++++++++------ .../apache/solr/update/DocumentBuilder.java | 7 +- .../conf/schema-unifiedhighlight.xml | 4 +- .../conf/solrconfig-managed-schema.xml | 2 +- .../solr/cloud/TestRandomFlRTGCloud.java | 2 +- .../highlight/TestUnifiedSolrHighlighter.java | 12 +- .../solr/schema/SortableBinaryField.java | 3 +- .../apache/solr/search/LargeFieldTest.java | 141 +++++++++ .../java/org/apache/solr/SolrTestCaseJ4.java | 2 +- 25 files changed, 446 insertions(+), 186 deletions(-) create mode 100644 solr/core/src/test/org/apache/solr/search/LargeFieldTest.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 7ec0c5298e8..af151eda840 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -197,6 +197,12 @@ New Features * SOLR-10254: significantTerms Streaming Expression should work in non-SolrCloud mode (Joel Bernstein) +* SOLR-10286: string/text fields may now declare themselves as large="true" in the schema. Large fields are always + lazy loaded and will only take up space in the document cache if the actual value is < 512KB. This option + requires "stored" and must not be multiValued. It's intended for fields that might have very large values so that + they don't get cached in memory. (David Smiley) + + Bug Fixes ---------------------- diff --git a/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java b/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java index 7d9e1c7c667..79cb6b38d3e 100644 --- a/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java +++ b/solr/contrib/analysis-extras/src/java/org/apache/solr/schema/ICUCollationField.java @@ -277,10 +277,9 @@ public class ICUCollationField extends FieldType { return new TermRangeQuery(field.getName(), low, high, minInclusive, maxInclusive); } } - + @Override - public void checkSchemaField(SchemaField field) { - // no-op + protected void checkSupportsDocValues() { // we support DocValues } @Override diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java index 123abeacc33..900c787e406 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java @@ -690,23 +690,51 @@ public class RealTimeGetComponent extends SearchComponent List vals = new ArrayList<>(); if (f.fieldType().docValuesType() == DocValuesType.SORTED_NUMERIC) { // SORTED_NUMERICS store sortable bits version of the value, need to retrieve the original - vals.add(sf.getType().toObject(f)); + vals.add(sf.getType().toObject(f)); // (will materialize by side-effect) } else { - vals.add( f ); + vals.add( materialize(f) ); } out.setField( f.name(), vals ); } else{ - out.setField( f.name(), f ); + out.setField( f.name(), materialize(f) ); } } else { - out.addField( f.name(), f ); + out.addField( f.name(), materialize(f) ); } } return out; } + /** + * Ensure we don't have {@link org.apache.lucene.document.LazyDocument.LazyField} or equivalent. + * It can pose problems if the searcher is about to be closed and we haven't fetched a value yet. + */ + private static IndexableField materialize(IndexableField in) { + if (in instanceof Field) { // already materialized + return in; + } + return new ClonedField(in); + } + + private static class ClonedField extends Field { // TODO Lucene Field has no copy constructor; maybe it should? + ClonedField(IndexableField in) { + super(in.name(), in.fieldType()); + this.fieldsData = in.numericValue(); + if (this.fieldsData == null) { + this.fieldsData = in.binaryValue(); + if (this.fieldsData == null) { + this.fieldsData = in.stringValue(); + if (this.fieldsData == null) { + // fallback: + assert false : in; // unexpected + } + } + } + } + } + /** * Converts a SolrInputDocument to SolrDocument, using an IndexSchema instance. * @lucene.experimental diff --git a/solr/core/src/java/org/apache/solr/response/transform/BaseEditorialTransformer.java b/solr/core/src/java/org/apache/solr/response/transform/BaseEditorialTransformer.java index 80e0b61373c..bc5fb65d637 100644 --- a/solr/core/src/java/org/apache/solr/response/transform/BaseEditorialTransformer.java +++ b/solr/core/src/java/org/apache/solr/response/transform/BaseEditorialTransformer.java @@ -17,12 +17,12 @@ package org.apache.solr.response.transform; -import org.apache.lucene.document.Field; +import java.util.Set; + +import org.apache.lucene.index.IndexableField; import org.apache.solr.common.SolrDocument; import org.apache.solr.schema.FieldType; -import java.util.Set; - /** * * @@ -40,8 +40,7 @@ public abstract class BaseEditorialTransformer extends DocTransformer { } @Override - public String getName() - { + public String getName() { return name; } @@ -61,22 +60,15 @@ public abstract class BaseEditorialTransformer extends DocTransformer { protected abstract Set getIdSet(); protected String getKey(SolrDocument doc) { - String key; - Object field = doc.get(idFieldName); - final Number n; - if (field instanceof Field) { - n = ((Field) field).numericValue(); - } else { - n = null; + Object obj = doc.get(idFieldName); + if (obj instanceof IndexableField) { + IndexableField f = (IndexableField) obj; + Number n = f.numericValue(); + if (n != null) { + return ft.readableToIndexed(n.toString()); + } + return ft.readableToIndexed(f.stringValue()); } - if (n != null) { - key = n.toString(); - key = ft.readableToIndexed(key); - } else if (field instanceof Field){ - key = ((Field)field).stringValue(); - } else { - key = field.toString(); - } - return key; + throw new AssertionError("Expected an IndexableField but got: " + obj.getClass()); } -} +} \ No newline at end of file diff --git a/solr/core/src/java/org/apache/solr/schema/BinaryField.java b/solr/core/src/java/org/apache/solr/schema/BinaryField.java index d1882b12fb3..a0e7d051a13 100644 --- a/solr/core/src/java/org/apache/solr/schema/BinaryField.java +++ b/solr/core/src/java/org/apache/solr/schema/BinaryField.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import org.apache.lucene.index.IndexableField; import org.apache.lucene.search.SortField; import org.apache.lucene.util.BytesRef; +import org.apache.solr.common.SolrException; import org.apache.solr.common.util.Base64; import org.apache.solr.response.TextResponseWriter; import org.apache.solr.uninverting.UninvertingReader.Type; @@ -34,6 +35,14 @@ public class BinaryField extends FieldType { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + @Override + public void checkSchemaField(SchemaField field) { + super.checkSchemaField(field); + if (field.isLarge()) { + throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Field type " + this + " is 'large'; not supported (yet)"); + } + } + private String toBase64String(ByteBuffer buf) { return Base64.byteArrayToBase64(buf.array(), buf.position(), buf.limit()-buf.position()); } diff --git a/solr/core/src/java/org/apache/solr/schema/BoolField.java b/solr/core/src/java/org/apache/solr/schema/BoolField.java index 7d5c0d71ebf..607e384c94a 100644 --- a/solr/core/src/java/org/apache/solr/schema/BoolField.java +++ b/solr/core/src/java/org/apache/solr/schema/BoolField.java @@ -207,9 +207,6 @@ public class BoolField extends PrimitiveFieldType { return Collections.singletonList(fval); } - @Override - public void checkSchemaField(final SchemaField field) { - } } // TODO - this can be much more efficient - use FixedBitSet or Bits diff --git a/solr/core/src/java/org/apache/solr/schema/CollationField.java b/solr/core/src/java/org/apache/solr/schema/CollationField.java index a498c0163f0..c6a4f81a143 100644 --- a/solr/core/src/java/org/apache/solr/schema/CollationField.java +++ b/solr/core/src/java/org/apache/solr/schema/CollationField.java @@ -247,10 +247,9 @@ public class CollationField extends FieldType { return new TermRangeQuery(field.getName(), low, high, minInclusive, maxInclusive); } } - + @Override - public void checkSchemaField(SchemaField field) { - // no-op + protected void checkSupportsDocValues() { // we support DocValues } @Override diff --git a/solr/core/src/java/org/apache/solr/schema/EnumField.java b/solr/core/src/java/org/apache/solr/schema/EnumField.java index 2e73f746125..3127262548a 100644 --- a/solr/core/src/java/org/apache/solr/schema/EnumField.java +++ b/solr/core/src/java/org/apache/solr/schema/EnumField.java @@ -277,13 +277,6 @@ public class EnumField extends PrimitiveFieldType { return query; } - /** - * {@inheritDoc} - */ - @Override - public void checkSchemaField(SchemaField field) { - } - /** * {@inheritDoc} */ diff --git a/solr/core/src/java/org/apache/solr/schema/FieldProperties.java b/solr/core/src/java/org/apache/solr/schema/FieldProperties.java index 2b1a8bbe654..6762345c2e8 100644 --- a/solr/core/src/java/org/apache/solr/schema/FieldProperties.java +++ b/solr/core/src/java/org/apache/solr/schema/FieldProperties.java @@ -30,29 +30,30 @@ public abstract class FieldProperties { // many of the variables are independent or semi-independent. // bit values for boolean field properties. - protected final static int INDEXED = 0x00000001; - protected final static int TOKENIZED = 0x00000002; - protected final static int STORED = 0x00000004; - protected final static int BINARY = 0x00000008; - protected final static int OMIT_NORMS = 0x00000010; - protected final static int OMIT_TF_POSITIONS = 0x00000020; - protected final static int STORE_TERMVECTORS = 0x00000040; - protected final static int STORE_TERMPOSITIONS = 0x00000080; - protected final static int STORE_TERMOFFSETS = 0x00000100; + protected final static int INDEXED = 0b1; + protected final static int TOKENIZED = 0b10; + protected final static int STORED = 0b100; + protected final static int BINARY = 0b1000; + protected final static int OMIT_NORMS = 0b10000; + protected final static int OMIT_TF_POSITIONS = 0b100000; + protected final static int STORE_TERMVECTORS = 0b1000000; + protected final static int STORE_TERMPOSITIONS = 0b10000000; + protected final static int STORE_TERMOFFSETS = 0b100000000; - protected final static int MULTIVALUED = 0x00000200; - protected final static int SORT_MISSING_FIRST = 0x00000400; - protected final static int SORT_MISSING_LAST = 0x00000800; + protected final static int MULTIVALUED = 0b1000000000; + protected final static int SORT_MISSING_FIRST = 0b10000000000; + protected final static int SORT_MISSING_LAST = 0b100000000000; - protected final static int REQUIRED = 0x00001000; - protected final static int OMIT_POSITIONS = 0x00002000; + protected final static int REQUIRED = 0b1000000000000; + protected final static int OMIT_POSITIONS = 0b10000000000000; - protected final static int STORE_OFFSETS = 0x00004000; - protected final static int DOC_VALUES = 0x00008000; + protected final static int STORE_OFFSETS = 0b100000000000000; + protected final static int DOC_VALUES = 0b1000000000000000; - protected final static int STORE_TERMPAYLOADS = 0x00010000; - protected final static int USE_DOCVALUES_AS_STORED = 0x00020000; + protected final static int STORE_TERMPAYLOADS = 0b10000000000000000; + protected final static int USE_DOCVALUES_AS_STORED = 0b100000000000000000; + protected final static int LARGE_FIELD = 0b1000000000000000000; static final String[] propertyNames = { "indexed", "tokenized", "stored", @@ -60,7 +61,7 @@ public abstract class FieldProperties { "termVectors", "termPositions", "termOffsets", "multiValued", "sortMissingFirst","sortMissingLast","required", "omitPositions", - "storeOffsetsWithPositions", "docValues", "termPayloads", "useDocValuesAsStored" + "storeOffsetsWithPositions", "docValues", "termPayloads", "useDocValuesAsStored", "large" }; static final Map propertyMap = new HashMap<>(); diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java index 67b7be7feb1..016e1666a9c 100644 --- a/solr/core/src/java/org/apache/solr/schema/FieldType.java +++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java @@ -791,17 +791,27 @@ public abstract class FieldType extends FieldProperties { * *

* This method is called by the SchemaField constructor to - * check that its initialization does not violate any fundemental - * requirements of the FieldType. The default implementation - * does nothing, but subclasses may chose to throw a {@link SolrException} + * check that its initialization does not violate any fundamental + * requirements of the FieldType. + * Subclasses may choose to throw a {@link SolrException} * if invariants are violated by the SchemaField. *

*/ public void checkSchemaField(final SchemaField field) { - // override if your field type supports doc values if (field.hasDocValues()) { - throw new SolrException(ErrorCode.SERVER_ERROR, "Field type " + this + " does not support doc values"); + checkSupportsDocValues(); } + if (field.isLarge() && field.multiValued()) { + throw new SolrException(ErrorCode.SERVER_ERROR, "Field type " + this + " is 'large'; can't support multiValued"); + } + if (field.isLarge() && getNumberType() != null) { + throw new SolrException(ErrorCode.SERVER_ERROR, "Field type " + this + " is 'large'; can't support numerics"); + } + } + + /** Called by {@link #checkSchemaField(SchemaField)} if the field has docValues. By default none do. */ + protected void checkSupportsDocValues() { + throw new SolrException(ErrorCode.SERVER_ERROR, "Field type " + this + " does not support doc values"); } public static final String TYPE = "type"; diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java index c09856a0f67..8ed54841d41 100644 --- a/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java +++ b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java @@ -57,8 +57,7 @@ public class LatLonPointSpatialField extends AbstractSpatialFieldType implements // TODO handle polygons @Override - public void checkSchemaField(SchemaField field) { - // override because if we didn't, FieldType will complain about docValues not being supported (we do support it) + protected void checkSupportsDocValues() { // we support DocValues } @Override diff --git a/solr/core/src/java/org/apache/solr/schema/PointField.java b/solr/core/src/java/org/apache/solr/schema/PointField.java index 6c75105ead4..91d3eff1b0d 100644 --- a/solr/core/src/java/org/apache/solr/schema/PointField.java +++ b/solr/core/src/java/org/apache/solr/schema/PointField.java @@ -244,8 +244,4 @@ public abstract class PointField extends NumericFieldType { protected abstract StoredField getStoredField(SchemaField sf, Object value); - @Override - public void checkSchemaField(final SchemaField field) { - // PointFields support DocValues - } } diff --git a/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java b/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java index a920f350beb..9d9da4709aa 100644 --- a/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java +++ b/solr/core/src/java/org/apache/solr/schema/PrimitiveFieldType.java @@ -33,4 +33,8 @@ public abstract class PrimitiveFieldType extends FieldType { properties |= OMIT_NORMS; } } + + @Override + protected void checkSupportsDocValues() { // primitive types support DocValues + } } diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaField.java b/solr/core/src/java/org/apache/solr/schema/SchemaField.java index e690a136bd8..1e18ee053dc 100644 --- a/solr/core/src/java/org/apache/solr/schema/SchemaField.java +++ b/solr/core/src/java/org/apache/solr/schema/SchemaField.java @@ -66,7 +66,7 @@ public final class SchemaField extends FieldProperties implements IndexableField args = prototype.args; } - /** Create a new SchemaField with the given name and type, + /** Create a new SchemaField with the given name and type, * and with the specified properties. Properties are *not* * inherited from the type in this case, so users of this * constructor should derive the properties from type.getSolrProperties() @@ -106,7 +106,8 @@ public final class SchemaField extends FieldProperties implements IndexableField public boolean multiValued() { return (properties & MULTIVALUED)!=0; } public boolean sortMissingFirst() { return (properties & SORT_MISSING_FIRST)!=0; } public boolean sortMissingLast() { return (properties & SORT_MISSING_LAST)!=0; } - public boolean isRequired() { return required; } + public boolean isRequired() { return required; } + public boolean isLarge() { return (properties & LARGE_FIELD)!=0;} public Map getArgs() { return Collections.unmodifiableMap(args); } // things that should be determined by field type, not set as options @@ -240,7 +241,7 @@ public final class SchemaField extends FieldProperties implements IndexableField // that depend on that. // if (on(falseProps,STORED)) { - int pp = STORED | BINARY; + int pp = STORED | BINARY | LARGE_FIELD; if (on(pp,trueProps)) { throw new RuntimeException("SchemaField: " + name + " conflicting stored field options:" + props); } diff --git a/solr/core/src/java/org/apache/solr/schema/StrField.java b/solr/core/src/java/org/apache/solr/schema/StrField.java index 0b1576b11a6..3294b0484ea 100644 --- a/solr/core/src/java/org/apache/solr/schema/StrField.java +++ b/solr/core/src/java/org/apache/solr/schema/StrField.java @@ -95,10 +95,6 @@ public class StrField extends PrimitiveFieldType { return term.utf8ToString(); } - @Override - public void checkSchemaField(SchemaField field) { - } - @Override public Object marshalSortValue(Object value) { return marshalStringSortValue(value); diff --git a/solr/core/src/java/org/apache/solr/schema/TrieField.java b/solr/core/src/java/org/apache/solr/schema/TrieField.java index 46240c9c4db..e7a33bdb64f 100644 --- a/solr/core/src/java/org/apache/solr/schema/TrieField.java +++ b/solr/core/src/java/org/apache/solr/schema/TrieField.java @@ -690,9 +690,6 @@ public class TrieField extends NumericFieldType { return null; } - @Override - public void checkSchemaField(final SchemaField field) { - } } class TrieDateFieldSource extends LongFieldSource { 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 19e47d0a022..83df60f2a42 100644 --- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java +++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java @@ -18,6 +18,7 @@ package org.apache.solr.search; import java.io.Closeable; import java.io.IOException; +import java.io.Reader; import java.lang.invoke.MethodHandles; import java.net.URL; import java.nio.charset.StandardCharsets; @@ -38,63 +39,16 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import com.google.common.collect.Iterables; +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.document.Document; import org.apache.lucene.document.DocumentStoredFieldVisitor; import org.apache.lucene.document.LazyDocument; -import org.apache.lucene.index.BinaryDocValues; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.DocValuesType; -import org.apache.lucene.index.ExitableDirectoryReader; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfos; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexableField; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.MultiPostingsEnum; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.PostingsEnum; -import org.apache.lucene.index.ReaderUtil; -import org.apache.lucene.index.SortedDocValues; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.index.SortedSetDocValues; -import org.apache.lucene.index.StoredFieldVisitor; +import org.apache.lucene.index.*; import org.apache.lucene.index.StoredFieldVisitor.Status; -import org.apache.lucene.index.Term; -import org.apache.lucene.index.TermContext; -import org.apache.lucene.index.Terms; -import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.*; import org.apache.lucene.search.BooleanClause.Occur; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.CollectionStatistics; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.ConstantScoreQuery; -import org.apache.lucene.search.DocIdSet; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.EarlyTerminatingSortingCollector; -import org.apache.lucene.search.Explanation; -import org.apache.lucene.search.FieldDoc; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.LeafCollector; -import org.apache.lucene.search.MatchAllDocsQuery; -import org.apache.lucene.search.MultiCollector; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.SimpleCollector; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.TermStatistics; -import org.apache.lucene.search.TimeLimitingCollector; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.TopDocsCollector; -import org.apache.lucene.search.TopFieldCollector; -import org.apache.lucene.search.TopFieldDocs; -import org.apache.lucene.search.TopScoreDocCollector; -import org.apache.lucene.search.TotalHitCountCollector; -import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; @@ -134,8 +88,6 @@ import org.apache.solr.update.SolrIndexConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.Iterables; - /** * SolrIndexSearcher adds schema awareness and caching functionality over {@link IndexSearcher}. * @@ -192,7 +144,12 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI /** Contains the names/patterns of all docValues=true,stored=false fields, excluding those that are copyField targets in the schema. */ private final Set nonStoredDVsWithoutCopyTargets; - private Collection storedHighlightFieldNames; + private static int largeValueLengthCacheThreshold = Integer.getInteger("solr.largeField.cacheThreshold", 512 * 1024); // internal setting + + private final Set largeFields; + + private Collection storedHighlightFieldNames; // lazy populated; use getter + private DirectoryFactory directoryFactory; private final LeafReader leafReader; @@ -204,6 +161,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI private final NamedList readerStats; + private static DirectoryReader getReader(SolrCore core, SolrIndexConfig config, DirectoryFactory directoryFactory, String path) throws IOException { final Directory dir = directoryFactory.get(path, DirContext.DEFAULT, config.lockType); @@ -367,11 +325,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI final Set nonStoredDVsUsedAsStored = new HashSet<>(); final Set allNonStoredDVs = new HashSet<>(); final Set nonStoredDVsWithoutCopyTargets = new HashSet<>(); + final Set storedLargeFields = new HashSet<>(); this.fieldInfos = leafReader.getFieldInfos(); - for (FieldInfo fieldInfo : fieldInfos) { + for (FieldInfo fieldInfo : fieldInfos) { // can find materialized dynamic fields, unlike using the Solr IndexSchema. final SchemaField schemaField = schema.getFieldOrNull(fieldInfo.name); - if (schemaField != null && !schemaField.stored() && schemaField.hasDocValues()) { + if (schemaField == null) { + continue; + } + if (!schemaField.stored() && schemaField.hasDocValues()) { if (schemaField.useDocValuesAsStored()) { nonStoredDVsUsedAsStored.add(fieldInfo.name); } @@ -380,11 +342,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI nonStoredDVsWithoutCopyTargets.add(fieldInfo.name); } } + if (schemaField.stored() && schemaField.isLarge()) { + storedLargeFields.add(schemaField.getName()); + } } this.nonStoredDVsUsedAsStored = Collections.unmodifiableSet(nonStoredDVsUsedAsStored); this.allNonStoredDVs = Collections.unmodifiableSet(allNonStoredDVs); this.nonStoredDVsWithoutCopyTargets = Collections.unmodifiableSet(nonStoredDVsWithoutCopyTargets); + this.largeFields = Collections.unmodifiableSet(storedLargeFields); // We already have our own filter cache setQueryCache(null); @@ -677,26 +643,41 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI /* * Future optimizations (yonik) * - * If no cache is present: - use NO_LOAD instead of LAZY_LOAD - use LOAD_AND_BREAK if a single field is begin + * If no cache is present: - use NO_LOAD instead of LAZY_LOAD - use LOAD_AND_BREAK if a single field is being * retrieved */ - /** FieldSelector which loads the specified fields, and loads all other field lazily. */ - private static class SetNonLazyFieldSelector extends DocumentStoredFieldVisitor { + /** {@link StoredFieldVisitor} which loads the specified fields eagerly (or all if null). + * If {@link #enableLazyFieldLoading} then the rest get special lazy field entries. Designated "large" + * fields will always get a special field entry. */ + private class SolrDocumentStoredFieldVisitor extends DocumentStoredFieldVisitor { private final Document doc; - private final LazyDocument lazyDoc; + private final LazyDocument lazyFieldProducer; // arguably a better name than LazyDocument; at least how we use it here + private final int docId; + private final boolean addLargeFieldsLazily; - SetNonLazyFieldSelector(Set toLoad, IndexReader reader, int docID) { + SolrDocumentStoredFieldVisitor(Set toLoad, IndexReader reader, int docId) { super(toLoad); - lazyDoc = new LazyDocument(reader, docID); - doc = getDocument(); + this.docId = docId; + this.doc = getDocument(); + this.lazyFieldProducer = toLoad != null && enableLazyFieldLoading ? new LazyDocument(reader, docId) : null; + this.addLargeFieldsLazily = (documentCache != null && !largeFields.isEmpty()); + //TODO can we return Status.STOP after a val is loaded and we know there are no other fields of interest? + // When: toLoad is one single-valued field, no lazyFieldProducer } @Override public Status needsField(FieldInfo fieldInfo) throws IOException { Status status = super.needsField(fieldInfo); - if (status == Status.NO) { - doc.add(lazyDoc.getField(fieldInfo)); + assert status != Status.STOP : "Status.STOP not supported or expected"; + if (addLargeFieldsLazily && largeFields.contains(fieldInfo.name)) { // load "large" fields using this lazy mechanism + if (lazyFieldProducer != null || status == Status.YES) { + doc.add(new LargeLazyField(fieldInfo.name, docId)); + } + return Status.NO; + } + if (status == Status.NO && lazyFieldProducer != null) { // lazy + doc.add(lazyFieldProducer.getField(fieldInfo)); } return status; } @@ -717,15 +698,15 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI * @see IndexReader#document(int, StoredFieldVisitor) */ @Override - public void doc(int n, StoredFieldVisitor visitor) throws IOException { + public void doc(int docId, StoredFieldVisitor visitor) throws IOException { if (documentCache != null) { - Document cached = documentCache.get(n); + Document cached = documentCache.get(docId); if (cached != null) { visitFromCached(cached, visitor); return; } } - getIndexReader().document(n, visitor); + getIndexReader().document(docId, visitor); } /** Executes a stored field visitor against a hit from the document cache */ @@ -735,13 +716,13 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI final Status needsField = visitor.needsField(info); if (needsField == Status.STOP) return; if (needsField == Status.NO) continue; - if (f.binaryValue() != null) { - final BytesRef binaryValue = f.binaryValue(); - final byte copy[] = new byte[binaryValue.length]; - System.arraycopy(binaryValue.bytes, binaryValue.offset, copy, 0, copy.length); - visitor.binaryField(info, copy); - } else if (f.numericValue() != null) { - final Number numericValue = f.numericValue(); + BytesRef binaryValue = f.binaryValue(); + if (binaryValue != null) { + visitor.binaryField(info, toByteArrayUnwrapIfPossible(binaryValue)); + continue; + } + Number numericValue = f.numericValue(); + if (numericValue != null) { if (numericValue instanceof Double) { visitor.doubleField(info, numericValue.doubleValue()); } else if (numericValue instanceof Integer) { @@ -753,12 +734,25 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI } else { throw new AssertionError(); } + continue; + } + // must be String + if (f instanceof LargeLazyField) { // optimization to avoid premature string conversion + visitor.stringField(info, toByteArrayUnwrapIfPossible(((LargeLazyField) f).readBytes())); } else { visitor.stringField(info, f.stringValue().getBytes(StandardCharsets.UTF_8)); } } } + private byte[] toByteArrayUnwrapIfPossible(BytesRef bytesRef) { + if (bytesRef.offset == 0 && bytesRef.bytes.length == bytesRef.length) { + return bytesRef.bytes; + } else { + return Arrays.copyOfRange(bytesRef.bytes, bytesRef.offset, bytesRef.offset + bytesRef.length); + } + } + /** * Retrieve the {@link Document} instance corresponding to the document id. *

@@ -775,23 +769,16 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI } final DirectoryReader reader = getIndexReader(); - if (fields != null) { - if (enableLazyFieldLoading) { - final SetNonLazyFieldSelector visitor = new SetNonLazyFieldSelector(fields, reader, i); - reader.document(i, visitor); - d = visitor.doc; - } else if (documentCache == null) { - d = reader.document(i, fields); - } else { - // we do not pass the fields in this case because that would return an incomplete document which would - // be eventually cached. The alternative would be to read the stored fields twice; once with the fields - // and then without for caching leading to a performance hit - // see SOLR-8858 for related discussion - d = reader.document(i); - } - } else { - d = reader.document(i); + if (documentCache != null && !enableLazyFieldLoading) { + // we do not filter the fields in this case because that would return an incomplete document which would + // be eventually cached. The alternative would be to read the stored fields twice; once with the fields + // and then without for caching leading to a performance hit + // see SOLR-8858 for related discussion + fields = null; } + final SolrDocumentStoredFieldVisitor visitor = new SolrDocumentStoredFieldVisitor(fields, reader, i); + reader.document(i, visitor); + d = visitor.getDocument(); if (documentCache != null) { documentCache.put(i, d); @@ -800,6 +787,103 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI return d; } + /** Unlike LazyDocument.LazyField, we (a) don't cache large values, and (b) provide access to the byte[]. */ + class LargeLazyField implements IndexableField { + + final String name; + final int docId; + // synchronize on 'this' to access: + BytesRef cachedBytes; // we only conditionally populate this if it's big enough + + private LargeLazyField(String name, int docId) { + this.name = name; + this.docId = docId; + } + + @Override + public String toString() { + return fieldType().toString() + "<" + name() + ">"; // mimic Field.java + } + + @Override + public String name() { + return name; + } + + @Override + public IndexableFieldType fieldType() { + return schema.getField(name()); + } + + @Override + public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) { + return analyzer.tokenStream(name(), stringValue()); // or we could throw unsupported exception? + } + /** (for tests) */ + synchronized boolean hasBeenLoaded() { + return cachedBytes != null; + } + + @Override + public synchronized String stringValue() { + try { + return readBytes().utf8ToString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + synchronized BytesRef readBytes() throws IOException { + if (cachedBytes != null) { + return cachedBytes; + } else { + BytesRef bytesRef = new BytesRef(); + getIndexReader().document(docId, new StoredFieldVisitor() { + boolean done = false; + @Override + public Status needsField(FieldInfo fieldInfo) throws IOException { + if (done) { + return Status.STOP; + } + return fieldInfo.name.equals(name()) ? Status.YES : Status.NO; + } + + @Override + public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException { + bytesRef.bytes = value; + bytesRef.length = value.length; + done = true; + } + + @Override + public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { + throw new UnsupportedOperationException("'large' binary fields are not (yet) supported"); + } + }); + if (bytesRef.length < largeValueLengthCacheThreshold) { + return cachedBytes = bytesRef; + } else { + return bytesRef; + } + } + } + + @Override + public BytesRef binaryValue() { + return null; + } + + @Override + public Reader readerValue() { + return null; + } + + @Override + public Number numericValue() { + return null; + } + } + /** * This will fetch and add the docValues fields to a given SolrDocument/SolrInputDocument * diff --git a/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java b/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java index abf4a1f1599..e3d20116fae 100644 --- a/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java +++ b/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Set; import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.IndexableField; import org.apache.solr.common.SolrException; @@ -55,7 +54,7 @@ public class DocumentBuilder { assert val instanceof NumericDocValuesField: "Expected in-place update to be done on" + " NDV fields only."; } - doc.add((Field)val); + doc.add((IndexableField)val); return; } for (IndexableField f : field.getType().createFields(field, val)) { @@ -69,10 +68,10 @@ public class DocumentBuilder { // assert f instanceof NumericDocValuesField if (forInPlaceUpdate) { if (f instanceof NumericDocValuesField) { - doc.add((Field) f); + doc.add(f); } } else { - doc.add((Field) f); + doc.add(f); } } } diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-unifiedhighlight.xml b/solr/core/src/test-files/solr/collection1/conf/schema-unifiedhighlight.xml index ab18354e9aa..8d710889430 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-unifiedhighlight.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-unifiedhighlight.xml @@ -17,7 +17,7 @@ - + @@ -39,7 +39,7 @@ - + text id diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml index abd4fbec25f..26224ad9b79 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml @@ -24,7 +24,7 @@ ${managed.schema.mutable} - managed-schema + ${managed.schema.resourceName:managed-schema} diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java b/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java index 966d8ef75c3..140fd7ec7ce 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestRandomFlRTGCloud.java @@ -160,7 +160,7 @@ public class TestRandomFlRTGCloud extends SolrCloudTestCase { } /** - * Tests thta all TransformerFactories that are implicitly provided by Solr are tested in this class + * Tests that all TransformerFactories that are implicitly provided by Solr are tested in this class * * @see FlValidator#getDefaultTransformerFactoryName * @see #FL_VALIDATORS diff --git a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java index 98355189997..2f7a003c88a 100644 --- a/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java +++ b/solr/core/src/test/org/apache/solr/highlight/TestUnifiedSolrHighlighter.java @@ -19,6 +19,7 @@ package org.apache.solr.highlight; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.schema.IndexSchema; +import org.junit.AfterClass; import org.junit.BeforeClass; /** Tests for the UnifiedHighlighter Solr plugin **/ @@ -26,7 +27,10 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 { @BeforeClass public static void beforeClass() throws Exception { - initCore("solrconfig-basic.xml", "schema-unifiedhighlight.xml"); + System.setProperty("filterCache.enabled", "false"); + System.setProperty("queryResultCache.enabled", "false"); + System.setProperty("documentCache.enabled", "true"); // this is why we use this particular solrconfig + initCore("solrconfig-cache-enable-disable.xml", "schema-unifiedhighlight.xml"); // test our config is sane, just to be sure: @@ -36,6 +40,12 @@ public class TestUnifiedSolrHighlighter extends SolrTestCaseJ4 { assertTrue(schema.getField("text3").storeOffsetsWithPositions()); assertFalse(schema.getField("text2").storeOffsetsWithPositions()); } + @AfterClass + public static void afterClass() { + System.clearProperty("filterCache.enabled"); + System.clearProperty("queryResultCache.enabled"); + System.clearProperty("documentCache.enabled"); + } @Override public void setUp() throws Exception { diff --git a/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java b/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java index b8ed296537c..90c92d07cc6 100644 --- a/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java +++ b/solr/core/src/test/org/apache/solr/schema/SortableBinaryField.java @@ -35,8 +35,7 @@ import org.apache.lucene.util.BytesRef; public class SortableBinaryField extends BinaryField { @Override - public void checkSchemaField(final SchemaField field) { - // NOOP, It's Aaaaaall Good. + protected void checkSupportsDocValues() { // we support DocValues } @Override diff --git a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java new file mode 100644 index 00000000000..d05c69cec6a --- /dev/null +++ b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java @@ -0,0 +1,141 @@ +/* + * 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.search; + +import java.util.Arrays; +import java.util.Collections; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.LazyDocument; +import org.apache.lucene.index.IndexableField; +import org.apache.solr.SolrTestCaseJ4; +import org.apache.solr.common.util.ContentStreamBase; +import org.apache.solr.request.SolrQueryRequestBase; +import org.apache.solr.response.SolrQueryResponse; +import org.apache.solr.schema.IndexSchema; +import org.apache.solr.util.RefCounted; +import org.junit.BeforeClass; +import org.junit.Test; + +public class LargeFieldTest extends SolrTestCaseJ4 { + + private static final String ID_FLD = "str"; // TODO alter underlying schema to be 'id' + private static final String LAZY_FIELD = "lazyField"; + private static final String BIG_FIELD = "bigField"; + + @BeforeClass + public static void initManagedSchemaCore() throws Exception { + // TODO propose convenience API for this? + // This testing approach means no new solrconfig or schema file or per-test temp solr-home! + System.setProperty("managed.schema.mutable", "true"); + System.setProperty("managed.schema.resourceName", "schema-one-field-no-dynamic-field-unique-key.xml"); + System.setProperty("enable.update.log", "false"); + initCore("solrconfig-managed-schema.xml", "ignoredSchemaName?"); + + // modify solr config TODO propose more convenient API for this; maybe with JSON-ification of a map + try (SolrQueryRequestBase req = (SolrQueryRequestBase) req()) { + req.getContext().put("httpMethod", "POST"); + req.setContentStreams(Collections.singleton(new ContentStreamBase.StringStream( + "{ 'set-property':{" + + "'query.enableLazyFieldLoading':true, " + + "'query.documentCache.class':'solr.LRUCache'" + + "}}" + ))); + SolrQueryResponse rsp = new SolrQueryResponse(); + h.getCore().execute(h.getCore().getRequestHandler("/config"), req, rsp); + assertNull(rsp.getException()); + } + + boolean PERSIST_FALSE = false; // don't write to test resource dir + + IndexSchema schema = h.getCore().getLatestSchema(); + schema = schema.addFieldTypes(Collections.singletonList( + schema.newFieldType("textType", "solr.TextField", // redundant; TODO improve api + map("name", "textType", "class", "solr.TextField", + "analyzer", map("class", "org.apache.lucene.analysis.standard.StandardAnalyzer")))), + PERSIST_FALSE); + schema = schema.addFields(Arrays.asList( + schema.newField(LAZY_FIELD, "textType", map()), + schema.newField(BIG_FIELD, "textType", map("large", true))), + Collections.emptyMap(), + PERSIST_FALSE); + + h.getCore().setLatestSchema(schema); + } + + @Test + public void test() throws Exception { + // add just one document (docid 0) + assertU(adoc(ID_FLD, "101", LAZY_FIELD, "lzy", BIG_FIELD, "big document field one")); + assertU(commit()); + + // trigger the ID_FLD to get into the doc cache; don't reference other fields + assertQ(req("q", "101", "df", ID_FLD, "fl", ID_FLD)); // eager load ID_FLD; rest are lazy + + // fetch the document; we know it will be from the documentCache, docId 0 + final Document d; + RefCounted searcherRef = h.getCore().getSearcher(); + try { + d = searcherRef.get().doc(0); + } finally { + searcherRef.decref(); + } + + assertEager(d, ID_FLD); + assertLazyNotLoaded(d, LAZY_FIELD); + assertLazyNotLoaded(d, BIG_FIELD); + + assertQ(req("q", "101", "df", ID_FLD, "fl", LAZY_FIELD)); // trigger load of LAZY_FIELD + + assertEager(d, ID_FLD); + assertLazyLoaded(d, LAZY_FIELD); // loaded now + assertLazyNotLoaded(d, BIG_FIELD); // because big fields are handled separately + + assertQ(req("q", "101", "df", ID_FLD, "fl", BIG_FIELD)); // trigger load of BIG_FIELD + + assertEager(d, ID_FLD); + assertLazyLoaded(d, LAZY_FIELD); + assertLazyLoaded(d, BIG_FIELD); // loaded now + } + + private void assertEager(Document d, String fieldName) { + assertFalse( d.getField(fieldName) instanceof LazyDocument.LazyField); + } + + private void assertLazyNotLoaded(Document d, String fieldName) { + IndexableField field = d.getField(fieldName); + if (fieldName == BIG_FIELD) { + assertTrue(field instanceof SolrIndexSearcher.LargeLazyField); + assertFalse(((SolrIndexSearcher.LargeLazyField)field).hasBeenLoaded()); + } else { + assertTrue(field instanceof LazyDocument.LazyField); + assertFalse(((LazyDocument.LazyField)field).hasBeenLoaded()); + } + } + + private void assertLazyLoaded(Document d, String fieldName) { + IndexableField field = d.getField(fieldName); + if (fieldName == BIG_FIELD) { + assertTrue(field instanceof SolrIndexSearcher.LargeLazyField); + assertTrue(((SolrIndexSearcher.LargeLazyField)field).hasBeenLoaded()); + } else { + assertTrue(field instanceof LazyDocument.LazyField); + assertTrue(((LazyDocument.LazyField)field).hasBeenLoaded()); + } + } +} diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java index 825e7c762b0..04fc3ffc4c8 100644 --- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java +++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java @@ -852,7 +852,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase { /** Validates a query matches some XPath test expressions and closes the query */ public static void assertQ(String message, SolrQueryRequest req, String... tests) { try { - String m = (null == message) ? "" : message + " "; + String m = (null == message) ? "" : message + " "; // TODO log 'm' !!! String response = h.query(req); if (req.getParams().getBool("facet", false)) { From 14397949006723d907d4ee681ff2802410aa19ab Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Thu, 16 Mar 2017 15:10:48 -0700 Subject: [PATCH 026/563] SOLR-9990: Avoid copyField in SolrExampleTests.testUpdateField --- .../solr/client/solrj/SolrExampleTests.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java index b1e728511f9..326dede7e97 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java @@ -1613,16 +1613,16 @@ abstract public class SolrExampleTests extends SolrExampleTestsBase SolrInputDocument doc = new SolrInputDocument(); doc.addField("id", "unique"); doc.addField("name", "gadget"); - doc.addField("price_f", 1); + doc.addField("price", 1); client.add(doc); client.commit(); SolrQuery q = new SolrQuery("*:*"); - q.setFields("id","price_f","name", "_version_"); + q.setFields("id","price","name", "_version_"); QueryResponse resp = client.query(q); assertEquals("Doc count does not match", 1, resp.getResults().getNumFound()); Long version = (Long)resp.getResults().get(0).getFirstValue("_version_"); assertNotNull("no version returned", version); - assertEquals(1.0f, resp.getResults().get(0).getFirstValue("price_f")); + assertEquals(1.0f, resp.getResults().get(0).getFirstValue("price")); //update "price" with incorrect version (optimistic locking) HashMap oper = new HashMap<>(); //need better api for this??? @@ -1631,7 +1631,7 @@ abstract public class SolrExampleTests extends SolrExampleTestsBase doc = new SolrInputDocument(); doc.addField("id", "unique"); doc.addField("_version_", version+1); - doc.addField("price_f", oper); + doc.addField("price", oper); try { client.add(doc); if(client instanceof HttpSolrClient) { //XXX concurrent client reports exceptions differently @@ -1650,24 +1650,24 @@ abstract public class SolrExampleTests extends SolrExampleTestsBase doc = new SolrInputDocument(); doc.addField("id", "unique"); doc.addField("_version_", version); - doc.addField("price_f", oper); + doc.addField("price", oper); client.add(doc); client.commit(); resp = client.query(q); assertEquals("Doc count does not match", 1, resp.getResults().getNumFound()); - assertEquals("price was not updated?", 100.0f, resp.getResults().get(0).getFirstValue("price_f")); + assertEquals("price was not updated?", 100.0f, resp.getResults().get(0).getFirstValue("price")); assertEquals("no name?", "gadget", resp.getResults().get(0).getFirstValue("name")); //update "price", no version oper.put("set", 200); doc = new SolrInputDocument(); doc.addField("id", "unique"); - doc.addField("price_f", oper); + doc.addField("price", oper); client.add(doc); client.commit(); resp = client.query(q); assertEquals("Doc count does not match", 1, resp.getResults().getNumFound()); - assertEquals("price was not updated?", 200.0f, resp.getResults().get(0).getFirstValue("price_f")); + assertEquals("price was not updated?", 200.0f, resp.getResults().get(0).getFirstValue("price")); assertEquals("no name?", "gadget", resp.getResults().get(0).getFirstValue("name")); } From 4ee7fc38907a94f025785ebd388dd372b260913d Mon Sep 17 00:00:00 2001 From: David Smiley Date: Thu, 16 Mar 2017 18:30:57 -0400 Subject: [PATCH 027/563] SOLR-10286: fix test; we were writing to read-only dir. Expand solrconfig-managed-schema.xml to have toggle-able elements vis system property flags --- .../conf/solrconfig-managed-schema.xml | 25 ++++++++++++++++ .../apache/solr/search/LargeFieldTest.java | 29 ++++++++----------- 2 files changed, 37 insertions(+), 17 deletions(-) diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml index 26224ad9b79..c1390c8f8c1 100644 --- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml +++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-managed-schema.xml @@ -29,6 +29,31 @@ + + + + + + + + ${enableLazyFieldLoading:false} + + ${solr.ulog.dir:} diff --git a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java index d05c69cec6a..09e7e9034de 100644 --- a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java +++ b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java @@ -29,6 +29,7 @@ import org.apache.solr.request.SolrQueryRequestBase; import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.schema.IndexSchema; import org.apache.solr.util.RefCounted; +import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -40,29 +41,17 @@ public class LargeFieldTest extends SolrTestCaseJ4 { @BeforeClass public static void initManagedSchemaCore() throws Exception { - // TODO propose convenience API for this? - // This testing approach means no new solrconfig or schema file or per-test temp solr-home! + // This testing approach means no schema file or per-test temp solr-home! System.setProperty("managed.schema.mutable", "true"); System.setProperty("managed.schema.resourceName", "schema-one-field-no-dynamic-field-unique-key.xml"); System.setProperty("enable.update.log", "false"); + System.setProperty("documentCache.enabled", "true"); + System.setProperty("enableLazyFieldLoading", "true"); + initCore("solrconfig-managed-schema.xml", "ignoredSchemaName?"); - // modify solr config TODO propose more convenient API for this; maybe with JSON-ification of a map - try (SolrQueryRequestBase req = (SolrQueryRequestBase) req()) { - req.getContext().put("httpMethod", "POST"); - req.setContentStreams(Collections.singleton(new ContentStreamBase.StringStream( - "{ 'set-property':{" + - "'query.enableLazyFieldLoading':true, " + - "'query.documentCache.class':'solr.LRUCache'" + - "}}" - ))); - SolrQueryResponse rsp = new SolrQueryResponse(); - h.getCore().execute(h.getCore().getRequestHandler("/config"), req, rsp); - assertNull(rsp.getException()); - } - + // TODO SOLR-10229 will make this easier boolean PERSIST_FALSE = false; // don't write to test resource dir - IndexSchema schema = h.getCore().getLatestSchema(); schema = schema.addFieldTypes(Collections.singletonList( schema.newFieldType("textType", "solr.TextField", // redundant; TODO improve api @@ -78,6 +67,12 @@ public class LargeFieldTest extends SolrTestCaseJ4 { h.getCore().setLatestSchema(schema); } + @AfterClass + public static void afterClass() { + System.clearProperty("documentCache.enabled"); + System.clearProperty("enableLazyFieldLoading"); + } + @Test public void test() throws Exception { // add just one document (docid 0) From d1b2fb33ef3bc0ced65feb98c31cffe4f209da7f Mon Sep 17 00:00:00 2001 From: Steve Rowe Date: Thu, 16 Mar 2017 19:41:37 -0400 Subject: [PATCH 028/563] SOLR-9185: Solr's edismax and Lucene/standard query parsers should optionally not split on whitespace before sending terms to analysis --- solr/CHANGES.txt | 7 + .../org/apache/solr/parser/QueryParser.java | 421 ++++++-- .../org/apache/solr/parser/QueryParser.jj | 276 +++-- .../solr/parser/SolrQueryParserBase.java | 213 +++- .../solr/search/ExtendedDismaxQParser.java | 175 ++- .../org/apache/solr/search/LuceneQParser.java | 3 + .../solr/search/LuceneQParserPlugin.java | 2 + .../QueryParserConfigurationException.java | 24 + .../org/apache/solr/search/QueryParsing.java | 1 + .../apache/solr/search/SolrQueryParser.java | 2 +- .../collection1/conf/multiword-synonyms.txt | 13 + .../conf/schema-multiword-synonyms.xml | 50 + .../solr/collection1/conf/synonyms.txt | 2 + .../solr/search/TestExtendedDismaxParser.java | 992 +++++++++++++----- .../solr/search/TestMultiWordSynonyms.java | 100 ++ .../solr/search/TestSolrQueryParser.java | 579 ++++++++-- 16 files changed, 2257 insertions(+), 603 deletions(-) create mode 100644 solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java create mode 100644 solr/core/src/test-files/solr/collection1/conf/multiword-synonyms.txt create mode 100644 solr/core/src/test-files/solr/collection1/conf/schema-multiword-synonyms.xml create mode 100644 solr/core/src/test/org/apache/solr/search/TestMultiWordSynonyms.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index af151eda840..9a5299ccd29 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -202,6 +202,13 @@ New Features requires "stored" and must not be multiValued. It's intended for fields that might have very large values so that they don't get cached in memory. (David Smiley) +* SOLR-9185: Solr's edismax and "Lucene"/standard query parsers will no longer split on whitespace before sending + terms to analysis, if given the "sow=false" request param ("sow"=>"split on whitespace"). This enables multi-term + source synonyms to match at query-time using SynonymGraphFilterFactory; other analysis components will also now + work at query time, e.g. ShingleFilterFactory. By default, and when the "sow=true" param is specified, these + parsers' behavior remains the same: queries will be split on whitespace before sending individual terms to analysis. + (Steve Rowe) + Bug Fixes ---------------------- diff --git a/solr/core/src/java/org/apache/solr/parser/QueryParser.java b/solr/core/src/java/org/apache/solr/parser/QueryParser.java index 42f359ee811..d9a64f4f7c0 100644 --- a/solr/core/src/java/org/apache/solr/parser/QueryParser.java +++ b/solr/core/src/java/org/apache/solr/parser/QueryParser.java @@ -3,13 +3,17 @@ package org.apache.solr.parser; import java.io.StringReader; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.Query; -import org.apache.lucene.util.Version; -import org.apache.solr.search.QParser; import org.apache.solr.search.SyntaxError; +import org.apache.solr.search.QParser; +import org.apache.solr.search.QueryParserConfigurationException; public class QueryParser extends SolrQueryParserBase implements QueryParserConstants { @@ -17,9 +21,44 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst */ static public enum Operator { OR, AND } - public QueryParser(Version matchVersion, String defaultField, QParser parser) { + /** default split on whitespace behavior */ + public static final boolean DEFAULT_SPLIT_ON_WHITESPACE = true; + + public QueryParser(String defaultField, QParser parser) { this(new FastCharStream(new StringReader(""))); - init(matchVersion, defaultField, parser); + init(defaultField, parser); + } + + /** + * @see #setSplitOnWhitespace(boolean) + */ + public boolean getSplitOnWhitespace() { + return splitOnWhitespace; + } + + /** + * Whether query text should be split on whitespace prior to analysis. + * Default is {@value #DEFAULT_SPLIT_ON_WHITESPACE}. + */ + public void setSplitOnWhitespace(boolean splitOnWhitespace) { + this.splitOnWhitespace = splitOnWhitespace; + } + + private boolean splitOnWhitespace = DEFAULT_SPLIT_ON_WHITESPACE; + private static Set disallowedPostMultiTerm + = new HashSet(Arrays.asList(COLON, STAR, FUZZY_SLOP, CARAT, AND, OR)); + private static boolean allowedPostMultiTerm(int tokenKind) { + return disallowedPostMultiTerm.contains(tokenKind) == false; + } + + @Override + protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, + boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { + if ((getAutoGeneratePhraseQueries() || fieldAutoGenPhraseQueries) && splitOnWhitespace == false) { + throw new QueryParserConfigurationException + ("Field '" + field + "': autoGeneratePhraseQueries == true is disallowed when sow/splitOnWhitespace == false"); + } + return super.newFieldQuery(analyzer, field, queryText, quoted, fieldAutoGenPhraseQueries); } // * Query ::= ( Clause )* @@ -96,13 +135,38 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst final public Query Query(String field) throws ParseException, SyntaxError { List clauses = new ArrayList(); - Query q, firstQuery=null; + Query q; int conj, mods; - mods = Modifiers(); - q = Clause(field); - addClause(clauses, CONJ_NONE, mods, q); - if (mods == MOD_NONE) - firstQuery=q; + if (jj_2_1(2)) { + MultiTerm(field, clauses); + } else { + switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { + case NOT: + case PLUS: + case MINUS: + case BAREOPER: + case LPAREN: + case STAR: + case QUOTED: + case TERM: + case PREFIXTERM: + case WILDTERM: + case REGEXPTERM: + case RANGEIN_START: + case RANGEEX_START: + case LPARAMS: + case FILTER: + case NUMBER: + mods = Modifiers(); + q = Clause(field); + addClause(clauses, CONJ_NONE, mods, q); + break; + default: + jj_la1[4] = jj_gen; + jj_consume_token(-1); + throw new ParseException(); + } + } label_1: while (true) { switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { @@ -127,19 +191,50 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst ; break; default: - jj_la1[4] = jj_gen; + jj_la1[5] = jj_gen; break label_1; } - conj = Conjunction(); - mods = Modifiers(); - q = Clause(field); - addClause(clauses, conj, mods, q); - } - if (clauses.size() == 1 && firstQuery != null) - {if (true) return rawToNormal(firstQuery);} - else { - {if (true) return getBooleanQuery(clauses);} + if (jj_2_2(2)) { + MultiTerm(field, clauses); + } else { + switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { + case AND: + case OR: + case NOT: + case PLUS: + case MINUS: + case BAREOPER: + case LPAREN: + case STAR: + case QUOTED: + case TERM: + case PREFIXTERM: + case WILDTERM: + case REGEXPTERM: + case RANGEIN_START: + case RANGEEX_START: + case LPARAMS: + case FILTER: + case NUMBER: + conj = Conjunction(); + mods = Modifiers(); + q = Clause(field); + addClause(clauses, conj, mods, q); + break; + default: + jj_la1[6] = jj_gen; + jj_consume_token(-1); + throw new ParseException(); + } } + } + if (clauses.size() == 1 && clauses.get(0).getOccur() == BooleanClause.Occur.SHOULD) { + Query firstQuery = clauses.get(0).getQuery(); + if ( ! (firstQuery instanceof RawQuery) || ((RawQuery)firstQuery).getTermCount() == 1) { + {if (true) return rawToNormal(firstQuery);} + } + } + {if (true) return getBooleanQuery(clauses);} throw new Error("Missing return statement in function"); } @@ -148,20 +243,20 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst Token fieldToken=null, boost=null; Token localParams=null; int flags = 0; - if (jj_2_1(2)) { + if (jj_2_3(2)) { switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { case TERM: fieldToken = jj_consume_token(TERM); jj_consume_token(COLON); - field=discardEscapeChar(fieldToken.image); + field = discardEscapeChar(fieldToken.image); break; case STAR: jj_consume_token(STAR); jj_consume_token(COLON); - field="*"; + field = "*"; break; default: - jj_la1[5] = jj_gen; + jj_la1[7] = jj_gen; jj_consume_token(-1); throw new ParseException(); } @@ -191,7 +286,7 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst boost = jj_consume_token(NUMBER); break; default: - jj_la1[6] = jj_gen; + jj_la1[8] = jj_gen; ; } break; @@ -206,10 +301,10 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst boost = jj_consume_token(NUMBER); break; default: - jj_la1[7] = jj_gen; + jj_la1[9] = jj_gen; ; } - q=getFilter(q); restoreFlags(flags); + q=getFilter(q); restoreFlags(flags); break; case LPARAMS: localParams = jj_consume_token(LPARAMS); @@ -219,17 +314,17 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst boost = jj_consume_token(NUMBER); break; default: - jj_la1[8] = jj_gen; + jj_la1[10] = jj_gen; ; } - q=getLocalParams(field, localParams.image); + q=getLocalParams(field, localParams.image); break; default: - jj_la1[9] = jj_gen; + jj_la1[11] = jj_gen; jj_consume_token(-1); throw new ParseException(); } - {if (true) return handleBoost(q, boost);} + {if (true) return handleBoost(q, boost);} throw new Error("Missing return statement in function"); } @@ -278,35 +373,48 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst term.image = term.image.substring(0,1); break; default: - jj_la1[10] = jj_gen; + jj_la1[12] = jj_gen; jj_consume_token(-1); throw new ParseException(); } switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { - case FUZZY_SLOP: - fuzzySlop = jj_consume_token(FUZZY_SLOP); - fuzzy=true; - break; - default: - jj_la1[11] = jj_gen; - ; - } - switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { case CARAT: - jj_consume_token(CARAT); - boost = jj_consume_token(NUMBER); + case FUZZY_SLOP: switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { + case CARAT: + jj_consume_token(CARAT); + boost = jj_consume_token(NUMBER); + switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { + case FUZZY_SLOP: + fuzzySlop = jj_consume_token(FUZZY_SLOP); + fuzzy=true; + break; + default: + jj_la1[13] = jj_gen; + ; + } + break; case FUZZY_SLOP: fuzzySlop = jj_consume_token(FUZZY_SLOP); - fuzzy=true; + fuzzy=true; + switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { + case CARAT: + jj_consume_token(CARAT); + boost = jj_consume_token(NUMBER); + break; + default: + jj_la1[14] = jj_gen; + ; + } break; default: - jj_la1[12] = jj_gen; - ; + jj_la1[15] = jj_gen; + jj_consume_token(-1); + throw new ParseException(); } break; default: - jj_la1[13] = jj_gen; + jj_la1[16] = jj_gen; ; } q = handleBareTokenQuery(getField(field), term, fuzzySlop, prefix, wildcard, fuzzy, regexp); @@ -316,13 +424,13 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { case RANGEIN_START: jj_consume_token(RANGEIN_START); - startInc=true; + startInc = true; break; case RANGEEX_START: jj_consume_token(RANGEEX_START); break; default: - jj_la1[14] = jj_gen; + jj_la1[17] = jj_gen; jj_consume_token(-1); throw new ParseException(); } @@ -334,7 +442,7 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst goop1 = jj_consume_token(RANGE_QUOTED); break; default: - jj_la1[15] = jj_gen; + jj_la1[18] = jj_gen; jj_consume_token(-1); throw new ParseException(); } @@ -343,7 +451,7 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst jj_consume_token(RANGE_TO); break; default: - jj_la1[16] = jj_gen; + jj_la1[19] = jj_gen; ; } switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { @@ -354,20 +462,20 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst goop2 = jj_consume_token(RANGE_QUOTED); break; default: - jj_la1[17] = jj_gen; + jj_la1[20] = jj_gen; jj_consume_token(-1); throw new ParseException(); } switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { case RANGEIN_END: jj_consume_token(RANGEIN_END); - endInc=true; + endInc = true; break; case RANGEEX_END: jj_consume_token(RANGEEX_END); break; default: - jj_la1[18] = jj_gen; + jj_la1[21] = jj_gen; jj_consume_token(-1); throw new ParseException(); } @@ -377,46 +485,71 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst boost = jj_consume_token(NUMBER); break; default: - jj_la1[19] = jj_gen; + jj_la1[22] = jj_gen; ; } - boolean startOpen=false; - boolean endOpen=false; - if (goop1.kind == RANGE_QUOTED) { - goop1.image = goop1.image.substring(1, goop1.image.length()-1); - } else if ("*".equals(goop1.image)) { - startOpen=true; - } - if (goop2.kind == RANGE_QUOTED) { - goop2.image = goop2.image.substring(1, goop2.image.length()-1); - } else if ("*".equals(goop2.image)) { - endOpen=true; - } - q = getRangeQuery(getField(field), startOpen ? null : discardEscapeChar(goop1.image), endOpen ? null : discardEscapeChar(goop2.image), startInc, endInc); + boolean startOpen=false; + boolean endOpen=false; + if (goop1.kind == RANGE_QUOTED) { + goop1.image = goop1.image.substring(1, goop1.image.length()-1); + } else if ("*".equals(goop1.image)) { + startOpen=true; + } + if (goop2.kind == RANGE_QUOTED) { + goop2.image = goop2.image.substring(1, goop2.image.length()-1); + } else if ("*".equals(goop2.image)) { + endOpen=true; + } + q = getRangeQuery(getField(field), + startOpen ? null : discardEscapeChar(goop1.image), + endOpen ? null : discardEscapeChar(goop2.image), startInc, endInc); break; case QUOTED: term = jj_consume_token(QUOTED); switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { - case FUZZY_SLOP: - fuzzySlop = jj_consume_token(FUZZY_SLOP); - break; - default: - jj_la1[20] = jj_gen; - ; - } - switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { case CARAT: - jj_consume_token(CARAT); - boost = jj_consume_token(NUMBER); + case FUZZY_SLOP: + switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { + case CARAT: + jj_consume_token(CARAT); + boost = jj_consume_token(NUMBER); + switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { + case FUZZY_SLOP: + fuzzySlop = jj_consume_token(FUZZY_SLOP); + fuzzy=true; + break; + default: + jj_la1[23] = jj_gen; + ; + } + break; + case FUZZY_SLOP: + fuzzySlop = jj_consume_token(FUZZY_SLOP); + fuzzy=true; + switch ((jj_ntk==-1)?jj_ntk():jj_ntk) { + case CARAT: + jj_consume_token(CARAT); + boost = jj_consume_token(NUMBER); + break; + default: + jj_la1[24] = jj_gen; + ; + } + break; + default: + jj_la1[25] = jj_gen; + jj_consume_token(-1); + throw new ParseException(); + } break; default: - jj_la1[21] = jj_gen; + jj_la1[26] = jj_gen; ; } - q = handleQuotedTerm(getField(field), term, fuzzySlop); + q = handleQuotedTerm(getField(field), term, fuzzySlop); break; default: - jj_la1[22] = jj_gen; + jj_la1[27] = jj_gen; jj_consume_token(-1); throw new ParseException(); } @@ -424,6 +557,44 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst throw new Error("Missing return statement in function"); } + final public void MultiTerm(String field, List clauses) throws ParseException, SyntaxError { + Token text; + List terms = null; + text = jj_consume_token(TERM); + if (splitOnWhitespace) { + Query q = getFieldQuery(getField(field), discardEscapeChar(text.image), false, true); + addClause(clauses, CONJ_NONE, MOD_NONE, q); + } else { + terms = new ArrayList(); + terms.add(discardEscapeChar(text.image)); + } + if (getToken(1).kind == TERM && allowedPostMultiTerm(getToken(2).kind)) { + + } else { + jj_consume_token(-1); + throw new ParseException(); + } + label_2: + while (true) { + text = jj_consume_token(TERM); + if (splitOnWhitespace) { + Query q = getFieldQuery(getField(field), discardEscapeChar(text.image), false, true); + addClause(clauses, CONJ_NONE, MOD_NONE, q); + } else { + terms.add(discardEscapeChar(text.image)); + } + if (getToken(1).kind == TERM && allowedPostMultiTerm(getToken(2).kind)) { + ; + } else { + break label_2; + } + } + if (splitOnWhitespace == false) { + Query q = getFieldQuery(getField(field), terms, true); + addMultiTermClause(clauses, q); + } + } + private boolean jj_2_1(int xla) { jj_la = xla; jj_lastpos = jj_scanpos = token; try { return !jj_3_1(); } @@ -431,28 +602,76 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst finally { jj_save(0, xla); } } - private boolean jj_3R_3() { - if (jj_scan_token(STAR)) return true; - if (jj_scan_token(COLON)) return true; + private boolean jj_2_2(int xla) { + jj_la = xla; jj_lastpos = jj_scanpos = token; + try { return !jj_3_2(); } + catch(LookaheadSuccess ls) { return true; } + finally { jj_save(1, xla); } + } + + private boolean jj_2_3(int xla) { + jj_la = xla; jj_lastpos = jj_scanpos = token; + try { return !jj_3_3(); } + catch(LookaheadSuccess ls) { return true; } + finally { jj_save(2, xla); } + } + + private boolean jj_3R_7() { + if (jj_scan_token(TERM)) return true; return false; } - private boolean jj_3R_2() { + private boolean jj_3R_4() { if (jj_scan_token(TERM)) return true; if (jj_scan_token(COLON)) return true; return false; } private boolean jj_3_1() { + if (jj_3R_3()) return true; + return false; + } + + private boolean jj_3R_6() { + return false; + } + + private boolean jj_3R_3() { + if (jj_scan_token(TERM)) return true; + jj_lookingAhead = true; + jj_semLA = getToken(1).kind == TERM && allowedPostMultiTerm(getToken(2).kind); + jj_lookingAhead = false; + if (!jj_semLA || jj_3R_6()) return true; + Token xsp; + if (jj_3R_7()) return true; + while (true) { + xsp = jj_scanpos; + if (jj_3R_7()) { jj_scanpos = xsp; break; } + } + return false; + } + + private boolean jj_3_3() { Token xsp; xsp = jj_scanpos; - if (jj_3R_2()) { + if (jj_3R_4()) { jj_scanpos = xsp; - if (jj_3R_3()) return true; + if (jj_3R_5()) return true; } return false; } + private boolean jj_3_2() { + if (jj_3R_3()) return true; + return false; + } + + private boolean jj_3R_5() { + if (jj_scan_token(STAR)) return true; + if (jj_scan_token(COLON)) return true; + return false; + } + /** Generated Token Manager. */ public QueryParserTokenManager token_source; /** Current token. */ @@ -462,8 +681,11 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst private int jj_ntk; private Token jj_scanpos, jj_lastpos; private int jj_la; + /** Whether we are looking ahead. */ + private boolean jj_lookingAhead = false; + private boolean jj_semLA; private int jj_gen; - final private int[] jj_la1 = new int[23]; + final private int[] jj_la1 = new int[28]; static private int[] jj_la1_0; static private int[] jj_la1_1; static { @@ -471,12 +693,12 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst jj_la1_init_1(); } private static void jj_la1_init_0() { - jj_la1_0 = new int[] {0x6000,0x6000,0x38000,0x38000,0xfb4fe000,0x2400000,0x800000,0x800000,0x800000,0xfb4c0000,0x3a440000,0x4000000,0x4000000,0x800000,0xc0000000,0x0,0x0,0x0,0x0,0x800000,0x4000000,0x800000,0xfb440000,}; + jj_la1_0 = new int[] {0x6000,0x6000,0x38000,0x38000,0xfb4f8000,0xfb4fe000,0xfb4fe000,0x2400000,0x800000,0x800000,0x800000,0xfb4c0000,0x3a440000,0x4000000,0x800000,0x4800000,0x4800000,0xc0000000,0x0,0x0,0x0,0x0,0x800000,0x4000000,0x800000,0x4800000,0x4800000,0xfb440000,}; } private static void jj_la1_init_1() { - jj_la1_1 = new int[] {0x0,0x0,0x0,0x0,0x7,0x0,0x0,0x0,0x0,0x7,0x4,0x0,0x0,0x0,0x0,0xc0,0x8,0xc0,0x30,0x0,0x0,0x0,0x4,}; + jj_la1_1 = new int[] {0x0,0x0,0x0,0x0,0x7,0x7,0x7,0x0,0x0,0x0,0x0,0x7,0x4,0x0,0x0,0x0,0x0,0x0,0xc0,0x8,0xc0,0x30,0x0,0x0,0x0,0x0,0x0,0x4,}; } - final private JJCalls[] jj_2_rtns = new JJCalls[1]; + final private JJCalls[] jj_2_rtns = new JJCalls[3]; private boolean jj_rescan = false; private int jj_gc = 0; @@ -486,7 +708,7 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst token = new Token(); jj_ntk = -1; jj_gen = 0; - for (int i = 0; i < 23; i++) jj_la1[i] = -1; + for (int i = 0; i < 28; i++) jj_la1[i] = -1; for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls(); } @@ -495,8 +717,9 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst token_source.ReInit(stream); token = new Token(); jj_ntk = -1; + jj_lookingAhead = false; jj_gen = 0; - for (int i = 0; i < 23; i++) jj_la1[i] = -1; + for (int i = 0; i < 28; i++) jj_la1[i] = -1; for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls(); } @@ -506,7 +729,7 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst token = new Token(); jj_ntk = -1; jj_gen = 0; - for (int i = 0; i < 23; i++) jj_la1[i] = -1; + for (int i = 0; i < 28; i++) jj_la1[i] = -1; for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls(); } @@ -516,7 +739,7 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst token = new Token(); jj_ntk = -1; jj_gen = 0; - for (int i = 0; i < 23; i++) jj_la1[i] = -1; + for (int i = 0; i < 28; i++) jj_la1[i] = -1; for (int i = 0; i < jj_2_rtns.length; i++) jj_2_rtns[i] = new JJCalls(); } @@ -579,7 +802,7 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst /** Get the specific Token. */ final public Token getToken(int index) { - Token t = token; + Token t = jj_lookingAhead ? jj_scanpos : token; for (int i = 0; i < index; i++) { if (t.next != null) t = t.next; else t = t.next = token_source.getNextToken(); @@ -633,7 +856,7 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst la1tokens[jj_kind] = true; jj_kind = -1; } - for (int i = 0; i < 23; i++) { + for (int i = 0; i < 28; i++) { if (jj_la1[i] == jj_gen) { for (int j = 0; j < 32; j++) { if ((jj_la1_0[i] & (1<{@value #DEFAULT_SPLIT_ON_WHITESPACE}. + */ + public void setSplitOnWhitespace(boolean splitOnWhitespace) { + this.splitOnWhitespace = splitOnWhitespace; + } + + private boolean splitOnWhitespace = DEFAULT_SPLIT_ON_WHITESPACE; + private static Set disallowedPostMultiTerm + = new HashSet(Arrays.asList(COLON, STAR, FUZZY_SLOP, CARAT, AND, OR)); + private static boolean allowedPostMultiTerm(int tokenKind) { + return disallowedPostMultiTerm.contains(tokenKind) == false; + } + + @Override + protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, + boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { + if ((getAutoGeneratePhraseQueries() || fieldAutoGenPhraseQueries) && splitOnWhitespace == false) { + throw new QueryParserConfigurationException + ("Field '" + field + "': autoGeneratePhraseQueries == true is disallowed when sow/splitOnWhitespace == false"); + } + return super.newFieldQuery(analyzer, field, queryText, quoted, fieldAutoGenPhraseQueries); } } @@ -63,17 +97,15 @@ TOKEN_MGR_DECLS : { /* ***************** */ <*> TOKEN : { - <#_NUM_CHAR: ["0"-"9"] > - // every character that follows a backslash is considered as an escaped character - | <#_ESCAPED_CHAR: "\\" ~[] > - | <#_TERM_START_CHAR: ( ~[ " ", "\t", "\n", "\r", "\u3000", "+", "-", "!", "(", ")", ":", "^", - "[", "]", "\"", "{", "}", "~", "*", "?", "\\", "/" ] - | <_ESCAPED_CHAR> ) > - | <#_TERM_CHAR: ( <_TERM_START_CHAR> - | <_ESCAPED_CHAR> | "-" | "+" | "/" | "!") > - | <#_WHITESPACE: ( " " | "\t" | "\n" | "\r" | "\u3000") > - | <#_QUOTED_CHAR: ( ~[ "\"", "\\" ] | <_ESCAPED_CHAR> ) > - | <#_SQUOTED_CHAR: ( ~[ "'", "\\" ] | <_ESCAPED_CHAR> ) > + <#_NUM_CHAR: ["0"-"9"] > +| <#_ESCAPED_CHAR: "\\" ~[] > // every character that follows a backslash is considered as an escaped character +| <#_TERM_START_CHAR: ( ~[ " ", "\t", "\n", "\r", "\u3000", "+", "-", "!", "(", ")", ":", "^", + "[", "]", "\"", "{", "}", "~", "*", "?", "\\", "/" ] + | <_ESCAPED_CHAR> ) > +| <#_TERM_CHAR: ( <_TERM_START_CHAR> | <_ESCAPED_CHAR> | "-" | "+" | "/" | "!") > +| <#_WHITESPACE: ( " " | "\t" | "\n" | "\r" | "\u3000") > +| <#_QUOTED_CHAR: ( ~[ "\"", "\\" ] | <_ESCAPED_CHAR> ) > +| <#_SQUOTED_CHAR: ( ~[ "'", "\\" ] | <_ESCAPED_CHAR> ) > } SKIP : { @@ -93,44 +125,43 @@ TOKEN_MGR_DECLS : { < <_WHITESPACE>> } - TOKEN : { - - | - | - | - | - | > - | - | - | - | - | : Boost - | )* "\""> - | (<_TERM_CHAR>)* > - | )+ ( "." (<_NUM_CHAR>)+ )? )? > - | (<_TERM_CHAR>)* "*" ) > - | | [ "*", "?" ]) (<_TERM_CHAR> | ( [ "*", "?" ] ))* > - | - | : Range - | : Range + +| +| +| +| +| > +| +| +| +| +| : Boost +| )* "\""> +| (<_TERM_CHAR>)* > +| )+ ( "." (<_NUM_CHAR>)+ )? )? > +| (<_TERM_CHAR>)* "*" ) > +| | [ "*", "?" ]) (<_TERM_CHAR> | ( [ "*", "?" ] ))* > +| +| : Range +| : Range // TODO: consider using token states instead of inlining SQUOTED -// | )* "'"> -// | )* (~["=","}"])+ ( "=" ( | | (~[" ","}"])+ )? )? )* "}")+ (~[")"," ","\t","\n","{","^"])* > - | )* (~["=","}"])+ ( "=" ( | ("'" (<_SQUOTED_CHAR>)* "'") | (~[" ","}"])+ )? )? )* "}")+ (~[")"," ","\t","\n","{","^"])* > - | +// | )* "'"> +// | )* (~["=","}"])+ ( "=" ( | | (~[" ","}"])+ )? )? )* "}")+ (~[")"," ","\t","\n","{","^"])* > +| )* (~["=","}"])+ ( "=" ( | ("'" (<_SQUOTED_CHAR>)* "'") | (~[" ","}"])+ )? )? )* "}")+ (~[")"," ","\t","\n","{","^"])* > +| } TOKEN : { - )+ ( "." (<_NUM_CHAR>)+ )? > : DEFAULT + )+ ( "." (<_NUM_CHAR>)+ )? > : DEFAULT } TOKEN : { - - | : DEFAULT - | : DEFAULT - | - | + +| : DEFAULT +| : DEFAULT +| +| } // * Query ::= ( Clause )* @@ -160,8 +191,7 @@ int Modifiers() : { } // This makes sure that there is no garbage after the query string -Query TopLevelQuery(String field) throws SyntaxError : -{ +Query TopLevelQuery(String field) throws SyntaxError : { Query q; } { @@ -174,27 +204,31 @@ Query TopLevelQuery(String field) throws SyntaxError : Query Query(String field) throws SyntaxError : { List clauses = new ArrayList(); - Query q, firstQuery=null; + Query q; int conj, mods; } { - mods=Modifiers() q=Clause(field) - { - addClause(clauses, CONJ_NONE, mods, q); - if (mods == MOD_NONE) - firstQuery=q; - } ( - conj=Conjunction() mods=Modifiers() q=Clause(field) - { addClause(clauses, conj, mods, q); } + LOOKAHEAD(2) + MultiTerm(field, clauses) + | mods=Modifiers() q=Clause(field) + { addClause(clauses, CONJ_NONE, mods, q); } + ) + ( + LOOKAHEAD(2) + MultiTerm(field, clauses) + | conj=Conjunction() mods=Modifiers() q=Clause(field) + { addClause(clauses, conj, mods, q); } )* - { - if (clauses.size() == 1 && firstQuery != null) + { + if (clauses.size() == 1 && clauses.get(0).getOccur() == BooleanClause.Occur.SHOULD) { + Query firstQuery = clauses.get(0).getQuery(); + if ( ! (firstQuery instanceof RawQuery) || ((RawQuery)firstQuery).getTermCount() == 1) { return rawToNormal(firstQuery); - else { - return getBooleanQuery(clauses); } } + return getBooleanQuery(clauses); + } } Query Clause(String field) throws SyntaxError : { @@ -204,26 +238,22 @@ Query Clause(String field) throws SyntaxError : { int flags = 0; } { - [ LOOKAHEAD(2) ( - fieldToken= {field=discardEscapeChar(fieldToken.image);} - | {field="*";} + fieldToken= { field = discardEscapeChar(fieldToken.image); } + | { field = "*"; } ) ] - - ( q=Term(field) - | q=Query(field) ( boost=)? - | ( { flags=startFilter(); } q=Query(field) ( boost=)? { q=getFilter(q); restoreFlags(flags); } ) - | (localParams = ( boost=)? { q=getLocalParams(field, localParams.image); } ) + | q=Query(field) [ boost= ] + | ( { flags=startFilter(); } q=Query(field) [ boost= ] { q=getFilter(q); restoreFlags(flags); } ) + | (localParams = [ boost= ] { q=getLocalParams(field, localParams.image); } ) ) - { return handleBoost(q, boost); } + { return handleBoost(q, boost); } } - Query Term(String field) throws SyntaxError : { Token term, boost=null, fuzzySlop=null, goop1, goop2; boolean prefix = false; @@ -245,40 +275,78 @@ Query Term(String field) throws SyntaxError : { | term= | term= { term.image = term.image.substring(0,1); } ) - [ fuzzySlop= { fuzzy=true; } ] - [ boost= [ fuzzySlop= { fuzzy=true; } ] ] + [ + boost= [ fuzzySlop= { fuzzy=true; } ] + | fuzzySlop= { fuzzy=true; } [ boost= ] + ] + { q = handleBareTokenQuery(getField(field), term, fuzzySlop, prefix, wildcard, fuzzy, regexp); } + + | ( { startInc = true; } | ) + ( goop1= | goop1= ) + [ ] + ( goop2= | goop2= ) + ( { endInc = true; } | ) + [ boost= ] { - q = handleBareTokenQuery(getField(field), term, fuzzySlop, prefix, wildcard, fuzzy, regexp); - } - | ( ( {startInc=true;} | ) - ( goop1=|goop1= ) - [ ] - ( goop2=|goop2= ) - ( {endInc=true;} | )) - [ boost= ] - { - boolean startOpen=false; - boolean endOpen=false; - if (goop1.kind == RANGE_QUOTED) { - goop1.image = goop1.image.substring(1, goop1.image.length()-1); - } else if ("*".equals(goop1.image)) { - startOpen=true; - } - if (goop2.kind == RANGE_QUOTED) { - goop2.image = goop2.image.substring(1, goop2.image.length()-1); - } else if ("*".equals(goop2.image)) { - endOpen=true; - } - q = getRangeQuery(getField(field), startOpen ? null : discardEscapeChar(goop1.image), endOpen ? null : discardEscapeChar(goop2.image), startInc, endInc); - } - | term= - [ fuzzySlop= ] - [ boost= ] - { - q = handleQuotedTerm(getField(field), term, fuzzySlop); + boolean startOpen=false; + boolean endOpen=false; + if (goop1.kind == RANGE_QUOTED) { + goop1.image = goop1.image.substring(1, goop1.image.length()-1); + } else if ("*".equals(goop1.image)) { + startOpen=true; } + if (goop2.kind == RANGE_QUOTED) { + goop2.image = goop2.image.substring(1, goop2.image.length()-1); + } else if ("*".equals(goop2.image)) { + endOpen=true; + } + q = getRangeQuery(getField(field), + startOpen ? null : discardEscapeChar(goop1.image), + endOpen ? null : discardEscapeChar(goop2.image), startInc, endInc); + } + | term= + [ + boost= [ fuzzySlop= { fuzzy=true; } ] + | fuzzySlop= { fuzzy=true; } [ boost= ] + ] + { q = handleQuotedTerm(getField(field), term, fuzzySlop); } ) + { return handleBoost(q, boost); } +} + +void MultiTerm(String field, List clauses) throws SyntaxError : { + Token text; + List terms = null; +} +{ + text= { - return handleBoost(q, boost); + if (splitOnWhitespace) { + Query q = getFieldQuery(getField(field), discardEscapeChar(text.image), false, true); + addClause(clauses, CONJ_NONE, MOD_NONE, q); + } else { + terms = new ArrayList(); + terms.add(discardEscapeChar(text.image)); + } + } + // Both lookaheads are required; the first lookahead vets the first following term and the second lookahead vets the rest + LOOKAHEAD({ getToken(1).kind == TERM && allowedPostMultiTerm(getToken(2).kind) }) + ( + LOOKAHEAD({ getToken(1).kind == TERM && allowedPostMultiTerm(getToken(2).kind) }) + text= + { + if (splitOnWhitespace) { + Query q = getFieldQuery(getField(field), discardEscapeChar(text.image), false, true); + addClause(clauses, CONJ_NONE, MOD_NONE, q); + } else { + terms.add(discardEscapeChar(text.image)); + } + } + )+ + { + if (splitOnWhitespace == false) { + Query q = getFieldQuery(getField(field), terms, true); + addMultiTermClause(clauses, q); + } } } 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 cb3b1eedf05..08ccdd11b58 100644 --- a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java +++ b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java @@ -18,10 +18,12 @@ package org.apache.solr.parser; import java.io.StringReader; import java.util.ArrayList; +import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.reverse.ReverseStringFilter; @@ -41,7 +43,6 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.RegexpQuery; import org.apache.lucene.search.WildcardQuery; import org.apache.lucene.util.QueryBuilder; -import org.apache.lucene.util.Version; import org.apache.lucene.util.automaton.Automata; import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.Operations; @@ -59,7 +60,7 @@ import org.apache.solr.search.SolrConstantScoreQuery; import org.apache.solr.search.SyntaxError; /** This class is overridden by QueryParser in QueryParser.jj - * and acts to separate the majority of the Java code from the .jj grammar file. + * and acts to separate the majority of the Java code from the .jj grammar file. */ public abstract class SolrQueryParserBase extends QueryBuilder { @@ -83,7 +84,7 @@ public abstract class SolrQueryParserBase extends QueryBuilder { public static final Operator OR_OPERATOR = Operator.OR; /** The default operator that parser uses to combine query terms */ - Operator operator = OR_OPERATOR; + protected Operator operator = OR_OPERATOR; MultiTermQuery.RewriteMethod multiTermRewriteMethod = MultiTermQuery.CONSTANT_SCORE_REWRITE; boolean allowLeadingWildcard = true; @@ -133,16 +134,32 @@ public abstract class SolrQueryParserBase extends QueryBuilder { // internal: A simple raw fielded query public static class RawQuery extends Query { final SchemaField sfield; - final String externalVal; + private final List externalVals; public RawQuery(SchemaField sfield, String externalVal) { + this(sfield, Collections.singletonList(externalVal)); + } + + public RawQuery(SchemaField sfield, List externalVals) { this.sfield = sfield; - this.externalVal = externalVal; + this.externalVals = externalVals; + } + + public int getTermCount() { + return externalVals.size(); + } + + public List getExternalVals() { + return externalVals; + } + + public String getJoinedExternalVal() { + return externalVals.size() == 1 ? externalVals.get(0) : String.join(" ", externalVals); } @Override public String toString(String field) { - return "RAW(" + field + "," + externalVal + ")"; + return "RAW(" + field + "," + getJoinedExternalVal() + ")"; } @Override @@ -165,7 +182,7 @@ public abstract class SolrQueryParserBase extends QueryBuilder { public abstract Query TopLevelQuery(String field) throws ParseException, SyntaxError; - public void init(Version matchVersion, String defaultField, QParser parser) { + public void init(String defaultField, QParser parser) { this.schema = parser.getReq().getSchema(); this.parser = parser; this.flags = parser.getFlags(); @@ -406,17 +423,30 @@ public abstract class SolrQueryParserBase extends QueryBuilder { throw new RuntimeException("Clause cannot be both required and prohibited"); } + /** + * Called from QueryParser's MultiTerm rule. + * Assumption: no conjunction or modifiers (conj == CONJ_NONE and mods == MOD_NONE) + */ + protected void addMultiTermClause(List clauses, Query q) { + // We might have been passed a null query; the term might have been + // filtered away by the analyzer. + if (q == null) { + return; + } + clauses.add(newBooleanClause(q, operator == AND_OPERATOR ? BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD)); + } - - protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, boolean quoted) throws SyntaxError { + protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, + boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { BooleanClause.Occur occur = operator == Operator.AND ? BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD; - return createFieldQuery(analyzer, occur, field, queryText, quoted || autoGeneratePhraseQueries, phraseSlop); + return createFieldQuery(analyzer, occur, field, queryText, + quoted || fieldAutoGenPhraseQueries || autoGeneratePhraseQueries, phraseSlop); } /** - * Base implementation delegates to {@link #getFieldQuery(String,String,boolean)}. + * Base implementation delegates to {@link #getFieldQuery(String,String,boolean,boolean)}. * This method may be overridden, for example, to return * a SpanNearQuery instead of a PhraseQuery. * @@ -440,7 +470,7 @@ public abstract class SolrQueryParserBase extends QueryBuilder { query = builder.build(); } else if (query instanceof MultiPhraseQuery) { MultiPhraseQuery mpq = (MultiPhraseQuery)query; - + if (slop != mpq.getSlop()) { query = new MultiPhraseQuery.Builder(mpq).setSlop(slop).build(); } @@ -492,7 +522,7 @@ public abstract class SolrQueryParserBase extends QueryBuilder { protected Query newFuzzyQuery(Term term, float minimumSimilarity, int prefixLength) { // FuzzyQuery doesn't yet allow constant score rewrite String text = term.text(); - int numEdits = FuzzyQuery.floatToEdits(minimumSimilarity, + int numEdits = FuzzyQuery.floatToEdits(minimumSimilarity, text.codePointCount(0, text.length())); return new FuzzyQuery(term,numEdits,prefixLength); } @@ -536,14 +566,21 @@ public abstract class SolrQueryParserBase extends QueryBuilder { } SchemaField sfield = null; - List fieldValues = null; + List fieldValues = null; - - boolean useTermsQuery = (flags & QParser.FLAG_FILTER)!=0 && clauses.size() > TERMS_QUERY_THRESHOLD; - int clausesAdded = 0; + boolean onlyRawQueries = true; + int allRawQueriesTermCount = 0; + for (BooleanClause clause : clauses) { + if (clause.getQuery() instanceof RawQuery) { + allRawQueriesTermCount += ((RawQuery)clause.getQuery()).getTermCount(); + } else { + onlyRawQueries = false; + } + } + boolean useTermsQuery = (flags & QParser.FLAG_FILTER)!=0 && allRawQueriesTermCount > TERMS_QUERY_THRESHOLD; BooleanQuery.Builder booleanBuilder = newBooleanQuery(); - Map> fmap = new HashMap<>(); + Map> fmap = new HashMap<>(); for (BooleanClause clause : clauses) { Query subq = clause.getQuery(); @@ -563,14 +600,14 @@ public abstract class SolrQueryParserBase extends QueryBuilder { // If this field isn't indexed, or if it is indexed and we want to use TermsQuery, then collect this value. // We are currently relying on things like PointField not being marked as indexed in order to bypass // the "useTermQuery" check. - if (fieldValues == null && useTermsQuery || !sfield.indexed()) { + if ((fieldValues == null && useTermsQuery) || !sfield.indexed()) { fieldValues = new ArrayList<>(2); fmap.put(sfield, fieldValues); } } if (fieldValues != null) { - fieldValues.add(rawq.externalVal); + fieldValues.add(rawq); continue; } @@ -578,33 +615,50 @@ public abstract class SolrQueryParserBase extends QueryBuilder { } } - clausesAdded++; booleanBuilder.add(clause); } - for (Map.Entry> entry : fmap.entrySet()) { + for (Map.Entry> entry : fmap.entrySet()) { sfield = entry.getKey(); fieldValues = entry.getValue(); FieldType ft = sfield.getType(); // TODO: pull more of this logic out to FieldType? We would need to be able to add clauses to our existing booleanBuilder. - if (sfield.indexed() && fieldValues.size() < TERMS_QUERY_THRESHOLD || fieldValues.size() == 1) { + int termCount = fieldValues.stream().mapToInt(RawQuery::getTermCount).sum(); + if ((sfield.indexed() && termCount < TERMS_QUERY_THRESHOLD) || termCount == 1) { // use boolean query instead - for (String externalVal : fieldValues) { - Query subq = ft.getFieldQuery(this.parser, sfield, externalVal); - clausesAdded++; - booleanBuilder.add(subq, BooleanClause.Occur.SHOULD); + for (RawQuery rawq : fieldValues) { + Query subq; + if (ft.isTokenized() && sfield.indexed()) { + boolean fieldAutoGenPhraseQueries = ft instanceof TextField && ((TextField)ft).getAutoGeneratePhraseQueries(); + subq = newFieldQuery(getAnalyzer(), sfield.getName(), rawq.getJoinedExternalVal(), + false, fieldAutoGenPhraseQueries); + booleanBuilder.add(subq, BooleanClause.Occur.SHOULD); + } else { + for (String externalVal : rawq.getExternalVals()) { + subq = ft.getFieldQuery(this.parser, sfield, externalVal); + booleanBuilder.add(subq, BooleanClause.Occur.SHOULD); + } + } } } else { - Query subq = ft.getSetQuery(this.parser, sfield, fieldValues); - if (fieldValues.size() == clauses.size()) return subq; // if this is everything, don't wrap in a boolean query - clausesAdded++; + List externalVals + = fieldValues.stream().flatMap(rawq -> rawq.getExternalVals().stream()).collect(Collectors.toList()); + Query subq = ft.getSetQuery(this.parser, sfield, externalVals); + if (onlyRawQueries && termCount == allRawQueriesTermCount) return subq; // if this is everything, don't wrap in a boolean query booleanBuilder.add(subq, BooleanClause.Occur.SHOULD); } } - return booleanBuilder.build(); + BooleanQuery bq = booleanBuilder.build(); + if (bq.clauses().size() == 1) { // Unwrap single SHOULD query + BooleanClause clause = bq.clauses().iterator().next(); + if (clause.getOccur() == BooleanClause.Occur.SHOULD) { + return clause.getQuery(); + } + } + return bq; } @@ -835,9 +889,26 @@ public abstract class SolrQueryParserBase extends QueryBuilder { // Create a "normal" query from a RawQuery (or just return the current query if it's not raw) Query rawToNormal(Query q) { - if (!(q instanceof RawQuery)) return q; - RawQuery rq = (RawQuery)q; - return rq.sfield.getType().getFieldQuery(parser, rq.sfield, rq.externalVal); + Query normal = q; + if (q instanceof RawQuery) { + RawQuery rawq = (RawQuery)q; + if (rawq.sfield.getType().isTokenized()) { + normal = rawq.sfield.getType().getFieldQuery(parser, rawq.sfield, rawq.getJoinedExternalVal()); + } else { + FieldType ft = rawq.sfield.getType(); + if (rawq.getTermCount() == 1) { + normal = ft.getFieldQuery(this.parser, rawq.sfield, rawq.getExternalVals().get(0)); + } else { + BooleanQuery.Builder booleanBuilder = newBooleanQuery(); + for (String externalVal : rawq.getExternalVals()) { + Query subq = ft.getFieldQuery(this.parser, rawq.sfield, externalVal); + booleanBuilder.add(subq, BooleanClause.Occur.SHOULD); + } + normal = booleanBuilder.build(); + } + } + } + return normal; } protected Query getFieldQuery(String field, String queryText, boolean quoted) throws SyntaxError { @@ -877,21 +948,87 @@ public abstract class SolrQueryParserBase extends QueryBuilder { FieldType ft = sf.getType(); // delegate to type for everything except tokenized fields if (ft.isTokenized() && sf.indexed()) { - return newFieldQuery(getAnalyzer(), field, queryText, quoted || (ft instanceof TextField && ((TextField)ft).getAutoGeneratePhraseQueries())); + boolean fieldAutoGenPhraseQueries = ft instanceof TextField && ((TextField)ft).getAutoGeneratePhraseQueries(); + return newFieldQuery(getAnalyzer(), field, queryText, quoted, fieldAutoGenPhraseQueries); } else { if (raw) { return new RawQuery(sf, queryText); } else { - return sf.getType().getFieldQuery(parser, sf, queryText); + return ft.getFieldQuery(parser, sf, queryText); } } } // default to a normal field query - return newFieldQuery(getAnalyzer(), field, queryText, quoted); + return newFieldQuery(getAnalyzer(), field, queryText, quoted, false); } - protected boolean isRangeShouldBeProtectedFromReverse(String field, String part1){ + // Assumption: quoted is always false + protected Query getFieldQuery(String field, List queryTerms, boolean raw) throws SyntaxError { + checkNullField(field); + + SchemaField sf; + if (field.equals(lastFieldName)) { + // only look up the SchemaField on a field change... this helps with memory allocation of dynamic fields + // and large queries like foo_i:(1 2 3 4 5 6 7 8 9 10) when we are passed "foo_i" each time. + sf = lastField; + } else { + // intercept magic field name of "_" to use as a hook for our + // own functions. + if (field.charAt(0) == '_' && parser != null) { + MagicFieldName magic = MagicFieldName.get(field); + if (null != magic) { + subQParser = parser.subQuery(String.join(" ", queryTerms), magic.subParser); + return subQParser.getQuery(); + } + } + + lastFieldName = field; + sf = lastField = schema.getFieldOrNull(field); + } + + if (sf != null) { + FieldType ft = sf.getType(); + // delegate to type for everything except tokenized fields + if (ft.isTokenized() && sf.indexed()) { + String queryText = queryTerms.size() == 1 ? queryTerms.get(0) : String.join(" ", queryTerms); + boolean fieldAutoGenPhraseQueries = ft instanceof TextField && ((TextField)ft).getAutoGeneratePhraseQueries(); + return newFieldQuery(getAnalyzer(), field, queryText, false, fieldAutoGenPhraseQueries); + } else { + if (raw) { + return new RawQuery(sf, queryTerms); + } else { + if (queryTerms.size() == 1) { + return ft.getFieldQuery(parser, sf, queryTerms.get(0)); + } else { + List subqs = new ArrayList<>(); + for (String queryTerm : queryTerms) { + try { + subqs.add(ft.getFieldQuery(parser, sf, queryTerm)); + } catch (Exception e) { // assumption: raw = false only when called from ExtendedDismaxQueryParser.getQuery() + // for edismax: ignore parsing failures + } + } + if (subqs.size() == 1) { + return subqs.get(0); + } else { // delay building boolean query until we must + final BooleanClause.Occur occur + = operator == AND_OPERATOR ? BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD; + BooleanQuery.Builder booleanBuilder = newBooleanQuery(); + subqs.forEach(subq -> booleanBuilder.add(subq, occur)); + return booleanBuilder.build(); + } + } + } + } + } + + // default to a normal field query + String queryText = queryTerms.size() == 1 ? queryTerms.get(0) : String.join(" ", queryTerms); + return newFieldQuery(getAnalyzer(), field, queryText, false, false); + } + + protected boolean isRangeShouldBeProtectedFromReverse(String field, String part1){ checkNullField(field); SchemaField sf = schema.getField(field); diff --git a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java index ebb6188ec18..c0aee881c93 100644 --- a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java +++ b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java @@ -17,6 +17,7 @@ package org.apache.solr.search; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -160,6 +161,8 @@ public class ExtendedDismaxQParser extends QParser { // but always for unstructured implicit bqs created by getFieldQuery up.minShouldMatch = config.minShouldMatch; + + up.setSplitOnWhitespace(config.splitOnWhitespace); parsedUserQuery = parseOriginalQuery(up, mainUserQuery, clauses, config); @@ -307,6 +310,8 @@ public class ExtendedDismaxQParser extends QParser { up.setRemoveStopFilter(true); query = up.parse(mainUserQuery); } + } catch (QueryParserConfigurationException e) { + throw e; // Don't ignore configuration exceptions } catch (Exception e) { // ignore failure and reparse later after escaping reserved chars up.exceptions = false; @@ -545,6 +550,7 @@ public class ExtendedDismaxQParser extends QParser { pp.addAlias(IMPOSSIBLE_FIELD_NAME, tiebreaker, getFieldBoosts(fields)); pp.setPhraseSlop(slop); pp.setRemoveStopFilter(true); // remove stop filter and keep stopwords + pp.setSplitOnWhitespace(config.splitOnWhitespace); /* :TODO: reevaluate using makeDismax=true vs false... * @@ -976,6 +982,7 @@ public class ExtendedDismaxQParser extends QParser { private String field; private String val; private String val2; + private List vals; private boolean bool; private boolean bool2; private float flt; @@ -1036,6 +1043,7 @@ public class ExtendedDismaxQParser extends QParser { this.type = quoted ? QType.PHRASE : QType.FIELD; this.field = field; this.val = val; + this.vals = null; this.slop = getPhraseSlop(); // unspecified return getAliasedQuery(); } @@ -1045,10 +1053,21 @@ public class ExtendedDismaxQParser extends QParser { this.type = QType.PHRASE; this.field = field; this.val = val; + this.vals = null; this.slop = slop; return getAliasedQuery(); } - + + @Override + protected Query getFieldQuery(String field, List queryTerms, boolean raw) throws SyntaxError { + this.type = QType.FIELD; + this.field = field; + this.val = null; + this.vals = queryTerms; + this.slop = getPhraseSlop(); + return getAliasedMultiTermQuery(queryTerms); + } + @Override protected Query getPrefixQuery(String field, String val) throws SyntaxError { if (val.equals("") && field.equals("*")) { @@ -1057,11 +1076,17 @@ public class ExtendedDismaxQParser extends QParser { this.type = QType.PREFIX; this.field = field; this.val = val; + this.vals = null; return getAliasedQuery(); } @Override - protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, boolean quoted) throws SyntaxError { + protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, + boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { + if ((getAutoGeneratePhraseQueries() || fieldAutoGenPhraseQueries) && getSplitOnWhitespace() == false) { + throw new QueryParserConfigurationException + ("Field '" + field + "': autoGeneratePhraseQueries == true is disallowed when sow/splitOnWhitespace == false"); + } Analyzer actualAnalyzer; if (removeStopFilter) { if (nonStopFilterAnalyzerPerField == null) { @@ -1074,7 +1099,7 @@ public class ExtendedDismaxQParser extends QParser { } else { actualAnalyzer = parser.getReq().getSchema().getFieldType(field).getQueryAnalyzer(); } - return super.newFieldQuery(actualAnalyzer, field, queryText, quoted); + return super.newFieldQuery(actualAnalyzer, field, queryText, quoted, fieldAutoGenPhraseQueries); } @Override @@ -1083,6 +1108,7 @@ public class ExtendedDismaxQParser extends QParser { this.field = field; this.val = a; this.val2 = b; + this.vals = null; this.bool = startInclusive; this.bool2 = endInclusive; return getAliasedQuery(); @@ -1100,6 +1126,7 @@ public class ExtendedDismaxQParser extends QParser { this.type = QType.WILDCARD; this.field = field; this.val = val; + this.vals = null; return getAliasedQuery(); } @@ -1108,6 +1135,7 @@ public class ExtendedDismaxQParser extends QParser { this.type = QType.FUZZY; this.field = field; this.val = val; + this.vals = null; this.flt = minSimilarity; return getAliasedQuery(); } @@ -1157,7 +1185,129 @@ public class ExtendedDismaxQParser extends QParser { return getQuery(); } } - + + /** + * Delegates to the super class unless the field has been specified + * as an alias -- in which case we recurse on each of + * the aliased fields, and the results are composed into a + * DisjunctionMaxQuery. (so yes: aliases which point at other + * aliases should work) + */ + protected Query getAliasedMultiTermQuery(List queryTerms) throws SyntaxError { + Alias a = aliases.get(field); + this.validateCyclicAliasing(field); + if (a != null) { + List lst = getQueries(a); + if (lst == null || lst.size() == 0) { + return getQuery(); + } + + // make a DisjunctionMaxQuery in this case too... it will stop + // the "mm" processing from making everything required in the case + // that the query expanded to multiple clauses. + // DisMaxQuery.rewrite() removes itself if there is just a single clause anyway. + // if (lst.size()==1) return lst.get(0); + if (makeDismax) { + if (lst.get(0) instanceof BooleanQuery && allSameQueryStructure(lst)) { + BooleanQuery.Builder q = new BooleanQuery.Builder(); + List subs = new ArrayList<>(lst.size()); + for (int c = 0 ; c < ((BooleanQuery)lst.get(0)).clauses().size() ; ++c) { + subs.clear(); + // Make a dismax query for each clause position in the boolean per-field queries. + for (int n = 0 ; n < lst.size() ; ++n) { + subs.add(((BooleanQuery)lst.get(n)).clauses().get(c).getQuery()); + } + q.add(newBooleanClause(new DisjunctionMaxQuery(subs, a.tie), BooleanClause.Occur.SHOULD)); + } + return q.build(); + } else { + return new DisjunctionMaxQuery(lst, a.tie); + } + } else { + BooleanQuery.Builder q = new BooleanQuery.Builder(); + for (Query sub : lst) { + q.add(sub, BooleanClause.Occur.SHOULD); + } + return q.build(); + } + } else { + // verify that a fielded query is actually on a field that exists... if not, + // then throw an exception to get us out of here, and we'll treat it like a + // literal when we try the escape+re-parse. + if (exceptions) { + FieldType ft = schema.getFieldTypeNoEx(field); + if (ft == null && null == MagicFieldName.get(field)) { + throw unknownField; + } + } + return getQuery(); + } + } + + /** Recursively examines the given query list for identical structure in all queries. */ + private boolean allSameQueryStructure(List lst) { + boolean allSame = true; + Query firstQuery = lst.get(0); + for (int n = 1 ; n < lst.size(); ++n) { + Query nthQuery = lst.get(n); + if (nthQuery.getClass() != firstQuery.getClass()) { + allSame = false; + break; + } + if (firstQuery instanceof BooleanQuery) { + List firstBooleanClauses = ((BooleanQuery)firstQuery).clauses(); + List nthBooleanClauses = ((BooleanQuery)nthQuery).clauses(); + if (firstBooleanClauses.size() != nthBooleanClauses.size()) { + allSame = false; + break; + } + for (int c = 0 ; c < firstBooleanClauses.size() ; ++c) { + if (nthBooleanClauses.get(c).getQuery().getClass() != firstBooleanClauses.get(c).getQuery().getClass() + || nthBooleanClauses.get(c).getOccur() != firstBooleanClauses.get(c).getOccur()) { + allSame = false; + break; + } + if (firstBooleanClauses.get(c).getQuery() instanceof BooleanQuery && ! allSameQueryStructure + (Arrays.asList(firstBooleanClauses.get(c).getQuery(), nthBooleanClauses.get(c).getQuery()))) { + allSame = false; + break; + } + } + } + } + return allSame; + } + + @Override + protected void addMultiTermClause(List clauses, Query q) { + // We might have been passed a null query; the terms might have been filtered away by the analyzer. + if (q == null) { + return; + } + + boolean required = operator == AND_OPERATOR; + BooleanClause.Occur occur = required ? BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD; + + if (q instanceof BooleanQuery) { + boolean allOptionalDisMaxQueries = true; + for (BooleanClause c : ((BooleanQuery)q).clauses()) { + if (c.getOccur() != BooleanClause.Occur.SHOULD || ! (c.getQuery() instanceof DisjunctionMaxQuery)) { + allOptionalDisMaxQueries = false; + break; + } + } + if (allOptionalDisMaxQueries) { + // getAliasedMultiTermQuery() constructed a BooleanQuery containing only SHOULD DisjunctionMaxQuery-s. + // Unwrap the query and add a clause for each contained DisMax query. + for (BooleanClause c : ((BooleanQuery)q).clauses()) { + clauses.add(newBooleanClause(c.getQuery(), occur)); + } + return; + } + } + clauses.add(newBooleanClause(q, occur)); + } + /** * Validate there is no cyclic referencing in the aliasing */ @@ -1212,7 +1362,12 @@ public class ExtendedDismaxQParser extends QParser { switch (type) { case FIELD: // fallthrough case PHRASE: - Query query = super.getFieldQuery(field, val, type == QType.PHRASE, false); + Query query; + if (val == null) { + query = super.getFieldQuery(field, vals, false); + } else { + query = super.getFieldQuery(field, val, type == QType.PHRASE, false); + } // Boolean query on a whitespace-separated string // If these were synonyms we would have a SynonymQuery if (query instanceof BooleanQuery) { @@ -1248,6 +1403,8 @@ public class ExtendedDismaxQParser extends QParser { } return null; + } catch (QueryParserConfigurationException e) { + throw e; // Don't ignore configuration exceptions } catch (Exception e) { // an exception here is due to the field query not being compatible with the input text // for example, passing a string to a numeric field. @@ -1442,7 +1599,7 @@ public class ExtendedDismaxQParser extends QParser { */ public class ExtendedDismaxConfiguration { - /** + /** * The field names specified by 'qf' that (most) clauses will * be queried against */ @@ -1478,7 +1635,9 @@ public class ExtendedDismaxQParser extends QParser { protected boolean lowercaseOperators; protected String[] boostFuncs; - + + protected boolean splitOnWhitespace; + public ExtendedDismaxConfiguration(SolrParams localParams, SolrParams params, SolrQueryRequest req) { solrParams = SolrParams.wrapDefaults(localParams, params); @@ -1522,6 +1681,8 @@ public class ExtendedDismaxQParser extends QParser { boostFuncs = solrParams.getParams(DisMaxParams.BF); multBoosts = solrParams.getParams(DMP.MULT_BOOST); + + splitOnWhitespace = solrParams.getBool(QueryParsing.SPLIT_ON_WHITESPACE, SolrQueryParser.DEFAULT_SPLIT_ON_WHITESPACE); } /** * diff --git a/solr/core/src/java/org/apache/solr/search/LuceneQParser.java b/solr/core/src/java/org/apache/solr/search/LuceneQParser.java index 9ac318b9ad4..9668d8f8e4a 100644 --- a/solr/core/src/java/org/apache/solr/search/LuceneQParser.java +++ b/solr/core/src/java/org/apache/solr/search/LuceneQParser.java @@ -19,6 +19,7 @@ package org.apache.solr.search; import org.apache.lucene.search.Query; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.SolrParams; +import org.apache.solr.common.util.StrUtils; import org.apache.solr.request.SolrQueryRequest; /** @@ -46,6 +47,8 @@ public class LuceneQParser extends QParser { lparser.setDefaultOperator (QueryParsing.getQueryParserDefaultOperator(getReq().getSchema(), getParam(QueryParsing.OP))); + lparser.setSplitOnWhitespace(StrUtils.parseBool + (getParam(QueryParsing.SPLIT_ON_WHITESPACE), SolrQueryParser.DEFAULT_SPLIT_ON_WHITESPACE)); return lparser.parse(qstr); } diff --git a/solr/core/src/java/org/apache/solr/search/LuceneQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/LuceneQParserPlugin.java index 07b35ade7ef..4e2a4d6cc13 100644 --- a/solr/core/src/java/org/apache/solr/search/LuceneQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/LuceneQParserPlugin.java @@ -28,6 +28,8 @@ import java.util.List; *
Other parameters:

    *
  • q.op - the default operator "OR" or "AND"
  • *
  • df - the default field name
  • + *
  • sow - split on whitespace prior to analysis, boolean, + * default={@value org.apache.solr.search.SolrQueryParser#DEFAULT_SPLIT_ON_WHITESPACE}
  • *
*
Example: {!lucene q.op=AND df=text sort='price asc'}myfield:foo +bar -baz */ diff --git a/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java b/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java new file mode 100644 index 00000000000..0dd2a338a2c --- /dev/null +++ b/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java @@ -0,0 +1,24 @@ +/* + * 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.search; + +public class QueryParserConfigurationException extends IllegalArgumentException { + public QueryParserConfigurationException(String message) { + super(message); + } +} diff --git a/solr/core/src/java/org/apache/solr/search/QueryParsing.java b/solr/core/src/java/org/apache/solr/search/QueryParsing.java index fb32c6e934d..692de1a7097 100644 --- a/solr/core/src/java/org/apache/solr/search/QueryParsing.java +++ b/solr/core/src/java/org/apache/solr/search/QueryParsing.java @@ -51,6 +51,7 @@ public class QueryParsing { public static final String F = "f"; // field that a query or command pertains to public static final String TYPE = "type";// parser for this query or command public static final String DEFTYPE = "defType"; // default parser for any direct subqueries + public static final String SPLIT_ON_WHITESPACE = "sow"; // Whether to split on whitespace prior to analysis public static final String LOCALPARAM_START = "{!"; public static final char LOCALPARAM_END = '}'; // true if the value was specified by the "v" param (i.e. v=myval, or v=$param) diff --git a/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java b/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java index 49a492b556c..60ef9fb36c0 100644 --- a/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java +++ b/solr/core/src/java/org/apache/solr/search/SolrQueryParser.java @@ -25,7 +25,7 @@ import org.apache.solr.parser.QueryParser; public class SolrQueryParser extends QueryParser { public SolrQueryParser(QParser parser, String defaultField) { - super(parser.getReq().getCore().getSolrConfig().luceneMatchVersion, defaultField, parser); + super(defaultField, parser); } } diff --git a/solr/core/src/test-files/solr/collection1/conf/multiword-synonyms.txt b/solr/core/src/test-files/solr/collection1/conf/multiword-synonyms.txt new file mode 100644 index 00000000000..0ef4d78182c --- /dev/null +++ b/solr/core/src/test-files/solr/collection1/conf/multiword-synonyms.txt @@ -0,0 +1,13 @@ +# 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. + +US, U.S., U S, USA, U.S.A., U S A, United States, United States of America \ No newline at end of file diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-multiword-synonyms.xml b/solr/core/src/test-files/solr/collection1/conf/schema-multiword-synonyms.xml new file mode 100644 index 00000000000..5544e22a639 --- /dev/null +++ b/solr/core/src/test-files/solr/collection1/conf/schema-multiword-synonyms.xml @@ -0,0 +1,50 @@ + + + + + + + + + + + + + + + + + + + + + + + id + + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/synonyms.txt b/solr/core/src/test-files/solr/collection1/conf/synonyms.txt index b0e31cb7ec8..340abd7847c 100644 --- a/solr/core/src/test-files/solr/collection1/conf/synonyms.txt +++ b/solr/core/src/test-files/solr/collection1/conf/synonyms.txt @@ -29,3 +29,5 @@ Television, Televisions, TV, TVs # Synonym mappings can be used for spelling correction too pixima => pixma +# multiword synonyms +wi fi => wifi diff --git a/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java b/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java index c3b119f1182..27bf40fb818 100644 --- a/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java +++ b/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java @@ -16,7 +16,9 @@ */ package org.apache.solr.search; +import java.util.Arrays; import java.util.HashSet; +import java.util.Map; import java.util.Random; import java.util.Set; @@ -32,9 +34,11 @@ import org.apache.solr.common.SolrException; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.schema.TextField; import org.apache.solr.util.SolrPluginUtils; import org.junit.BeforeClass; import org.junit.Test; +import org.noggit.ObjectBuilder; public class TestExtendedDismaxParser extends SolrTestCaseJ4 { @@ -62,7 +66,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { "foo_i", "8" )); assertU(adoc("id", "47", "trait_ss", "Pig", - "text", "line up and fly directly at the enemy death cannons, clogging them with wreckage!")); + "text_sw", "line up and fly directly at the enemy death cannons, clogging them with wreckage!")); assertU(adoc("id", "48", "text_sw", "this has gigabyte potential", "foo_i","100")); assertU(adoc("id", "49", "text_sw", "start the big apple end", "foo_i","-100")); assertU(adoc("id", "50", "text_sw", "start new big city end")); @@ -88,98 +92,109 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { assertU(adoc("id", "69", "text_sw", "ties barbie")); assertU(adoc("id", "70", "text_sw", "hair")); assertU(adoc("id", "71", "text_sw", "ties")); + assertU(adoc("id", "72", "text_sw", "wifi ATM")); assertU(commit()); } @Test public void testSyntax() throws Exception { - // a bare * should be treated as *:* - assertJQ(req("defType","edismax", "q","*", "df","doesnotexist_s") - ,"/response/docs/[0]==" // make sure we get something... - ); - assertJQ(req("defType","edismax", "q","doesnotexist_s:*") - ,"/response/numFound==0" // nothing should be found - ); - assertJQ(req("defType","edismax","q","doesnotexist_s:*") - ,"/response/numFound==0" // nothing should be found - ); - assertJQ(req("defType","edismax","q","doesnotexist_s:( * * * )") - ,"/response/numFound==0" // nothing should be found - ); + for (String sow : Arrays.asList("true", "false")) { + // a bare * should be treated as *:* + assertJQ(req("defType", "edismax", "q", "*", "df", "doesnotexist_s", "sow", sow) + , "/response/docs/[0]==" // make sure we get something... + ); + assertJQ(req("defType", "edismax", "q", "doesnotexist_s:*", "sow", sow) + , "/response/numFound==0" // nothing should be found + ); + assertJQ(req("defType", "edismax", "q", "doesnotexist_s:*", "sow", sow) + , "/response/numFound==0" // nothing should be found + ); + assertJQ(req("defType", "edismax", "q", "doesnotexist_s:( * * * )", "sow", sow) + , "/response/numFound==0" // nothing should be found + ); + } } public void testTrailingOperators() throws Exception { - // really just test that exceptions aren't thrown by - // single + - + for (String sow : Arrays.asList("true", "false")) { + // really just test that exceptions aren't thrown by + // single + - - assertJQ(req("defType","edismax", "q","-") - ,"/response=="); + assertJQ(req("defType", "edismax", "q", "-", "df", "text_sw", "sow", sow) + , "/response=="); - assertJQ(req("defType","edismax", "q","+") - ,"/response=="); + assertJQ(req("defType", "edismax", "q", "+", "df", "text_sw", "sow", sow) + , "/response=="); - assertJQ(req("defType","edismax", "q","+ - +") - ,"/response=="); + assertJQ(req("defType", "edismax", "q", "+ - +", "df", "text_sw", "sow", sow) + , "/response=="); - assertJQ(req("defType","edismax", "q","- + -") - ,"/response=="); + assertJQ(req("defType", "edismax", "q", "- + -", "df", "text_sw", "sow", sow) + , "/response=="); - assertJQ(req("defType","edismax", "q","id:47 +") - ,"/response/numFound==1"); + assertJQ(req("defType", "edismax", "q", "id:47 +", "df", "text_sw", "sow", sow) + , "/response/numFound==1"); - assertJQ(req("defType","edismax", "q","id:47 -") - ,"/response/numFound==1"); + assertJQ(req("defType", "edismax", "q", "id:47 -", "df", "text_sw", "sow", sow) + , "/response/numFound==1"); - Random r = random(); - for (int i=0; i<100; i++) { - StringBuilder sb = new StringBuilder(); - for (int j=0; j 0 hits)", req("q", "(line notfound) OR notfound", - "qf", "text", + "qf", "text_sw", "q.op", "AND", "mm", "0%", "defType", "edismax") , "*[count(//doc)=0]"); assertQ("test default operator with mm (OR + 0% => 1 hit)", req("q", "line notfound OR notfound", - "qf", "text", + "qf", "text_sw", "q.op", "OR", "mm", "0%", "defType", "edismax") , "*[count(//doc)=1]"); assertQ("test default operator with mm (OR + 100% => 0 hits)", req("q", "line notfound OR notfound", - "qf", "text", + "qf", "text_sw", "q.op", "OR", "mm", "100%", "defType", "edismax") , "*[count(//doc)=0]"); assertQ("test default operator with mm (OR + 35% => 1 hit)", req("q", "line notfound notfound2 OR notfound", - "qf", "text", + "qf", "text_sw", "q.op", "OR", "mm", "35%", "defType", "edismax") , "*[count(//doc)=1]"); assertQ("test default operator with mm (OR + 75% => 0 hits)", req("q", "line notfound notfound2 OR notfound3", - "qf", "text", + "qf", "text_sw", "q.op", "OR", "mm", "75%", "defType", "edismax") , "*[count(//doc)=0]"); assertQ("test default operator with mm (AND + 0% => 1 hit)", req("q", "(line enemy) OR notfound", - "qf", "text", + "qf", "text_sw", "q.op", "AND", "mm", "0%", "defType", "edismax") , "*[count(//doc)=1]"); assertQ("test default operator with mm (AND + 50% => 1 hit)", req("q", "(line enemy) OR (line notfound) OR (death cannons) OR (death notfound)", - "qf", "text", + "qf", "text_sw", "q.op", "AND", "mm", "50%", "defType", "edismax") , "*[count(//doc)=1]"); assertQ("test default operator with mm (AND + 75% => 0 hits)", req("q", "(line enemy) OR (line notfound) OR (death cannons) OR (death notfound)", - "qf", "text", + "qf", "text_sw", "q.op", "AND", "mm", "75%", "defType", "edismax") @@ -1092,214 +1107,257 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { * Test that minShouldMatch applies to Optional terms only */ public void testMinShouldMatchOptional() throws Exception { - assertQ("test minShouldMatch (top level optional terms only)", - req("q", "stocks oil gold", // +(((text_sw:stock) (text_sw:oil) (text_sw:gold))~1) - "qf", "text_sw", - "mm", "50%", - "defType", "edismax") - , "*[count(//doc)=4]"); - - assertQ("test minShouldMatch (top level optional and negative terms mm=50%)", - req("q", "stocks oil gold -stockade", // +(((text_sw:stock) (text_sw:oil) (text_sw:gold) -(text_sw:stockad))~1) - "qf", "text_sw", - "mm", "50%", - "defType", "edismax") - , "*[count(//doc)=3]"); + for (String sow : Arrays.asList("true", "false")) { + assertQ("test minShouldMatch (top level optional terms only)", + req("q", "stocks oil gold", // +(((text_sw:stock) (text_sw:oil) (text_sw:gold))~1) + "qf", "text_sw", + "mm", "50%", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=4]"); - assertQ("test minShouldMatch (top level optional and negative terms mm=100%)", - req("q", "stocks gold -stockade", // +(((text_sw:stock) (text_sw:oil) (text_sw:gold) -(text_sw:stockad))~2) - "qf", "text_sw", - "mm", "100%", - "defType", "edismax") - , "*[count(//doc)=1]"); + assertQ("test minShouldMatch (top level optional terms only and sow=false)", + req("q", "stocks oil gold", // +(((text_sw:stock) (text_sw:oil) (text_sw:gold))~1) + "qf", "text_sw", + "mm", "50%", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=4]"); - assertQ("test minShouldMatch (top level required terms only)", - req("q", "stocks AND oil", // +(+(text_sw:stock) +(text_sw:oil)) - "qf", "text_sw", - "mm", "50%", - "defType", "edismax") - , "*[count(//doc)=1]"); + assertQ("test minShouldMatch (top level optional and negative terms mm=50%)", + req("q", "stocks oil gold -stockade", // +(((text_sw:stock) (text_sw:oil) (text_sw:gold) -(text_sw:stockad))~1) + "qf", "text_sw", + "mm", "50%", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=3]"); - assertQ("test minShouldMatch (top level optional and required terms)", - req("q", "oil gold +stocks", // +(((text_sw:oil) (text_sw:gold) +(text_sw:stock))~1) - "qf", "text_sw", - "mm", "50%", - "defType", "edismax") - , "*[count(//doc)=3]"); + assertQ("test minShouldMatch (top level optional and negative terms mm=100%)", + req("q", "stocks gold -stockade", // +(((text_sw:stock) (text_sw:oil) (text_sw:gold) -(text_sw:stockad))~2) + "qf", "text_sw", + "mm", "100%", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=1]"); - assertQ("test minShouldMatch (top level optional with explicit OR and parens)", - req("q", "(snake OR stocks) oil", - "qf", "text_sw", - "mm", "100%", - "defType", "edismax") - , "*[count(//doc)=2]"); + assertQ("test minShouldMatch (top level required terms only)", + req("q", "stocks AND oil", // +(+(text_sw:stock) +(text_sw:oil)) + "qf", "text_sw", + "mm", "50%", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=1]"); - // The results for these two appear odd, but are correct as per BooleanQuery processing. - // See: http://searchhub.org/2011/12/28/why-not-and-or-and-not/ - // Non-parenthesis OR/AND precedence is not true to abstract boolean logic in solr when q.op = AND - // and when q.op = OR all three clauses are top-level and optional so mm takes over - assertQ("test minShouldMatch (top level optional with explicit OR without parens)", - req("q", "snake OR stocks oil", - "qf", "text_sw", - "q.op", "OR", - "mm", "100%", - "defType", "edismax") - , "*[count(//doc)=0]"); - assertQ("test minShouldMatch (top level optional with explicit OR without parens)", - req("q", "snake OR stocks oil", - "qf", "text_sw", - "q.op", "AND", - "mm", "100%", - "defType", "edismax") - , "*[count(//doc)=0]"); + assertQ("test minShouldMatch (top level optional and required terms)", + req("q", "oil gold +stocks", // +(((text_sw:oil) (text_sw:gold) +(text_sw:stock))~1) + "qf", "text_sw", + "mm", "50%", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=3]"); - // SOLR-9174 - assertQ("test minShouldMatch=1<-1 with explicit OR, one impossible clause, and no explicit q.op", - req("q", "barbie OR (hair AND nonexistentword)", - "qf", "text_sw", - "mm", "1<-1", - "defType", "edismax") - , "*[count(//doc)=3]"); + assertQ("test minShouldMatch (top level optional with explicit OR and parens)", + req("q", "(snake OR stocks) oil", + "qf", "text_sw", + "mm", "100%", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=2]"); + + // The results for these two appear odd, but are correct as per BooleanQuery processing. + // See: http://searchhub.org/2011/12/28/why-not-and-or-and-not/ + // Non-parenthesis OR/AND precedence is not true to abstract boolean logic in solr when q.op = AND + // and when q.op = OR all three clauses are top-level and optional so mm takes over + assertQ("test minShouldMatch (top level optional with explicit OR without parens)", + req("q", "snake OR stocks oil", + "qf", "text_sw", + "q.op", "OR", + "mm", "100%", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=0]"); + assertQ("test minShouldMatch (top level optional with explicit OR without parens)", + req("q", "snake OR stocks oil", + "qf", "text_sw", + "q.op", "AND", + "mm", "100%", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=0]"); + + // SOLR-9174 + assertQ("test minShouldMatch=1<-1 with explicit OR, one impossible clause, and no explicit q.op", + req("q", "barbie OR (hair AND nonexistentword)", + "qf", "text_sw", + "mm", "1<-1", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=3]"); + } } /* SOLR-8812 */ @Test public void testDefaultMM() throws Exception { // Ensure MM is off when explicit operators (+/-/OR/NOT) are used and no explicit mm spec is specified. - assertQ("Explicit OR in query with no explicit mm and q.op=AND => mm = 0%", - req("q", "oil OR stocks", - "qf", "text_sw", - "q.op", "AND", - "defType", "edismax") - , "*[count(//doc)=4]"); - assertQ("Explicit 'or' in query with lowercaseOperators=true, no explicit mm and q.op=AND => mm = 0%", - req("q", "oil or stocks", - "qf", "text_sw", - "q.op", "AND", - "lowercaseOperators", "true", - "defType", "edismax") - , "*[count(//doc)=4]"); - assertQ("Explicit OR in query with no explicit mm and no explicit q.op => mm = 0%", - req("q", "oil OR stocks", - "qf", "text_sw", - "defType", "edismax") - , "*[count(//doc)=4]"); - assertQ("No operator in query with no explicit mm and q.op=OR => mm = 0%", - req("q", "oil stocks", - "qf", "text_sw", - "defType", "edismax") - , "*[count(//doc)=4]"); - assertQ("No operator in query with no explicit mm and q.op=AND => mm = 100%", - req("q", "oil stocks", - "qf", "text_sw", - "q.op", "AND", - "defType", "edismax") - , "*[count(//doc)=1]"); - assertQ("No operator in query with no explicit mm and q.op=OR => mm = 0%", - req("q", "oil stocks", - "qf", "text_sw", - "q.op", "OR", - "defType", "edismax") - , "*[count(//doc)=4]"); + for (String sow : Arrays.asList("true", "false")) { + assertQ("Explicit OR in query with no explicit mm and q.op=AND => mm = 0%", + req("q", "oil OR stocks", + "qf", "text_sw", + "q.op", "AND", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=4]"); + assertQ("Explicit 'or' in query with lowercaseOperators=true, no explicit mm and q.op=AND => mm = 0%", + req("q", "oil or stocks", + "qf", "text_sw", + "q.op", "AND", + "lowercaseOperators", "true", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=4]"); + assertQ("Explicit OR in query with no explicit mm and no explicit q.op => mm = 0%", + req("q", "oil OR stocks", + "qf", "text_sw", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=4]"); + assertQ("No operator in query with no explicit mm and q.op=OR => mm = 0%", + req("q", "oil stocks", + "qf", "text_sw", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=4]"); + assertQ("No operator in query with no explicit mm and q.op=AND => mm = 100%", + req("q", "oil stocks", + "qf", "text_sw", + "q.op", "AND", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=1]"); + assertQ("No operator in query with no explicit mm and q.op=OR => mm = 0%", + req("q", "oil stocks", + "qf", "text_sw", + "q.op", "OR", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=4]"); - assertQ("Explicit '-' operator in query with no explicit mm and no explicit q.op => mm = 0%", - req("q", "hair ties -barbie", - "qf", "text_sw", - "defType", "edismax") - , "*[count(//doc)=3]"); - assertQ("Explicit NOT in query with no explicit mm and no explicit q.op => mm = 0%", - req("q", "hair ties NOT barbie", - "qf", "text_sw", - "defType", "edismax") - , "*[count(//doc)=3]"); + assertQ("Explicit '-' operator in query with no explicit mm and no explicit q.op => mm = 0%", + req("q", "hair ties -barbie", + "qf", "text_sw", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=3]"); + assertQ("Explicit NOT in query with no explicit mm and no explicit q.op => mm = 0%", + req("q", "hair ties NOT barbie", + "qf", "text_sw", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=3]"); - assertQ("Explicit '-' operator in query with no explicit mm and q.op=OR => mm = 0%", - req("q", "hair ties -barbie", - "qf", "text_sw", - "q.op", "OR", - "defType", "edismax") - , "*[count(//doc)=3]"); - assertQ("Explicit NOT in query with no explicit mm and q.op=OR => mm = 0%", - req("q", "hair ties NOT barbie", - "qf", "text_sw", - "q.op", "OR", - "defType", "edismax") - , "*[count(//doc)=3]"); + assertQ("Explicit '-' operator in query with no explicit mm and q.op=OR => mm = 0%", + req("q", "hair ties -barbie", + "qf", "text_sw", + "q.op", "OR", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=3]"); + assertQ("Explicit NOT in query with no explicit mm and q.op=OR => mm = 0%", + req("q", "hair ties NOT barbie", + "qf", "text_sw", + "q.op", "OR", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=3]"); - assertQ("Explicit '-' operator in query with no explicit mm and q.op=OR => mm = 0%", - req("q", "hair AND ties -barbie", - "qf", "text_sw", - "q.op", "OR", - "defType", "edismax") - , "*[count(//doc)=1]"); - assertQ("Explicit NOT in query with no explicit mm and q.op=OR => mm = 0%", - req("q", "hair AND ties -barbie", - "qf", "text_sw", - "q.op", "OR", - "defType", "edismax") - , "*[count(//doc)=1]"); + assertQ("Explicit '-' operator in query with no explicit mm and q.op=OR => mm = 0%", + req("q", "hair AND ties -barbie", + "qf", "text_sw", + "q.op", "OR", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=1]"); + assertQ("Explicit NOT in query with no explicit mm and q.op=OR => mm = 0%", + req("q", "hair AND ties -barbie", + "qf", "text_sw", + "q.op", "OR", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=1]"); - assertQ("No explicit non-AND operator in query with no explicit mm and q.op=OR => mm = 0%", - req("q", "hair AND ties barbie", - "qf", "text_sw", - "q.op", "OR", - "defType", "edismax") - , "*[count(//doc)=2]"); - assertQ("No explicit non-AND operator in query with no explicit mm and q.op=AND => mm = 100%", - req("q", "hair AND ties barbie", - "qf", "text_sw", - "q.op", "AND", - "defType", "edismax") - , "*[count(//doc)=1]"); - assertQ("No explicit non-AND operator in query with no explicit mm and no explicit q.op => mm = 0%", - req("q", "hair AND ties barbie", - "qf", "text_sw", - "defType", "edismax") - , "*[count(//doc)=2]"); - assertQ("No explicit non-AND operator in query with no explicit mm and no explicit q.op => mm = 0%", - req("q", "hair and ties barbie", - "qf", "text_sw", - "lowercaseOperators", "true", - "defType", "edismax") - , "*[count(//doc)=2]"); + assertQ("No explicit non-AND operator in query with no explicit mm and q.op=OR => mm = 0%", + req("q", "hair AND ties barbie", + "qf", "text_sw", + "q.op", "OR", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=2]"); + assertQ("No explicit non-AND operator in query with no explicit mm and q.op=AND => mm = 100%", + req("q", "hair AND ties barbie", + "qf", "text_sw", + "q.op", "AND", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=1]"); + assertQ("No explicit non-AND operator in query with no explicit mm and no explicit q.op => mm = 0%", + req("q", "hair AND ties barbie", + "qf", "text_sw", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=2]"); + assertQ("No explicit non-AND operator in query with no explicit mm and no explicit q.op => mm = 0%", + req("q", "hair and ties barbie", + "qf", "text_sw", + "lowercaseOperators", "true", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=2]"); - assertQ("Explicit '-' operator in query with no explicit mm and q.op=AND => mm = 100%", - req("q", "hair ties -barbie", - "qf", "text_sw", - "q.op", "AND", - "defType", "edismax") - , "*[count(//doc)=1]"); - assertQ("Explicit NOT in query with no explicit mm and q.op=AND => mm = 100%", - req("q", "hair ties NOT barbie", - "qf", "text_sw", - "q.op", "AND", - "defType", "edismax") - , "*[count(//doc)=1]"); + assertQ("Explicit '-' operator in query with no explicit mm and q.op=AND => mm = 100%", + req("q", "hair ties -barbie", + "qf", "text_sw", + "q.op", "AND", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=1]"); + assertQ("Explicit NOT in query with no explicit mm and q.op=AND => mm = 100%", + req("q", "hair ties NOT barbie", + "qf", "text_sw", + "q.op", "AND", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=1]"); - assertQ("Explicit OR in query with no explicit mm and q.op=AND => mm = 0%", - req("q", "hair OR ties barbie", - "qf", "text_sw", - "q.op", "AND", - "defType", "edismax") - , "*[count(//doc)=3]"); - assertQ("Explicit OR in query with no explicit mm and q.op=OR => mm = 0%", - req("q", "hair OR ties barbie", - "qf", "text_sw", - "q.op", "OR", - "defType", "edismax") - , "*[count(//doc)=6]"); - assertQ("Explicit OR in query with no explicit mm and no explicit q.op => mm = 0%", - req("q", "hair OR ties barbie", - "qf", "text_sw", - "defType", "edismax") - , "*[count(//doc)=6]"); + assertQ("Explicit OR in query with no explicit mm and q.op=AND => mm = 0%", + req("q", "hair OR ties barbie", + "qf", "text_sw", + "q.op", "AND", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=3]"); + assertQ("Explicit OR in query with no explicit mm and q.op=OR => mm = 0%", + req("q", "hair OR ties barbie", + "qf", "text_sw", + "q.op", "OR", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=6]"); + assertQ("Explicit OR in query with no explicit mm and no explicit q.op => mm = 0%", + req("q", "hair OR ties barbie", + "qf", "text_sw", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=6]"); - assertQ("Explicit '+' operator in query with no explicit mm and q.op=AND => mm = 0%", - req("q", "hair ties +barbie", - "qf", "text_sw", - "q.op", "AND", - "defType", "edismax") - , "*[count(//doc)=1]"); + assertQ("Explicit '+' operator in query with no explicit mm and q.op=AND => mm = 0%", + req("q", "hair ties +barbie", + "qf", "text_sw", + "q.op", "AND", + "sow", sow, + "defType", "edismax") + , "*[count(//doc)=1]"); + } } public void testEdismaxSimpleExtension() throws SyntaxError { @@ -1336,6 +1394,380 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { } + // LUCENE-7533 + public void testSplitOnWhitespace_with_autoGeneratePhraseQueries() throws Exception { + assertTrue(((TextField)h.getCore().getLatestSchema().getField("text").getType()).getAutoGeneratePhraseQueries()); + + try (SolrQueryRequest req = req()) { + final QParser qparser = QParser.getParser("{!edismax sow=false fq=text}blah blah)", req); + expectThrows(IllegalArgumentException.class, qparser::getQuery); + } + } + + @Test + public void testSplitOnWhitespace_Basic() throws Exception { + // The "text_sw" field has synonyms loaded from synonyms.txt + + // retrieve the single document containing literal "wifi" + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wifi", "sow","true") + , "/response/numFound==1" + , "/response/docs/[0]/id=='72'" + ); + + // trigger the "wi fi => wifi" synonym + assertJQ(req("qf", "text_sw title", "defType","edismax", "q","wi fi", "sow","false") + , "/response/numFound==1" + , "/response/docs/[0]/id=='72'" + ); + assertJQ(req("qf", "text_sw title", "defType","edismax", "q","wi fi", "sow","true") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi") // default sow=true + , "/response/numFound==0" + ); + + assertJQ(req("qf","text_sw title", "q","{!edismax sow=false}wi fi") + , "/response/numFound==1" + , "/response/docs/[0]/id=='72'" + ); + assertJQ(req("df", "text_sw title", "q","{!edismax sow=true}wi fi") + , "/response/numFound==0" + ); + assertJQ(req("df", "text_sw title", "q", "{!edismax}wi fi") // default sow=true + , "/response/numFound==0" + ); + + assertQ(req("qf", "name title", + "q", "barking curds of stigma", + "defType", "edismax", + "sow", "false", + "debugQuery", "true"), + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:barking | title:barking))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:curds | title:curds))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:of | title:of))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:stigma | title:stigma))')]" + ); + assertQ(req("qf", "name title", + "q", "barking curds of stigma", + "defType", "edismax", + "sow", "true", + "debugQuery", "true"), + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:barking | title:barking))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:curds | title:curds))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:of | title:of))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:stigma | title:stigma))')]" + ); + assertQ(req("qf", "name title", + "q", "barking curds of stigma", + "defType", "edismax", + "debugQuery", "true"), // Default sow=true + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:barking | title:barking))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:curds | title:curds))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:of | title:of))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((name:stigma | title:stigma))')]" + ); + } + + public void testSplitOnWhitespace_Different_Field_Analysis() throws Exception { + // When the *structure* of produced queries is different in each field, + // sow=true produces boolean-of-dismax query structure, + // and sow=false produces dismax-of-boolean query structure. + assertQ(req("qf", "text_sw title", + "q", "olive the other", + "defType", "edismax", + "sow", "true", + "debugQuery", "true"), + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((text_sw:oliv | title:olive))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((title:the))')]", + "//str[@name='parsedquery'][contains(.,'DisjunctionMaxQuery((text_sw:other | title:other))')]" + ); + assertQ(req("qf", "text_sw title", + "q", "olive the other", + "defType", "edismax", + "sow", "false", + "debugQuery", "true"), + "//str[@name='parsedquery'][contains(.,'+DisjunctionMaxQuery(((text_sw:oliv text_sw:other) | (title:olive title:the title:other)))')]" + ); + + // When field's analysis produce different query structures, mm processing is always done on the boolean query. + // sow=true produces (boolean-of-dismax)~ query structure, + // and sow=false produces dismax-of-(boolean)~ query structure. + assertQ(req("qf", "text_sw title", + "q", "olive the other", + "defType", "edismax", + "sow", "true", + "mm", "100%", + "debugQuery", "true"), + "//str[@name='parsedquery'][contains(.,'+(DisjunctionMaxQuery((text_sw:oliv | title:olive)) DisjunctionMaxQuery((title:the)) DisjunctionMaxQuery((text_sw:other | title:other)))~3')]" + ); + assertQ(req("qf", "text_sw title", + "q", "olive the other", + "defType", "edismax", + "sow", "false", + "mm", "100%", + "debugQuery", "true"), + "//str[@name='parsedquery'][contains(.,'+DisjunctionMaxQuery((((text_sw:oliv text_sw:other)~2) | ((title:olive title:the title:other)~3)))')]" + ); + + + // When the *structure* of produced queries is the same in each field, + // sow=false/true produce the same boolean-of-dismax query structure + for (String sow : Arrays.asList("true", "false")) { + assertQ(req("qf", "text_sw title", + "q", "olive blah other", + "defType", "edismax", + "sow", sow, + "debugQuery", "true"), + "//str[@name='parsedquery'][contains(.,'" + + "+(DisjunctionMaxQuery((text_sw:oliv | title:olive))" + + " DisjunctionMaxQuery((text_sw:blah | title:blah))" + + " DisjunctionMaxQuery((text_sw:other | title:other)))')]" + ); + } + } + + public void testOperatorsAndMultiWordSynonyms() throws Exception { + // The "text_sw" field has synonyms loaded from synonyms.txt + + // retrieve the single document containing literal "wifi" + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wifi", "sow","true") + , "/response/numFound==1" + , "/response/docs/[0]/id=='72'" + ); + // trigger the "wi fi => wifi" synonym + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi", "sow","false") + , "/response/numFound==1" + , "/response/docs/[0]/id=='72'" + ); + + assertJQ(req("qf","text_sw title", "defType","edismax", "q","+wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","-wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","!wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi* fi", "sow","false") + , "/response/numFound==2" // matches because wi* matches "wifi" in one doc and "with" in another + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","w? fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi~1 fi", "sow","false") + , "/response/numFound==4" // matches because wi~1 matches ti (stemmed "ties") + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi^2 fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi^=2 fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi +fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi -fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi !fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi*", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi?", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi~1", "sow","false") + , "/response/numFound==4" // matches because fi~1 matches ti (stemmed "ties") + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi^2", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi^=2", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","text_sw:wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi text_sw:fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","NOT wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi NOT fi", "sow","false") + , "/response/numFound==0" + ); + + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi AND ATM", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","ATM AND wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi && ATM", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","ATM && wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","(wi fi) AND ATM", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","ATM AND (wi fi)", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","(wi fi) && ATM", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","ATM && (wi fi)", "sow","false") + , "/response/numFound==1" + ); + + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi OR NotThereAtAll", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","NotThereAtAll OR wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi || NotThereAtAll", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","NotThereAtAll || wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","(wi fi) OR NotThereAtAll", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","NotThereAtAll OR (wi fi)", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","(wi fi) || NotThereAtAll", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","NotThereAtAll || (wi fi)", "sow","false") + , "/response/numFound==1" + ); + + assertJQ(req("qf","text_sw title", "defType","edismax", "q","\"wi\" fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi \"fi\"", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","(wi) fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi (fi)", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","/wi/ fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi /fi/", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","(wi fi)", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","+(wi fi)", "sow","false") + , "/response/numFound==1" + ); + + Map all = (Map)ObjectBuilder.fromJSON(h.query(req("q", "*:*", "rows", "0", "wt", "json"))); + int totalDocs = Integer.parseInt(((Map)all.get("response")).get("numFound").toString()); + int allDocsExceptOne = totalDocs - 1; + + assertJQ(req("qf","text_sw title", "defType","edismax", "q","-(wi fi)", "sow","false") + , "/response/numFound==" + allDocsExceptOne // one doc contains "wifi" in the text_sw field + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","!(wi fi)", "sow","false") + , "/response/numFound==" + allDocsExceptOne // one doc contains "wifi" in the text_sw field + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","NOT (wi fi)", "sow","false") + , "/response/numFound==" + allDocsExceptOne // one doc contains "wifi" in the text_sw field + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","(wi fi)^2", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","(wi fi)^=2", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","text_sw:(wi fi)", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","+ATM wi fi", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","-ATM wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","-NotThereAtAll wi fi", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","!ATM wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","!NotThereAtAll wi fi", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","NOT ATM wi fi", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","NOT NotThereAtAll wi fi", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","AT* wi fi", "sow","false") + , "/response/numFound==2" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","AT? wi fi", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","\"ATM\" wi fi", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi +ATM", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi -ATM", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi -NotThereAtAll", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi !ATM", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi !NotThereAtAll", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi NOT ATM", "sow","false") + , "/response/numFound==0" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi NOT NotThereAtAll", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi AT*", "sow","false") + , "/response/numFound==2" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi AT?", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi \"ATM\"", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","\"wi fi\"~2", "sow","false") + , "/response/numFound==1" + ); + assertJQ(req("qf","text_sw title", "defType","edismax", "q","text_sw:\"wi fi\"", "sow","false") + , "/response/numFound==1" + ); + } + + + private boolean containsClause(Query query, String field, String value, int boost, boolean fuzzy) { diff --git a/solr/core/src/test/org/apache/solr/search/TestMultiWordSynonyms.java b/solr/core/src/test/org/apache/solr/search/TestMultiWordSynonyms.java new file mode 100644 index 00000000000..ecc80c398a8 --- /dev/null +++ b/solr/core/src/test/org/apache/solr/search/TestMultiWordSynonyms.java @@ -0,0 +1,100 @@ +/* + * 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.search; + +import java.util.Arrays; + +import org.apache.solr.SolrTestCaseJ4; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestMultiWordSynonyms extends SolrTestCaseJ4 { + + @BeforeClass + public static void beforeClass() throws Exception { + initCore("solrconfig.xml", "schema-multiword-synonyms.xml"); + index(); + } + + private static void index() throws Exception { + assertU(adoc("id","1", "text","USA Today")); + assertU(adoc("id","2", "text","A dynamic US economy")); + assertU(adoc("id","3", "text","The United States of America's 50 states")); + assertU(adoc("id","4", "text","Party in the U.S.A.")); + assertU(adoc("id","5", "text","These United States")); + + assertU(adoc("id","6", "text","America United of States")); + assertU(adoc("id","7", "text","States United")); + + assertU(commit()); + } + + @Test + public void testNonPhrase() throws Exception { + // Don't split on whitespace (sow=false) + for (String q : Arrays.asList("US", "U.S.", "USA", "U.S.A.", "United States", "United States of America")) { + for (String defType : Arrays.asList("lucene", "edismax")) { + assertJQ(req("q", q, + "defType", defType, + "df", "text", + "sow", "false") + , "/response/numFound==7" + ); + } + } + + // Split on whitespace (sow=true) + for (String q : Arrays.asList("US", "U.S.", "USA", "U.S.A.")) { + for (String defType : Arrays.asList("lucene", "edismax")) { + assertJQ(req("q", q, + "defType", defType, + "df", "text", + "sow", "true") + , "/response/numFound==7" + ); + } + } + for (String q : Arrays.asList("United States", "United States of America")) { + for (String defType : Arrays.asList("lucene", "edismax")) { + assertJQ(req("q", q, + "defType", defType, + "df", "text", + "sow", "true") + , "/response/numFound==4" + ); + } + } + } + + @Test + public void testPhrase() throws Exception { + for (String q : Arrays.asList + ("\"US\"", "\"U.S.\"", "\"USA\"", "\"U.S.A.\"", "\"United States\"", "\"United States of America\"")) { + for (String defType : Arrays.asList("lucene", "edismax")) { + for (String sow : Arrays.asList("true", "false")) { + assertJQ(req("q", q, + "defType", defType, + "df", "text", + "sow", sow) + , "/response/numFound==5" + ); + } + } + } + } +} diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java index 8195c058a31..92bd6c0e2d6 100644 --- a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java +++ b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java @@ -16,7 +16,12 @@ */ package org.apache.solr.search; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Random; import org.apache.lucene.search.BooleanClause; @@ -28,12 +33,15 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TermInSetQuery; import org.apache.lucene.search.TermQuery; import org.apache.solr.SolrTestCaseJ4; +import org.apache.solr.common.params.MapSolrParams; import org.apache.solr.core.SolrInfoMBean; import org.apache.solr.parser.QueryParser; import org.apache.solr.query.FilterQuery; import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.schema.TextField; import org.junit.BeforeClass; import org.junit.Test; +import org.noggit.ObjectBuilder; public class TestSolrQueryParser extends SolrTestCaseJ4 { @@ -57,6 +65,8 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 { assertU(adoc("id", "12", "eee_s", "X")); assertU(adoc("id", "13", "eee_s", "'balance'", "rrr_s", "/leading_slash")); + assertU(adoc("id", "20", "syn", "wifi ATM")); + assertU(commit()); } @@ -208,86 +218,105 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 { QParser qParser; Query q,qq; - // relevance query should not be a filter - qParser = QParser.getParser("foo_s:(a b c)", req); - q = qParser.getQuery(); - assertEquals(3, ((BooleanQuery)q).clauses().size()); + Map sowFalseParamsMap = new HashMap<>(); + sowFalseParamsMap.put("sow", "false"); + Map sowTrueParamsMap = new HashMap<>(); + sowTrueParamsMap.put("sow", "true"); + List paramMaps = Arrays.asList + (new MapSolrParams(Collections.emptyMap()), // no sow param (i.e. the default sow value) + new MapSolrParams(sowFalseParamsMap), + new MapSolrParams(sowTrueParamsMap)); - // small filter query should still use BooleanQuery - if (QueryParser.TERMS_QUERY_THRESHOLD > 3) { + for (MapSolrParams params : paramMaps) { + // relevance query should not be a filter qParser = QParser.getParser("foo_s:(a b c)", req); - qParser.setIsFilter(true); // this may change in the future + qParser.setParams(params); q = qParser.getQuery(); assertEquals(3, ((BooleanQuery) q).clauses().size()); + + // small filter query should still use BooleanQuery + if (QueryParser.TERMS_QUERY_THRESHOLD > 3) { + qParser = QParser.getParser("foo_s:(a b c)", req); + qParser.setParams(params); + qParser.setIsFilter(true); // this may change in the future + q = qParser.getQuery(); + assertEquals(3, ((BooleanQuery) q).clauses().size()); + } + + // large relevancy query should use BooleanQuery + // TODO: we may decide that string fields shouldn't have relevance in the future... change to a text field w/o a stop filter if so + qParser = QParser.getParser("foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z)", req); + qParser.setParams(params); + q = qParser.getQuery(); + assertEquals(26, ((BooleanQuery)q).clauses().size()); + + // large filter query should use TermsQuery + qParser = QParser.getParser("foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z)", req); + qParser.setIsFilter(true); // this may change in the future + qParser.setParams(params); + q = qParser.getQuery(); + assertEquals(26, ((TermInSetQuery)q).getTermData().size()); + + // large numeric filter query should use TermsQuery (for trie fields) + qParser = QParser.getParser("foo_ti:(1 2 3 4 5 6 7 8 9 10 20 19 18 17 16 15 14 13 12 11)", req); + qParser.setIsFilter(true); // this may change in the future + qParser.setParams(params); + q = qParser.getQuery(); + assertEquals(20, ((TermInSetQuery)q).getTermData().size()); + + // for point fields large filter query should use PointInSetQuery + qParser = QParser.getParser("foo_pi:(1 2 3 4 5 6 7 8 9 10 20 19 18 17 16 15 14 13 12 11)", req); + qParser.setIsFilter(true); // this may change in the future + qParser.setParams(params); + q = qParser.getQuery(); + assertTrue(q instanceof PointInSetQuery); + assertEquals(20, ((PointInSetQuery)q).getPackedPoints().size()); + + // a filter() clause inside a relevancy query should be able to use a TermsQuery + qParser = QParser.getParser("foo_s:aaa filter(foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z))", req); + qParser.setParams(params); + q = qParser.getQuery(); + assertEquals(2, ((BooleanQuery)q).clauses().size()); + qq = ((BooleanQuery)q).clauses().get(0).getQuery(); + if (qq instanceof TermQuery) { + qq = ((BooleanQuery)q).clauses().get(1).getQuery(); + } + + if (qq instanceof FilterQuery) { + qq = ((FilterQuery)qq).getQuery(); + } + + assertEquals(26, ((TermInSetQuery) qq).getTermData().size()); + + // test mixed boolean query, including quotes (which shouldn't matter) + qParser = QParser.getParser("foo_s:(a +aaa b -bbb c d e f bar_s:(qqq www) g h i j k l m n o p q r s t u v w x y z)", req); + qParser.setIsFilter(true); // this may change in the future + qParser.setParams(params); + q = qParser.getQuery(); + assertEquals(4, ((BooleanQuery)q).clauses().size()); + qq = null; + for (BooleanClause clause : ((BooleanQuery)q).clauses()) { + qq = clause.getQuery(); + if (qq instanceof TermInSetQuery) break; + } + assertEquals(26, ((TermInSetQuery)qq).getTermData().size()); + + // test terms queries of two different fields (LUCENE-7637 changed to require all terms be in the same field) + StringBuilder sb = new StringBuilder(); + for (int i=0; i<17; i++) { + char letter = (char)('a'+i); + sb.append("foo_s:" + letter + " bar_s:" + letter + " "); + } + qParser = QParser.getParser(sb.toString(), req); + qParser.setIsFilter(true); // this may change in the future + qParser.setParams(params); + q = qParser.getQuery(); + assertEquals(2, ((BooleanQuery)q).clauses().size()); + for (BooleanClause clause : ((BooleanQuery)q).clauses()) { + qq = clause.getQuery(); + assertEquals(17, ((TermInSetQuery)qq).getTermData().size()); + } } - - // large relevancy query should use BooleanQuery - // TODO: we may decide that string fields shouldn't have relevance in the future... change to a text field w/o a stop filter if so - qParser = QParser.getParser("foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z)", req); - q = qParser.getQuery(); - assertEquals(26, ((BooleanQuery)q).clauses().size()); - - // large filter query should use TermsQuery - qParser = QParser.getParser("foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z)", req); - qParser.setIsFilter(true); // this may change in the future - q = qParser.getQuery(); - assertEquals(26, ((TermInSetQuery)q).getTermData().size()); - - // large numeric filter query should use TermsQuery (for trie fields) - qParser = QParser.getParser("foo_ti:(1 2 3 4 5 6 7 8 9 10 20 19 18 17 16 15 14 13 12 11)", req); - qParser.setIsFilter(true); // this may change in the future - q = qParser.getQuery(); - assertEquals(20, ((TermInSetQuery)q).getTermData().size()); - - // for point fields large filter query should use PointInSetQuery - qParser = QParser.getParser("foo_pi:(1 2 3 4 5 6 7 8 9 10 20 19 18 17 16 15 14 13 12 11)", req); - qParser.setIsFilter(true); // this may change in the future - q = qParser.getQuery(); - assertTrue(q instanceof PointInSetQuery); - assertEquals(20, ((PointInSetQuery)q).getPackedPoints().size()); - - // a filter() clause inside a relevancy query should be able to use a TermsQuery - qParser = QParser.getParser("foo_s:aaa filter(foo_s:(a b c d e f g h i j k l m n o p q r s t u v w x y z))", req); - q = qParser.getQuery(); - assertEquals(2, ((BooleanQuery)q).clauses().size()); - qq = ((BooleanQuery)q).clauses().get(0).getQuery(); - if (qq instanceof TermQuery) { - qq = ((BooleanQuery)q).clauses().get(1).getQuery(); - } - - if (qq instanceof FilterQuery) { - qq = ((FilterQuery)qq).getQuery(); - } - - assertEquals(26, ((TermInSetQuery)qq).getTermData().size()); - - // test mixed boolean query, including quotes (which shouldn't matter) - qParser = QParser.getParser("foo_s:(a +aaa b -bbb c d e f bar_s:(qqq www) g h i j k l m n o p q r s t u v w x y z)", req); - qParser.setIsFilter(true); // this may change in the future - q = qParser.getQuery(); - assertEquals(4, ((BooleanQuery)q).clauses().size()); - qq = null; - for (BooleanClause clause : ((BooleanQuery)q).clauses()) { - qq = clause.getQuery(); - if (qq instanceof TermInSetQuery) break; - } - assertEquals(26, ((TermInSetQuery)qq).getTermData().size()); - - // test terms queries of two different fields (LUCENE-7637 changed to require all terms be in the same field) - StringBuilder sb = new StringBuilder(); - for (int i=0; i<17; i++) { - char letter = (char)('a'+i); - sb.append("foo_s:" + letter + " bar_s:" + letter + " "); - } - qParser = QParser.getParser(sb.toString(), req); - qParser.setIsFilter(true); // this may change in the future - q = qParser.getQuery(); - assertEquals(2, ((BooleanQuery)q).clauses().size()); - for (BooleanClause clause : ((BooleanQuery)q).clauses()) { - qq = clause.getQuery(); - assertEquals(17, ((TermInSetQuery)qq).getTermData().size()); - } - req.close(); } @@ -306,6 +335,10 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 { // This will still fail when used as the main query, but will pass in a filter query since TermsQuery can be used. assertJQ(req("q","*:*", "fq", q) ,"/response/numFound==6"); + assertJQ(req("q","*:*", "fq", q, "sow", "false") + ,"/response/numFound==6"); + assertJQ(req("q","*:*", "fq", q, "sow", "true") + ,"/response/numFound==6"); } @Test @@ -540,4 +573,400 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 { req.close(); } + // LUCENE-7533 + public void testSplitOnWhitespace_with_autoGeneratePhraseQueries() throws Exception { + assertTrue(((TextField)h.getCore().getLatestSchema().getField("text").getType()).getAutoGeneratePhraseQueries()); + + try (SolrQueryRequest req = req()) { + final QParser qparser = QParser.getParser("{!lucene sow=false qf=text}blah blah", req); + expectThrows(QueryParserConfigurationException.class, qparser::getQuery); + } + } + + @Test + public void testSplitOnWhitespace_Basic() throws Exception { + // The "syn" field has synonyms loaded from synonyms.txt + + assertJQ(req("df", "syn", "q", "wifi", "sow", "true") // retrieve the single document containing literal "wifi" + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + + assertJQ(req("df", "syn", "q", "wi fi", "sow", "false") // trigger the "wi fi => wifi" synonym + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + + assertJQ(req("df", "syn", "q", "wi fi", "sow", "true") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi") // default sow=true + , "/response/numFound==0" + ); + + assertJQ(req("df", "syn", "q", "{!lucene sow=false}wi fi") + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", "{!lucene sow=true}wi fi") + , "/response/numFound==0" + ); + + assertJQ(req("df", "syn", "q", "{!lucene}wi fi") // default sow=true + , "/response/numFound==0" + ); + } + + public void testSplitOnWhitespace_Comments() throws Exception { + // The "syn" field has synonyms loaded from synonyms.txt + + assertJQ(req("df", "syn", "q", "wifi", "sow", "true") // retrieve the single document containing literal "wifi" + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", "wi fi", "sow", "false") // trigger the "wi fi => wifi" synonym + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", "wi /* foo */ fi", "sow", "false") // trigger the "wi fi => wifi" synonym + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", "wi /* foo */ /* bar */ fi", "sow", "false") // trigger the "wi fi => wifi" synonym + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", " /* foo */ wi fi /* bar */", "sow", "false") // trigger the "wi fi => wifi" synonym + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", " /* foo */ wi /* bar */ fi /* baz */", "sow", "false") // trigger the "wi fi => wifi" synonym + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + + assertJQ(req("df", "syn", "q", "wi fi", "sow", "true") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi /* foo */ fi", "sow", "true") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi /* foo */ /* bar */ fi", "sow", "true") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "/* foo */ wi fi /* bar */", "sow", "true") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "/* foo */ wi /* bar */ fi /* baz */", "sow", "true") + , "/response/numFound==0" + ); + + assertJQ(req("df", "syn", "q", "wi fi") // default sow=true + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi /* foo */ fi") // default sow=true + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi /* foo */ /* bar */ fi") // default sow=true + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "/* foo */ wi fi /* bar */") // default sow=true + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "/* foo */ wi /* bar */ fi /* baz */") // default sow=true + , "/response/numFound==0" + ); + + + assertJQ(req("df", "syn", "q", "{!lucene sow=false}wi fi") + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", "{!lucene sow=false}wi /* foo */ fi") + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", "{!lucene sow=false}wi /* foo */ /* bar */ fi") + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", "{!lucene sow=false}/* foo */ wi fi /* bar */") + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", "{!lucene sow=false}/* foo */ wi /* bar */ fi /* baz */") + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + + assertJQ(req("df", "syn", "q", "{!lucene sow=true}wi fi") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "{!lucene sow=true}wi /* foo */ fi") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "{!lucene sow=true}wi /* foo */ /* bar */ fi") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "{!lucene sow=true}/* foo */ wi fi /* bar */") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "{!lucene sow=true}/* foo */ wi /* bar */ fi /* baz */") + , "/response/numFound==0" + ); + + assertJQ(req("df", "syn", "q", "{!lucene}wi fi") // default sow=true + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "{!lucene}wi /* foo */ fi") // default sow=true + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "{!lucene}wi /* foo */ /* bar */ fi") // default sow=true + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "{!lucene}/* foo */ wi fi /* bar */") // default sow=true + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "{!lucene}/* foo */ wi /* bar */ fi /* baz */") // default sow=true + , "/response/numFound==0" + ); + } + + public void testOperatorsAndMultiWordSynonyms() throws Exception { + // The "syn" field has synonyms loaded from synonyms.txt + + assertJQ(req("df", "syn", "q", "wifi", "sow", "true") // retrieve the single document containing literal "wifi" + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + assertJQ(req("df", "syn", "q", "wi fi", "sow", "false") // trigger the "wi fi => wifi" synonym + , "/response/numFound==1" + , "/response/docs/[0]/id=='20'" + ); + + assertJQ(req("df", "syn", "q", "+wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "-wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "!wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi* fi", "sow", "false") // matches because wi* matches wifi + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "w? fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi~1 fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi^2 fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi^=2 fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi +fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi -fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi !fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi*", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi?", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi~1", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi^2", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi^=2", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "syn:wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi syn:fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "NOT wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi NOT fi", "sow", "false") + , "/response/numFound==0" + ); + + assertJQ(req("df", "syn", "q", "wi fi AND ATM", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "ATM AND wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi && ATM", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "ATM && wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "(wi fi) AND ATM", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "ATM AND (wi fi)", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "(wi fi) && ATM", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "ATM && (wi fi)", "sow", "false") + , "/response/numFound==1" + ); + + assertJQ(req("df", "syn", "q", "wi fi OR NotThereAtAll", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "NotThereAtAll OR wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi || NotThereAtAll", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "NotThereAtAll || wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "(wi fi) OR NotThereAtAll", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "NotThereAtAll OR (wi fi)", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "(wi fi) || NotThereAtAll", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "NotThereAtAll || (wi fi)", "sow", "false") + , "/response/numFound==1" + ); + + assertJQ(req("df", "syn", "q", "\"wi\" fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi \"fi\"", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "(wi) fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi (fi)", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "/wi/ fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi /fi/", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "(wi fi)", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "+(wi fi)", "sow", "false") + , "/response/numFound==1" + ); + + Map all = (Map)ObjectBuilder.fromJSON(h.query(req("q", "*:*", "rows", "0", "wt", "json"))); + int totalDocs = Integer.parseInt(((Map)all.get("response")).get("numFound").toString()); + int allDocsExceptOne = totalDocs - 1; + + assertJQ(req("df", "syn", "q", "-(wi fi)", "sow", "false") + , "/response/numFound==" + allDocsExceptOne // one doc contains "wifi" in the syn field + ); + assertJQ(req("df", "syn", "q", "!(wi fi)", "sow", "false") + , "/response/numFound==" + allDocsExceptOne // one doc contains "wifi" in the syn field + ); + assertJQ(req("df", "syn", "q", "NOT (wi fi)", "sow", "false") + , "/response/numFound==" + allDocsExceptOne // one doc contains "wifi" in the syn field + ); + assertJQ(req("df", "syn", "q", "(wi fi)^2", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "(wi fi)^=2", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "syn:(wi fi)", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "+ATM wi fi", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "-ATM wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "-NotThereAtAll wi fi", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "!ATM wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "!NotThereAtAll wi fi", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "NOT ATM wi fi", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "NOT NotThereAtAll wi fi", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "AT* wi fi", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "AT? wi fi", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "\"ATM\" wi fi", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "wi fi +ATM", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "wi fi -ATM", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi -NotThereAtAll", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "wi fi !ATM", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi !NotThereAtAll", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "wi fi NOT ATM", "sow", "false") + , "/response/numFound==0" + ); + assertJQ(req("df", "syn", "q", "wi fi NOT NotThereAtAll", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "wi fi AT*", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "wi fi AT?", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "wi fi \"ATM\"", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "\"wi fi\"~2", "sow", "false") + , "/response/numFound==1" + ); + assertJQ(req("df", "syn", "q", "syn:\"wi fi\"", "sow", "false") + , "/response/numFound==1" + ); + } } \ No newline at end of file From 4a55bc4e0f7a5b227f774fa3d7bbf4f1a4767eb1 Mon Sep 17 00:00:00 2001 From: David Smiley Date: Thu, 16 Mar 2017 21:11:39 -0400 Subject: [PATCH 029/563] SOLR-10286: fix precommit (unused imports) --- solr/core/src/test/org/apache/solr/search/LargeFieldTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java index 09e7e9034de..e32859bb2d4 100644 --- a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java +++ b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java @@ -24,9 +24,6 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.LazyDocument; import org.apache.lucene.index.IndexableField; import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.common.util.ContentStreamBase; -import org.apache.solr.request.SolrQueryRequestBase; -import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.schema.IndexSchema; import org.apache.solr.util.RefCounted; import org.junit.AfterClass; From 8fbd9f1e403cc697f77d827cd1aa85876c8665ae Mon Sep 17 00:00:00 2001 From: David Smiley Date: Thu, 16 Mar 2017 21:22:08 -0400 Subject: [PATCH 030/563] SOLR-10273: DocumentBuilder move longest field to last position --- solr/CHANGES.txt | 4 ++ .../apache/solr/update/DocumentBuilder.java | 60 ++++++++++++++++- .../solr/update/DocumentBuilderTest.java | 67 +++++++++++++++++++ 3 files changed, 130 insertions(+), 1 deletion(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 9a5299ccd29..dfe8d93393a 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -301,6 +301,10 @@ Optimizations * SOLR-10143: PointFields will create IndexOrDocValuesQuery when a field is both, indexed=true and docValues=true (Tomás Fernández Löbbe) +* SOLR-10273: The field with the longest value (if it exceeds 4K) is moved to be last in the Lucene Document in order + to benefit from stored field optimizations in Lucene that can avoid reading it when it's not needed. If the field is + multi-valued, they all move together to the end to retain order. (David Smiley) + Other Changes ---------------------- * SOLR-9980: Expose configVersion in core admin status (Jessica Cheng Mallet via Tomás Fernández Löbbe) diff --git a/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java b/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java index e3d20116fae..b97af3bcb05 100644 --- a/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java +++ b/solr/core/src/java/org/apache/solr/update/DocumentBuilder.java @@ -16,12 +16,15 @@ */ package org.apache.solr.update; +import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Set; import org.apache.lucene.document.Document; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.IndexableField; +import org.apache.lucene.util.BytesRef; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.SolrInputField; @@ -33,10 +36,13 @@ import org.apache.solr.schema.SchemaField; import com.google.common.collect.Sets; /** - * + * Builds a Lucene {@link Document} from a {@link SolrInputDocument}. */ public class DocumentBuilder { + // accessible only for tests + static int MIN_LENGTH_TO_MOVE_LAST = Integer.getInteger("solr.docBuilder.minLengthToMoveLast", 4*1024); // internal setting + /** * Add a field value to a given document. * @param doc Document that the field needs to be added to @@ -227,6 +233,58 @@ public class DocumentBuilder { } } } + + if (!forInPlaceUpdate) { + moveLargestFieldLast(out); + } + return out; } + + /** Move the largest stored field last, because Lucene can avoid loading that one if it's not needed. */ + private static void moveLargestFieldLast(Document doc) { + String largestField = null; + int largestFieldLen = -1; + boolean largestIsLast = true; + for (IndexableField field : doc) { + if (!field.fieldType().stored()) { + continue; + } + if (largestIsLast && !field.name().equals(largestField)) { + largestIsLast = false; + } + if (field.numericValue() != null) { // just ignore these as non-competitive (avoid toString'ing their number) + continue; + } + String strVal = field.stringValue(); + if (strVal != null) { + if (strVal.length() > largestFieldLen) { + largestField = field.name(); + largestFieldLen = strVal.length(); + largestIsLast = true; + } + } else { + BytesRef bytesRef = field.binaryValue(); + if (bytesRef != null && bytesRef.length > largestFieldLen) { + largestField = field.name(); + largestFieldLen = bytesRef.length; + largestIsLast = true; + } + } + } + if (!largestIsLast && largestField != null && largestFieldLen > MIN_LENGTH_TO_MOVE_LAST) { // only bother if the value isn't tiny + LinkedList addToEnd = new LinkedList<>(); + Iterator iterator = doc.iterator(); + while (iterator.hasNext()) { + IndexableField field = iterator.next(); + if (field.name().equals(largestField)) { + addToEnd.add(field); + iterator.remove(); // Document may not have "remove" but it's iterator allows mutation + } + } + for (IndexableField field : addToEnd) { + doc.add(field); + } + } + } } diff --git a/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java b/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java index 2a78d6b7923..03dd17cd7ef 100644 --- a/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java +++ b/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java @@ -16,7 +16,14 @@ */ package org.apache.solr.update; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import com.carrotsearch.randomizedtesting.generators.RandomStrings; import org.apache.lucene.document.Document; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.TestUtil; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.SolrDocument; @@ -25,6 +32,8 @@ import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.SolrInputField; import org.apache.solr.core.SolrCore; import org.apache.solr.schema.FieldType; +import org.junit.After; +import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -33,12 +42,23 @@ import org.junit.Test; * */ public class DocumentBuilderTest extends SolrTestCaseJ4 { + static final int save_min_len = DocumentBuilder.MIN_LENGTH_TO_MOVE_LAST; @BeforeClass public static void beforeClass() throws Exception { initCore("solrconfig.xml", "schema.xml"); } + @AfterClass + public static void afterClass() { + DocumentBuilder.MIN_LENGTH_TO_MOVE_LAST = save_min_len; + } + + @After + public void afterTest() { + DocumentBuilder.MIN_LENGTH_TO_MOVE_LAST = save_min_len; + } + @Test public void testBuildDocument() throws Exception { @@ -222,7 +242,54 @@ public class DocumentBuilderTest extends SolrTestCaseJ4 { sif2.setName("foo"); assertFalse(assertSolrInputFieldEquals(sif1, sif2)); + } + public void testMoveLargestLast() { + SolrInputDocument inDoc = new SolrInputDocument(); + String TEXT_FLD = "text"; // not stored. It won't be moved. This value is the longest, however. + inDoc.addField(TEXT_FLD, + "NOT STORED|" + RandomStrings.randomAsciiOfLength(random(), 4 * DocumentBuilder.MIN_LENGTH_TO_MOVE_LAST)); + + String CAT_FLD = "cat"; // stored, multiValued + inDoc.addField(CAT_FLD, + "STORED V1|"); + // pretty long value + inDoc.addField(CAT_FLD, + "STORED V2|" + RandomStrings.randomAsciiOfLength(random(), 2 * DocumentBuilder.MIN_LENGTH_TO_MOVE_LAST)); + inDoc.addField(CAT_FLD, + "STORED V3|" + RandomStrings.randomAsciiOfLength(random(), DocumentBuilder.MIN_LENGTH_TO_MOVE_LAST)); + + String SUBJECT_FLD = "subject"; // stored. This value is long, but not long enough. + inDoc.addField(SUBJECT_FLD, + "2ndplace|" + RandomStrings.randomAsciiOfLength(random(), DocumentBuilder.MIN_LENGTH_TO_MOVE_LAST)); + + Document outDoc = DocumentBuilder.toDocument(inDoc, h.getCore().getLatestSchema()); + + // filter outDoc by stored fields; convert to list. + List storedFields = StreamSupport.stream(outDoc.spliterator(), false) + .filter(f -> f.fieldType().stored()).collect(Collectors.toList()); + // clip to last 3. We expect these to be for CAT_FLD + storedFields = storedFields.subList(storedFields.size() - 3, storedFields.size()); + + Iterator fieldIterator = storedFields.iterator(); + IndexableField field; + + // Test that we retained the particular value ordering, even though though the 2nd of three was longest + + assertTrue(fieldIterator.hasNext()); + field = fieldIterator.next(); + assertEquals(CAT_FLD, field.name()); + assertTrue(field.stringValue().startsWith("STORED V1|")); + + assertTrue(fieldIterator.hasNext()); + field = fieldIterator.next(); + assertEquals(CAT_FLD, field.name()); + assertTrue(field.stringValue().startsWith("STORED V2|")); + + assertTrue(fieldIterator.hasNext()); + field = fieldIterator.next(); + assertEquals(CAT_FLD, field.name()); + assertTrue(field.stringValue().startsWith("STORED V3|")); } } From d374193e73e8c6025aaac72719757a230f0c8596 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Fri, 17 Mar 2017 11:55:59 +0100 Subject: [PATCH 031/563] Add 6.6 version --- lucene/CHANGES.txt | 3 +++ .../java/org/apache/lucene/util/Version.java | 7 +++++++ solr/CHANGES.txt | 17 +++++++++++++++++ 3 files changed, 27 insertions(+) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index bd38f3f30cb..b2ea412e8bf 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -80,6 +80,9 @@ Other * LUCENE-7681: MemoryIndex uses new DocValues API (Alan Woodward) +======================= Lucene 6.6.0 ======================= +(No Changes) + ======================= Lucene 6.5.0 ======================= API Changes diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java index 895f169ab01..da6d653bce7 100644 --- a/lucene/core/src/java/org/apache/lucene/util/Version.java +++ b/lucene/core/src/java/org/apache/lucene/util/Version.java @@ -101,6 +101,13 @@ public final class Version { @Deprecated public static final Version LUCENE_6_5_0 = new Version(6, 5, 0); + /** + * Match settings and bugs in Lucene's 6.6.0 release. + * @deprecated Use latest + */ + @Deprecated + public static final Version LUCENE_6_6_0 = new Version(6, 6, 0); + /** * Match settings and bugs in Lucene's 7.0.0 release. *

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index dfe8d93393a..1548410b718 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -88,6 +88,23 @@ Other Changes ---------------------- +================== 6.6.0 ================== + +Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. + +Versions of Major Components +--------------------- +Apache Tika 1.13 +Carrot2 3.15.0 +Velocity 1.7 and Velocity Tools 2.0 +Apache UIMA 2.3.1 +Apache ZooKeeper 3.4.6 +Jetty 9.3.14.v20161028 + + +(No Changes) + + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. From 3ca4d800babe68c39b8ea6e30ba0bdcc225ed907 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Fri, 17 Mar 2017 15:53:21 +0100 Subject: [PATCH 032/563] LUCENE_7747: QueryBuilder now iterates lazily over the possible paths when building a graph query --- lucene/CHANGES.txt | 2 + .../org/apache/lucene/util/QueryBuilder.java | 46 +++++++++---------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index b2ea412e8bf..22c83f0ef3a 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -229,6 +229,8 @@ Improvements * LUCENE-7695: ComplexPhraseQueryParser to support query time synonyms (Markus Jelsma via Mikhail Khludnev) +* LUCENE_7747: QueryBuilder now iterates lazily over the possible paths when building a graph query + Optimizations * LUCENE-7641: Optimized point range queries to compute documents that do not diff --git a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java index 0832bdb2637..9fb474a8fbe 100644 --- a/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java +++ b/lucene/core/src/java/org/apache/lucene/util/QueryBuilder.java @@ -19,7 +19,6 @@ package org.apache.lucene.util; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Iterator; import java.util.List; @@ -512,21 +511,20 @@ public class QueryBuilder { lastState = end; final Query queryPos; if (graph.hasSidePath(start)) { - List queries = new ArrayList<> (); - Iterator it = graph.getFiniteStrings(start, end); - while (it.hasNext()) { - TokenStream ts = it.next(); - // This is a synonym path so all terms are mandatory (MUST). - Query q = createFieldQuery(ts, BooleanClause.Occur.MUST, field, getAutoGenerateMultiTermSynonymsPhraseQuery(), 0); - if (q != null) { - queries.add(q); + final Iterator it = graph.getFiniteStrings(start, end); + Iterator queries = new Iterator() { + @Override + public boolean hasNext() { + return it.hasNext(); } - } - if (queries.size() > 0) { - queryPos = newGraphSynonymQuery(queries.toArray(new Query[queries.size()])); - } else { - queryPos = null; - } + + @Override + public Query next() { + TokenStream ts = it.next(); + return createFieldQuery(ts, BooleanClause.Occur.MUST, field, getAutoGenerateMultiTermSynonymsPhraseQuery(), 0); + } + }; + queryPos = newGraphSynonymQuery(queries); } else { Term[] terms = graph.getTerms(field, start); assert terms.length > 0; @@ -636,16 +634,16 @@ public class QueryBuilder { * This is intended for subclasses that wish to customize the generated queries. * @return new Query instance */ - protected Query newGraphSynonymQuery(Query queries[]) { - if (queries == null) { - return new BooleanQuery.Builder().build(); - } else if (queries.length == 1) { - return queries[0]; - } else { - BooleanQuery.Builder builder = new BooleanQuery.Builder(); - Arrays.stream(queries).forEachOrdered(qry -> builder.add(qry, BooleanClause.Occur.SHOULD)); - return builder.build(); + protected Query newGraphSynonymQuery(Iterator queries) { + BooleanQuery.Builder builder = new BooleanQuery.Builder(); + while (queries.hasNext()) { + builder.add(queries.next(), BooleanClause.Occur.SHOULD); } + BooleanQuery bq = builder.build(); + if (bq.clauses().size() == 1) { + return bq.clauses().get(0).getQuery(); + } + return bq; } /** From 540ee1db10b64aead7d7756b161c2c7348319d81 Mon Sep 17 00:00:00 2001 From: yonik Date: Fri, 17 Mar 2017 12:13:43 -0400 Subject: [PATCH 033/563] SOLR-7452: add refine param to json facets, implement for array field faceting --- .../search/facet/FacetFieldProcessor.java | 42 +++++- .../facet/FacetFieldProcessorByArray.java | 4 + .../FacetFieldProcessorByEnumTermsStream.java | 2 +- .../apache/solr/search/facet/FacetModule.java | 141 +++++++++++------- .../solr/search/facet/FacetProcessor.java | 27 +++- .../apache/solr/search/facet/FacetQuery.java | 6 +- .../apache/solr/search/facet/FacetRange.java | 4 +- .../solr/search/facet/FacetRequest.java | 3 + .../search/facet/TestJsonFacetRefinement.java | 94 ++++++++++++ 9 files changed, 254 insertions(+), 69 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java index bbc782cac7e..fb44f62f47b 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java @@ -31,6 +31,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Query; import org.apache.lucene.util.PriorityQueue; import org.apache.solr.common.util.SimpleOrderedMap; +import org.apache.solr.schema.FieldType; import org.apache.solr.schema.SchemaField; import org.apache.solr.search.DocSet; @@ -310,7 +311,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { if (freq.missing) { // TODO: it would be more efficient to build up a missing DocSet if we need it here anyway. SimpleOrderedMap missingBucket = new SimpleOrderedMap<>(); - fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null); + fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null, false); res.add("missing", missingBucket); } @@ -378,7 +379,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { } } - processSubs(target, filter, subDomain); + processSubs(target, filter, subDomain, false); } @Override @@ -510,4 +511,41 @@ abstract class FacetFieldProcessor extends FacetProcessor { } } } + + + + protected SimpleOrderedMap refineFacets() throws IOException { + List leaves = (List)fcontext.facetInfo.get("_l"); + + // For leaf refinements, we do full faceting for each leaf bucket. Any sub-facets of these buckets will be fully evaluated. Because of this, we should never + // encounter leaf refinements that have sub-facets that return partial results. + + SimpleOrderedMap res = new SimpleOrderedMap<>(); + List bucketList = new ArrayList<>(leaves.size()); + res.add("buckets", bucketList); + + // TODO: an alternate implementations can fill all accs at once + createAccs(-1, 1); + + FieldType ft = sf.getType(); + for (Object bucketVal : leaves) { + SimpleOrderedMap bucket = new SimpleOrderedMap<>(); + bucketList.add(bucket); + bucket.add("val", bucketVal); + + // String internal = ft.toInternal( tobj.toString() ); // TODO - we need a better way to get from object to query... + + Query domainQ = ft.getFieldQuery(null, sf, bucketVal.toString()); + + fillBucket(bucket, domainQ, null, false); + } + + // If there are just a couple of leaves, and if the domain is large, then + // going by term is likely the most efficient? + // If the domain is small, or if the number of leaves is large, then doing + // the normal collection method may be best. + + return res; + } + } diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java index 767bb55806d..95b9f0b82e9 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByArray.java @@ -57,6 +57,10 @@ abstract class FacetFieldProcessorByArray extends FacetFieldProcessor { } private SimpleOrderedMap calcFacets() throws IOException { + if (fcontext.facetInfo != null) { + return refineFacets(); + } + String prefix = freq.prefix; if (prefix == null || prefix.length() == 0) { prefixRef = null; diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java index 2feff15e1cc..94f3b2d4c1f 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java @@ -333,7 +333,7 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement bucket.add("val", bucketVal); addStats(bucket, 0); if (hasSubFacets) { - processSubs(bucket, bucketQuery, termSet); + processSubs(bucket, bucketQuery, termSet, false); } // TODO... termSet needs to stick around for streaming sub-facets? diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java index 87aaa8f47f4..630e96856f6 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java @@ -39,6 +39,7 @@ import org.apache.solr.search.QueryContext; import org.apache.solr.search.SyntaxError; import org.apache.solr.util.RTimer; import org.noggit.JSONUtil; +import org.noggit.ObjectBuilder; public class FacetModule extends SearchComponent { @@ -52,7 +53,7 @@ public class FacetModule extends SearchComponent { public final static int PURPOSE_REFINE_JSON_FACETS = 0x00200000; // Internal information passed down from the top level to shards for distributed faceting. - private final static String FACET_STATE = "_facet_"; + private final static String FACET_INFO = "_facet_"; private final static String FACET_REFINE = "refine"; @@ -62,6 +63,58 @@ public class FacetModule extends SearchComponent { return (FacetComponentState) rb.req.getContext().get(FacetComponentState.class); } + + @Override + public void prepare(ResponseBuilder rb) throws IOException { + Map json = rb.req.getJSON(); + Map jsonFacet = null; + if (json == null) { + int version = rb.req.getParams().getInt("facet.version",1); + if (version <= 1) return; + boolean facetsEnabled = rb.req.getParams().getBool(FacetParams.FACET, false); + if (!facetsEnabled) return; + jsonFacet = new LegacyFacet(rb.req.getParams()).getLegacy(); + } else { + jsonFacet = (Map) json.get("facet"); + } + if (jsonFacet == null) return; + + SolrParams params = rb.req.getParams(); + + boolean isShard = params.getBool(ShardParams.IS_SHARD, false); + Map facetInfo = null; + if (isShard) { + String jfacet = params.get(FACET_INFO); + if (jfacet == null) { + // if this is a shard request, but there is no _facet_ info, then don't do anything. + return; + } + facetInfo = (Map) ObjectBuilder.fromJSON(jfacet); + } + + // At this point, we know we need to do something. Create and save the state. + rb.setNeedDocSet(true); + + // Parse the facet in the prepare phase? + FacetParser parser = new FacetTopParser(rb.req); + FacetRequest facetRequest = null; + try { + facetRequest = parser.parse(jsonFacet); + } catch (SyntaxError syntaxError) { + throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, syntaxError); + } + + FacetComponentState fcState = new FacetComponentState(); + fcState.rb = rb; + fcState.isShard = isShard; + fcState.facetInfo = facetInfo; + fcState.facetCommands = jsonFacet; + fcState.facetRequest = facetRequest; + + rb.req.getContext().put(FacetComponentState.class, fcState); + } + + @Override public void process(ResponseBuilder rb) throws IOException { // if this is null, faceting is not enabled @@ -77,6 +130,11 @@ public class FacetModule extends SearchComponent { fcontext.qcontext = QueryContext.newContext(fcontext.searcher); if (isShard) { fcontext.flags |= FacetContext.IS_SHARD; + fcontext.facetInfo = facetState.facetInfo.isEmpty() ? null : (Map)facetState.facetInfo.get(FACET_REFINE); + if (fcontext.facetInfo != null) { + fcontext.flags |= FacetContext.IS_REFINEMENT; + fcontext.flags |= FacetContext.SKIP_FACET; // the root bucket should have been received from all shards previously + } } FacetProcessor fproc = facetState.facetRequest.createFacetProcessor(fcontext); @@ -100,60 +158,14 @@ public class FacetModule extends SearchComponent { } - @Override - public void prepare(ResponseBuilder rb) throws IOException { - Map json = rb.req.getJSON(); - Map jsonFacet = null; - if (json == null) { - int version = rb.req.getParams().getInt("facet.version",1); - if (version <= 1) return; - boolean facetsEnabled = rb.req.getParams().getBool(FacetParams.FACET, false); - if (!facetsEnabled) return; - jsonFacet = new LegacyFacet(rb.req.getParams()).getLegacy(); - } else { - jsonFacet = (Map) json.get("facet"); - } - if (jsonFacet == null) return; - - SolrParams params = rb.req.getParams(); - - boolean isShard = params.getBool(ShardParams.IS_SHARD, false); - if (isShard) { - String jfacet = params.get(FACET_STATE); - if (jfacet == null) { - // if this is a shard request, but there is no facet state, then don't do anything. - return; - } - } - - // At this point, we know we need to do something. Create and save the state. - rb.setNeedDocSet(true); - - // Parse the facet in the prepare phase? - FacetParser parser = new FacetTopParser(rb.req); - FacetRequest facetRequest = null; - try { - facetRequest = parser.parse(jsonFacet); - } catch (SyntaxError syntaxError) { - throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, syntaxError); - } - - FacetComponentState fcState = new FacetComponentState(); - fcState.rb = rb; - fcState.isShard = isShard; - fcState.facetCommands = jsonFacet; - fcState.facetRequest = facetRequest; - - rb.req.getContext().put(FacetComponentState.class, fcState); - } private void clearFaceting(List outgoing) { // turn off faceting for requests not marked as being for faceting refinements for (ShardRequest sreq : outgoing) { if ((sreq.purpose & PURPOSE_REFINE_JSON_FACETS) != 0) continue; - sreq.params.remove("json.facet"); // this just saves space... the presence of FACET_STATE really control the faceting - sreq.params.remove(FACET_STATE); + sreq.params.remove("json.facet"); // this just saves space... the presence of FACET_INFO is enough to control the faceting + sreq.params.remove(FACET_INFO); } } @@ -215,16 +227,15 @@ public class FacetModule extends SearchComponent { // don't request any documents shardsRefineRequest.params.remove(CommonParams.START); shardsRefineRequest.params.set(CommonParams.ROWS, "0"); - shardsRefineRequest.params.set(CommonParams.ROWS, "0"); shardsRefineRequest.params.set(FacetParams.FACET, false); } shardsRefineRequest.purpose |= PURPOSE_REFINE_JSON_FACETS; - Map fstate = new HashMap<>(1); - fstate.put(FACET_REFINE, refinement); - String fstateString = JSONUtil.toJSON(fstate); - shardsRefineRequest.params.add(FACET_STATE, fstateString); + Map finfo = new HashMap<>(1); + finfo.put(FACET_REFINE, refinement); + String finfoStr = JSONUtil.toJSON(finfo); + shardsRefineRequest.params.add(FACET_INFO, finfoStr); if (newRequest) { rb.addRequest(this, shardsRefineRequest); @@ -242,12 +253,12 @@ public class FacetModule extends SearchComponent { if ((sreq.purpose & ShardRequest.PURPOSE_GET_TOP_IDS) != 0) { sreq.purpose |= FacetModule.PURPOSE_GET_JSON_FACETS; - sreq.params.set(FACET_STATE, "{}"); // The presence of FACET_STATE (_facet_) turns on json faceting + sreq.params.set(FACET_INFO, "{}"); // The presence of FACET_INFO (_facet_) turns on json faceting } else { // turn off faceting on other requests /*** distributedProcess will need to use other requests for refinement - sreq.params.remove("json.facet"); // this just saves space... the presence of FACET_STATE really control the faceting - sreq.params.remove(FACET_STATE); + sreq.params.remove("json.facet"); // this just saves space... the presence of FACET_INFO really control the faceting + sreq.params.remove(FACET_INFO); **/ } } @@ -267,6 +278,18 @@ public class FacetModule extends SearchComponent { facetState.merger = facetState.facetRequest.createFacetMerger(facet); facetState.mcontext = new FacetMerger.Context( sreq.responses.size() ); } + + if ((sreq.purpose & PURPOSE_REFINE_JSON_FACETS) != 0) { + System.err.println("REFINE FACET RESULT FROM SHARD = " + facet); + // call merge again with a diff flag set on the context??? +// throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "WORK IN PROGRESS, MERGING FACET REFINEMENT NOT SUPPORTED YET!"); + + facetState.mcontext.root = facet; + facetState.mcontext.setShard(shardRsp.getShard()); // TODO: roll newShard into setShard? + facetState.merger.merge(facet , facetState.mcontext); + return; + } + facetState.mcontext.root = facet; facetState.mcontext.newShard(shardRsp.getShard()); facetState.merger.merge(facet , facetState.mcontext); @@ -304,11 +327,15 @@ public class FacetModule extends SearchComponent { } +// TODO: perhaps factor out some sort of root/parent facet object that doesn't depend +// on stuff like ResponseBuilder, but contains request parameters, +// root filter lists (for filter exclusions), etc? class FacetComponentState { ResponseBuilder rb; Map facetCommands; FacetRequest facetRequest; boolean isShard; + Map facetInfo; // _facet_ param: contains out-of-band facet info, mainly for refinement requests // // Only used for distributed search diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java index 4a839a29ccf..de6dd722d02 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java @@ -366,10 +366,13 @@ public abstract class FacetProcessor { } } - void fillBucket(SimpleOrderedMap bucket, Query q, DocSet result) throws IOException { + // TODO: rather than just have a raw "response", perhaps we should model as a bucket object that contains the response plus extra info? + void fillBucket(SimpleOrderedMap bucket, Query q, DocSet result, boolean skip) throws IOException { + + // TODO: we don't need the DocSet if we've already calculated everything during the first phase boolean needDocSet = freq.getFacetStats().size() > 0 || freq.getSubFacets().size() > 0; - // TODO: always collect counts or not??? + // TODO: put info in for the merger (like "skip=true"?) Maybe we don't need to if we leave out all extraneous info? int count; @@ -382,7 +385,7 @@ public abstract class FacetProcessor { } else { result = fcontext.searcher.getDocSet(q, fcontext.base); } - count = result.size(); + count = result.size(); // don't really need this if we are skipping, but it's free. } else { if (q == null) { count = fcontext.base.size(); @@ -392,8 +395,10 @@ public abstract class FacetProcessor { } try { - processStats(bucket, result, count); - processSubs(bucket, q, result); + if (!skip) { + processStats(bucket, result, count); + } + processSubs(bucket, q, result, skip); } finally { if (result != null) { // result.decref(); // OFF-HEAP @@ -402,7 +407,7 @@ public abstract class FacetProcessor { } } - void processSubs(SimpleOrderedMap response, Query filter, DocSet domain) throws IOException { + void processSubs(SimpleOrderedMap response, Query filter, DocSet domain, boolean skip) throws IOException { boolean emptyDomain = domain == null || domain.size() == 0; @@ -417,8 +422,18 @@ public abstract class FacetProcessor { continue; } + MapfacetInfoSub = null; + if (fcontext.facetInfo != null) { + facetInfoSub = (Map)fcontext.facetInfo.get(sub.getKey()); + } + + // If we're skipping this node, then we only need to process sub-facets that have facet info specified. + if (skip && facetInfoSub == null) continue; + // make a new context for each sub-facet since they can change the domain FacetContext subContext = fcontext.sub(filter, domain); + subContext.facetInfo = facetInfoSub; + if (!skip) subContext.flags &= ~FacetContext.SKIP_FACET; // turn off the skip flag if we're not skipping this bucket FacetProcessor subProcessor = subRequest.createFacetProcessor(subContext); if (fcontext.getDebugInfo() != null) { // if fcontext.debugInfo != null, it means rb.debug() == true diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java b/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java index 174b832f006..584bec344c4 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java @@ -56,8 +56,12 @@ class FacetQueryProcessor extends FacetProcessor { @Override public void process() throws IOException { super.process(); + + if (fcontext.facetInfo != null) { + // FIXME - what needs to be done here? + } response = new SimpleOrderedMap<>(); - fillBucket(response, freq.q, null); + fillBucket(response, freq.q, null, (fcontext.flags & FacetContext.SKIP_FACET)!=0); } 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 a50fa2c9811..5d0989bd8d7 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 @@ -350,7 +350,7 @@ class FacetRangeProcessor extends FacetProcessor { if (freq.getSubFacets().size() > 0) { DocSet subBase = intersections[slot]; try { - processSubs(bucket, filters[slot], subBase); + processSubs(bucket, filters[slot], subBase, false); } finally { // subContext.base.decref(); // OFF-HEAP // subContext.base = null; // do not modify context after creation... there may be deferred execution (i.e. streaming) @@ -367,7 +367,7 @@ class FacetRangeProcessor extends FacetProcessor { } Query rangeQ = sf.getType().getRangeQuery(null, sf, range.low == null ? null : calc.formatValue(range.low), range.high==null ? null : calc.formatValue(range.high), range.includeLower, range.includeUpper); - fillBucket(bucket, rangeQ, null); + fillBucket(bucket, rangeQ, null, false); return bucket; } diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java index 636460f7e60..9835f7d7165 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java @@ -168,7 +168,10 @@ public abstract class FacetRequest { class FacetContext { // Context info for actually executing a local facet command public static final int IS_SHARD=0x01; + public static final int IS_REFINEMENT=0x02; + public static final int SKIP_FACET=0x04; // refinement: skip calculating this immediate facet, but proceed to specific sub-facets based on facetInfo + Map facetInfo; // refinement info for this node QueryContext qcontext; SolrQueryRequest req; // TODO: replace with params? SolrIndexSearcher searcher; diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java index a8f8ff207e3..f23ae8c297a 100644 --- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java +++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java @@ -18,9 +18,12 @@ package org.apache.solr.search.facet; import java.io.IOException; +import java.util.List; import org.apache.solr.JSONTestUtil; import org.apache.solr.SolrTestCaseHS; +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.request.SolrQueryRequest; import org.junit.AfterClass; @@ -209,6 +212,97 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { } + @Test + public void testBasicRefinement() throws Exception { + initServers(); + Client client = servers.getClient(random().nextInt()); + client.queryDefaults().set( "shards", servers.getShards(), "debugQuery", Boolean.toString(random().nextBoolean()) ); + + List clients = client.getClientProvider().all(); + assertTrue(clients.size() >= 3); + + client.deleteByQuery("*:*", null); + + ModifiableSolrParams p = params("cat_s", "cat_s", "num_d", "num_d"); + String cat_s = p.get("cat_s"); + String num_d = p.get("num_d"); + + clients.get(0).add( sdoc("id", "01", cat_s, "A", num_d, -1) ); // A wins count tie + clients.get(0).add( sdoc("id", "02", cat_s, "B", num_d, 3) ); + + clients.get(1).add( sdoc("id", "11", cat_s, "B", num_d, -5) ); // B highest count + clients.get(1).add( sdoc("id", "12", cat_s, "B", num_d, -11) ); + clients.get(1).add( sdoc("id", "13", cat_s, "A", num_d, 7) ); + + clients.get(2).add( sdoc("id", "21", cat_s, "A", num_d, 17) ); // A highest count + clients.get(2).add( sdoc("id", "22", cat_s, "A", num_d, -19) ); + clients.get(2).add( sdoc("id", "23", cat_s, "B", num_d, 11) ); + + client.commit(); + + // Shard responses should be A=1, B=2, A=2, merged should be "A=3, B=2" + // One shard will have _facet_={"refine":{"cat0":{"_l":["A"]}}} on the second phase + + /**** + // fake a refinement request... good for development/debugging + assertJQ(clients.get(1), + params(p, "q", "*:*", "_facet_","{refine:{cat0:{_l:[A]}}}", "isShard","true", "distrib","false", "shards.purpose","2097216", "ids","11,12,13", + "json.facet", "{" + + "cat0:{type:terms, field:cat_s, sort:'count desc', limit:1, overrequest:0, refine:true}" + + "}" + ) + , "facets=={foo:555}" + ); + ****/ + + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "cat0:{type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:false}" + + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:A,count:3} ] }" + // w/o overrequest and refinement, count is lower than it should be (we don't see the A from the middle shard) + "}" + ); + + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "cat0:{type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true}" + + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:A,count:4} ] }" + // w/o overrequest, we need refining to get the correct count. + "}" + ); + + // test that basic stats work for refinement + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "cat0:{type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:true, facet:{ stat1:'sum(${num_d})'} }" + + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:A,count:4, stat1:4.0} ] }" + + "}" + ); + + // test sorting buckets by a different stat + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + " cat0:{type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:false, facet:{ min1:'min(${num_d})'} }" + + ",cat1:{type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} }" + + ",sum1:'sum(num_d)'" + // make sure that root bucket stats aren't affected by refinement + "}" + ) + , "facets=={ count:8" + + ", cat0:{ buckets:[ {val:A,count:3, min1:-19.0} ] }" + // B wins in shard2, so we're missing the "A" count for that shar w/o refinement. + ", cat1:{ buckets:[ {val:A,count:4, min1:-19.0} ] }" + // with refinement, we get the right count + ", sum1:2.0" + + "}" + ); + + + } } From 3b660018457234387558ff626c8b95bb6f4ce853 Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Fri, 17 Mar 2017 11:55:15 -0700 Subject: [PATCH 034/563] SOLR-10237: Poly-Fields should work with subfield that have docValues=true --- solr/CHANGES.txt | 2 + .../solr/schema/AbstractSubTypeFieldType.java | 17 +++++--- .../org/apache/solr/schema/LatLonType.java | 12 +++++- .../org/apache/solr/schema/PointType.java | 14 +++++-- .../solr/collection1/conf/schema.xml | 15 ++++--- .../org/apache/solr/schema/PolyFieldTest.java | 40 ++++++++++++------- .../solr/update/DocumentBuilderTest.java | 4 +- 7 files changed, 68 insertions(+), 36 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 1548410b718..75ac5bb9e3c 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -303,6 +303,8 @@ Bug Fixes * SOLR-10283: Learning to Rank (LTR) SolrFeature to reject searches with missing efi (External Feature Information) used by fq. (Christine Poerschke) +* SOLR-10237: Poly-fields should work with subfields that have docValues=true (Tomás Fernández Löbbe, David Smiley) + Optimizations ---------------------- diff --git a/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java b/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java index 11848769e52..73a4f3c47d4 100644 --- a/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java +++ b/solr/core/src/java/org/apache/solr/schema/AbstractSubTypeFieldType.java @@ -76,20 +76,25 @@ public abstract class AbstractSubTypeFieldType extends FieldType implements Sche * and props of indexed=true, stored=false. * * @param schema the IndexSchema - * @param type The {@link FieldType} of the prototype. + * @param subType The {@link FieldType} of the prototype. + * @param polyField The poly {@link FieldType}. * @return The {@link SchemaField} */ - static SchemaField registerPolyFieldDynamicPrototype(IndexSchema schema, FieldType type) { - String name = "*" + FieldType.POLY_FIELD_SEPARATOR + type.typeName; + static SchemaField registerPolyFieldDynamicPrototype(IndexSchema schema, FieldType subType, FieldType polyField) { + String name = "*" + FieldType.POLY_FIELD_SEPARATOR + subType.typeName; Map props = new HashMap<>(); //Just set these, delegate everything else to the field type props.put("indexed", "true"); props.put("stored", "false"); props.put("multiValued", "false"); - int p = SchemaField.calcProps(name, type, props); + // if polyField enables dv, add them to the subtypes + if (polyField.hasProperty(DOC_VALUES)) { + props.put("docValues", "true"); + } + int p = SchemaField.calcProps(name, subType, props); SchemaField proto = SchemaField.create(name, - type, p, null); + subType, p, null); schema.registerDynamicFields(proto); return proto; } @@ -107,7 +112,7 @@ public abstract class AbstractSubTypeFieldType extends FieldType implements Sche this.schema = schema; //Can't do this until here b/c the Dynamic Fields are not initialized until here. if (subType != null) { - SchemaField proto = registerPolyFieldDynamicPrototype(schema, subType); + SchemaField proto = registerPolyFieldDynamicPrototype(schema, subType, this); dynFieldProps = proto.getProperties(); } } diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonType.java b/solr/core/src/java/org/apache/solr/schema/LatLonType.java index c484f3a8f9f..8c4e19a47aa 100644 --- a/solr/core/src/java/org/apache/solr/schema/LatLonType.java +++ b/solr/core/src/java/org/apache/solr/schema/LatLonType.java @@ -75,10 +75,10 @@ public class LatLonType extends AbstractSubTypeFieldType implements SpatialQuery Point point = SpatialUtils.parsePointSolrException(externalVal, SpatialContext.GEO); //latitude SchemaField subLatSF = subField(field, LAT, schema); - f.add(subLatSF.createField(String.valueOf(point.getY()))); + f.addAll(subLatSF.createFields(String.valueOf(point.getY()))); //longitude SchemaField subLonSF = subField(field, LON, schema); - f.add(subLonSF.createField(String.valueOf(point.getX()))); + f.addAll(subLonSF.createFields(String.valueOf(point.getX()))); } if (field.stored()) { @@ -86,6 +86,14 @@ public class LatLonType extends AbstractSubTypeFieldType implements SpatialQuery } return f; } + + @Override + protected void checkSupportsDocValues() { + // DocValues supported only when enabled at the fieldType + if (!hasProperty(DOC_VALUES)) { + throw new UnsupportedOperationException("LatLonType can't have docValues=true in the field definition, use docValues=true in the fieldType definition, or in subFieldType/subFieldSuffix"); + } + } @Override diff --git a/solr/core/src/java/org/apache/solr/schema/PointType.java b/solr/core/src/java/org/apache/solr/schema/PointType.java index 4c022b85001..e088e7ff31d 100644 --- a/solr/core/src/java/org/apache/solr/schema/PointType.java +++ b/solr/core/src/java/org/apache/solr/schema/PointType.java @@ -71,12 +71,12 @@ public class PointType extends CoordinateFieldType implements SpatialQueryable { String[] point = parseCommaSeparatedList(externalVal, dimension); // TODO: this doesn't currently support polyFields as sub-field types - List f = new ArrayList<>(dimension+1); + List f = new ArrayList<>((dimension*2)+1); if (field.indexed()) { for (int i=0; i - - - - - + + + + - + @@ -621,7 +620,7 @@ - + @@ -671,7 +670,7 @@ - + fields = home.createFields(point); - assertEquals(fields.size(), 3);//should be 3, we have a stored field - //first two fields contain the values, third is just stored and contains the original - for (int i = 0; i < 3; i++) { + assertNotNull(pt.getSubType()); + int expectdNumFields = 3;//If DV=false, we expect one field per dimension plus a stored field + if (pt.subField(home, 0, schema).hasDocValues()) { + expectdNumFields+=2; // If docValues=true, then we expect two more fields + } + assertEquals("Unexpected fields created: " + Arrays.toString(fields.toArray()), expectdNumFields, fields.size()); + //first two/four fields contain the values, last one is just stored and contains the original + for (int i = 0; i < expectdNumFields; i++) { boolean hasValue = fields.get(i).binaryValue() != null || fields.get(i).stringValue() != null || fields.get(i).numericValue() != null; @@ -100,7 +111,7 @@ public class PolyFieldTest extends SolrTestCaseJ4 { home = schema.getField("home_ns"); assertNotNull(home); fields = home.createFields(point); - assertEquals(fields.size(), 2);//should be 2, since we aren't storing + assertEquals(expectdNumFields - 1, fields.size(), 2);//one less field than with "home", since we aren't storing home = schema.getField("home_ns"); assertNotNull(home); @@ -111,17 +122,12 @@ public class PolyFieldTest extends SolrTestCaseJ4 { // } - SchemaField s1 = schema.getField("test_p"); SchemaField s2 = schema.getField("test_p"); - // If we use [Int/Double/Long/Float]PointField, we can't get the valueSource, since docValues is false - if (s1.createFields("1,2").get(0).fieldType().pointDimensionCount() == 0) { - assertFalse(s2.getType().isPointField()); - ValueSource v1 = s1.getType().getValueSource(s1, null); - ValueSource v2 = s2.getType().getValueSource(s2, null); - assertEquals(v1, v2); - assertEquals(v1.hashCode(), v2.hashCode()); - } + ValueSource v1 = s1.getType().getValueSource(s1, null); + ValueSource v2 = s2.getType().getValueSource(s2, null); + assertEquals(v1, v2); + assertEquals(v1.hashCode(), v2.hashCode()); } @Test @@ -181,5 +187,9 @@ public class PolyFieldTest extends SolrTestCaseJ4 { assertEquals(2, bq.clauses().size()); clearIndex(); } + + private boolean usingPointFields() { + return h.getCore().getLatestSchema().getField("foo_d1_dv").getType().isPointField(); + } } diff --git a/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java b/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java index 03dd17cd7ef..5d98d8be410 100644 --- a/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java +++ b/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java @@ -131,8 +131,8 @@ public class DocumentBuilderTest extends SolrTestCaseJ4 { doc.addField( "home", "2.2,3.3" ); Document out = DocumentBuilder.toDocument( doc, core.getLatestSchema() ); assertNotNull( out.get( "home" ) );//contains the stored value and term vector, if there is one - assertNotNull( out.getField( "home_0" + FieldType.POLY_FIELD_SEPARATOR + "double" ) ); - assertNotNull( out.getField( "home_1" + FieldType.POLY_FIELD_SEPARATOR + "double" ) ); + assertNotNull( out.getField( "home_0" + FieldType.POLY_FIELD_SEPARATOR + System.getProperty("solr.tests.doubleClass", "pdouble") ) ); + assertNotNull( out.getField( "home_1" + FieldType.POLY_FIELD_SEPARATOR + System.getProperty("solr.tests.doubleClass", "pdouble") ) ); } /** From 258fddb91bd5cd8fb87e7fd38ffa31077cd91c6f Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Fri, 17 Mar 2017 13:45:29 -0700 Subject: [PATCH 035/563] Fix CHANGES.txt --- lucene/CHANGES.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 22c83f0ef3a..15f4ff3458a 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -229,7 +229,8 @@ Improvements * LUCENE-7695: ComplexPhraseQueryParser to support query time synonyms (Markus Jelsma via Mikhail Khludnev) -* LUCENE_7747: QueryBuilder now iterates lazily over the possible paths when building a graph query +* LUCENE-7747: QueryBuilder now iterates lazily over the possible paths when building a graph query + (Jim Ferenczi) Optimizations From 9b57545df01fd0f20fd473e80d4fba621a51abf9 Mon Sep 17 00:00:00 2001 From: Steve Rowe Date: Sat, 18 Mar 2017 00:00:59 -0400 Subject: [PATCH 036/563] SOLR-10218: The Schema API commands add-field-type and replace-field-type improperly specify SimilarityFactory params --- solr/CHANGES.txt | 3 + .../solr/rest/schema/FieldTypeXmlAdapter.java | 25 ++++++- .../solr/rest/schema/TestBulkSchemaAPI.java | 72 ++++++++++++++++++- 3 files changed, 97 insertions(+), 3 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 75ac5bb9e3c..b7e55cc9a4f 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -305,6 +305,9 @@ Bug Fixes * SOLR-10237: Poly-fields should work with subfields that have docValues=true (Tomás Fernández Löbbe, David Smiley) +* SOLR-10218: The Schema API commands "add-field-type" and "replace-field-type" improperly specify SimilarityFactory params. + (Benjamin Deininger, Troy Mohl, Steve Rowe) + Optimizations ---------------------- diff --git a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java index a6740405bc3..d25ab89c93e 100644 --- a/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java +++ b/solr/core/src/java/org/apache/solr/rest/schema/FieldTypeXmlAdapter.java @@ -25,7 +25,9 @@ import javax.xml.parsers.ParserConfigurationException; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.schema.IndexSchema; +import org.apache.solr.schema.SimilarityFactory; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.Node; @@ -83,9 +85,30 @@ public class FieldTypeXmlAdapter { jsonFieldName+" not a "+jsonField.getClass().getName()); Element similarity = doc.createElement("similarity"); - appendAttrs(similarity, (Map)jsonField); + Map config = (Map)jsonField; + similarity.setAttribute(SimilarityFactory.CLASS_NAME, (String)config.remove(SimilarityFactory.CLASS_NAME)); + for (Map.Entry entry : config.entrySet()) { + Object val = entry.getValue(); + if (val != null) { + Element child = doc.createElement(classToXmlTag(val.getClass())); + child.setAttribute(CommonParams.NAME, entry.getKey()); + child.setTextContent(entry.getValue().toString()); + similarity.appendChild(child); + } + } return similarity; } + + /** Convert types produced by noggit's ObjectBuilder (Boolean, Double, Long, String) to plugin param XML tags. */ + protected static String classToXmlTag(Class clazz) { + switch (clazz.getSimpleName()) { + case "Boolean": return "bool"; + case "Double": return "double"; + case "Long": return "long"; + case "String": return "str"; + } + throw new SolrException(ErrorCode.BAD_REQUEST, "Unsupported object type '" + clazz.getSimpleName() + "'"); + } @SuppressWarnings("unchecked") protected static Element transformAnalyzer(Document doc, Map json, String jsonFieldName, String analyzerType) { diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java index e2dc2bfd5d4..166d1fc183c 100644 --- a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java +++ b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java @@ -18,6 +18,7 @@ package org.apache.solr.rest.schema; import org.apache.commons.io.FileUtils; +import org.apache.lucene.search.similarities.DFISimilarity; import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper; import org.apache.lucene.search.similarities.BM25Similarity; import org.apache.lucene.misc.SweetSpotSimilarity; @@ -42,10 +43,12 @@ import java.io.File; import java.io.StringReader; import java.lang.invoke.MethodHandles; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Consumer; public class TestBulkSchemaAPI extends RestTestBase { @@ -798,6 +801,68 @@ public class TestBulkSchemaAPI extends RestTestBase { assertNull(map.get("errors")); } + public void testSimilarityParser() throws Exception { + RestTestHarness harness = restTestHarness; + + final float k1 = 2.25f; + final float b = 0.33f; + + String fieldTypeName = "MySimilarityField"; + String fieldName = "similarityTestField"; + String payload = "{\n" + + " 'add-field-type' : {" + + " 'name' : '" + fieldTypeName + "',\n" + + " 'class':'solr.TextField',\n" + + " 'analyzer' : {'tokenizer':{'class':'solr.WhitespaceTokenizerFactory'}},\n" + + " 'similarity' : {'class':'org.apache.solr.search.similarities.BM25SimilarityFactory', 'k1':"+k1+", 'b':"+b+" }\n" + + " },\n"+ + " 'add-field' : {\n" + + " 'name':'" + fieldName + "',\n" + + " 'type': 'MySimilarityField',\n" + + " 'stored':true,\n" + + " 'indexed':true\n" + + " }\n" + + "}\n"; + + String response = harness.post("/schema?wt=json&indent=on", json(payload)); + + Map map = (Map) ObjectBuilder.getVal(new JSONParser(new StringReader(response))); + assertNull(response, map.get("errors")); + + Map fields = getObj(harness, fieldName, "fields"); + assertNotNull("field " + fieldName + " not created", fields); + + assertFieldSimilarity(fieldName, BM25Similarity.class, + sim -> assertEquals("Unexpected k1", k1, sim.getK1(), .001), + sim -> assertEquals("Unexpected b", b, sim.getB(), .001)); + + final String independenceMeasure = "Saturated"; + final boolean discountOverlaps = false; + payload = "{\n" + + " 'replace-field-type' : {" + + " 'name' : '" + fieldTypeName + "',\n" + + " 'class':'solr.TextField',\n" + + " 'analyzer' : {'tokenizer':{'class':'solr.WhitespaceTokenizerFactory'}},\n" + + " 'similarity' : {\n" + + " 'class':'org.apache.solr.search.similarities.DFISimilarityFactory',\n" + + " 'independenceMeasure':'" + independenceMeasure + "',\n" + + " 'discountOverlaps':" + discountOverlaps + "\n" + + " }\n" + + " }\n"+ + "}\n"; + + response = harness.post("/schema?wt=json&indent=on", json(payload)); + + map = (Map)ObjectBuilder.getVal(new JSONParser(new StringReader(response))); + assertNull(response, map.get("errors")); + fields = getObj(harness, fieldName, "fields"); + assertNotNull("field " + fieldName + " not created", fields); + + assertFieldSimilarity(fieldName, DFISimilarity.class, + sim -> assertEquals("Unexpected independenceMeasure", independenceMeasure, sim.getIndependence().toString()), + sim -> assertEquals("Unexpected discountedOverlaps", discountOverlaps, sim.getDiscountOverlaps())); + } + public static Map getObj(RestTestHarness restHarness, String fld, String key) throws Exception { Map map = getRespMap(restHarness); List l = (List) ((Map)map.get("schema")).get(key); @@ -842,8 +907,11 @@ public class TestBulkSchemaAPI extends RestTestBase { /** * whitebox checks the Similarity for the specified field according to {@link SolrCore#getLatestSchema} + * + * Executes each of the specified Similarity-accepting validators. */ - private static void assertFieldSimilarity(String fieldname, Class expected) { + @SafeVarargs + private static void assertFieldSimilarity(String fieldname, Class expected, Consumer... validators) { CoreContainer cc = jetty.getCoreContainer(); try (SolrCore core = cc.getCore("collection1")) { SimilarityFactory simfac = core.getLatestSchema().getSimilarityFactory(); @@ -861,7 +929,7 @@ public class TestBulkSchemaAPI extends RestTestBase { mainSim instanceof PerFieldSimilarityWrapper); Similarity fieldSim = ((PerFieldSimilarityWrapper)mainSim).get(fieldname); assertEquals("wrong sim for field=" + fieldname, expected, fieldSim.getClass()); - + Arrays.asList(validators).forEach(v -> v.accept((T)fieldSim)); } } } From e7d9db9d6c4dccc158b52d53584ead93b7f55c38 Mon Sep 17 00:00:00 2001 From: David Smiley Date: Sat, 18 Mar 2017 10:42:34 -0400 Subject: [PATCH 037/563] SOLR-10286: fix test for Windows --- solr/core/src/test/org/apache/solr/search/LargeFieldTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java index e32859bb2d4..57dc2de27af 100644 --- a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java +++ b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java @@ -45,7 +45,7 @@ public class LargeFieldTest extends SolrTestCaseJ4 { System.setProperty("documentCache.enabled", "true"); System.setProperty("enableLazyFieldLoading", "true"); - initCore("solrconfig-managed-schema.xml", "ignoredSchemaName?"); + initCore("solrconfig-managed-schema.xml", "ignoredSchemaName"); // TODO SOLR-10229 will make this easier boolean PERSIST_FALSE = false; // don't write to test resource dir From d60849f37782e0740e4e9964d70002a7d3078e22 Mon Sep 17 00:00:00 2001 From: Steve Rowe Date: Sat, 18 Mar 2017 15:09:43 -0400 Subject: [PATCH 038/563] LUCENE-7748: buildAndPushRelease.py should fail if the project DOAP files are missing releases that are less than the release being produced --- dev-tools/scripts/buildAndPushRelease.py | 64 ++++++++++++++++++++++++ 1 file changed, 64 insertions(+) diff --git a/dev-tools/scripts/buildAndPushRelease.py b/dev-tools/scripts/buildAndPushRelease.py index d7422148d7b..57f093b3728 100644 --- a/dev-tools/scripts/buildAndPushRelease.py +++ b/dev-tools/scripts/buildAndPushRelease.py @@ -22,6 +22,8 @@ import os import sys import subprocess import textwrap +import urllib.request, urllib.error, urllib.parse +import xml.etree.ElementTree as ET LOG = '/tmp/release.log' @@ -57,6 +59,14 @@ def runAndSendGPGPassword(command, password): print(msg) raise RuntimeError(msg) +def load(urlString): + try: + content = urllib.request.urlopen(urlString).read().decode('utf-8') + except Exception as e: + print('Retrying download of url %s after exception: %s' % (urlString, e)) + content = urllib.request.urlopen(urlString).read().decode('utf-8') + return content + def getGitRev(): status = os.popen('git status').read().strip() if 'nothing to commit, working directory clean' not in status and 'nothing to commit, working tree clean' not in status: @@ -84,6 +94,9 @@ def prepare(root, version, gpgKeyID, gpgPassword): print(' git rev: %s' % rev) log('\nGIT rev: %s\n' % rev) + print(' Check DOAP files') + checkDOAPfiles(version) + print(' ant clean test') run('ant clean test') @@ -119,6 +132,57 @@ def prepare(root, version, gpgKeyID, gpgPassword): print() return rev +reVersion1 = re.compile(r'\>(\d+)\.(\d+)\.(\d+)(-alpha|-beta)?/\<', re.IGNORECASE) +reVersion2 = re.compile(r'-(\d+)\.(\d+)\.(\d+)(-alpha|-beta)?\.zip<', re.IGNORECASE) +reDoapRevision = re.compile(r'(\d+)\.(\d+)(?:\.(\d+))?(-alpha|-beta)?', re.IGNORECASE) +def checkDOAPfiles(version): + # In Lucene and Solr DOAP files, verify presence of all releases less than the one being produced. + errorMessages = [] + for product in 'lucene', 'solr': + url = 'https://archive.apache.org/dist/lucene/%s' % ('java' if product == 'lucene' else product) + distpage = load(url) + releases = set() + for regex in reVersion1, reVersion2: + for tup in regex.findall(distpage): + if tup[0] in ('1', '2'): # Ignore 1.X and 2.X releases + continue + releases.add(normalizeVersion(tup)) + doapNS = '{http://usefulinc.com/ns/doap#}' + xpathRevision = '{0}Project/{0}release/{0}Version/{0}revision'.format(doapNS) + doapFile = "dev-tools/doap/%s.rdf" % product + treeRoot = ET.parse(doapFile).getroot() + doapRevisions = set() + for revision in treeRoot.findall(xpathRevision): + match = reDoapRevision.match(revision.text) + if (match is not None): + if (match.group(1) not in ('0', '1', '2')): # Ignore 0.X, 1.X and 2.X revisions + doapRevisions.add(normalizeVersion(match.groups())) + else: + errorMessages.append('ERROR: Failed to parse revision: %s in %s' % (revision.text, doapFile)) + missingDoapRevisions = set() + for release in releases: + if release not in doapRevisions and release < version: # Ignore releases greater than the one being produced + missingDoapRevisions.add(release) + if len(missingDoapRevisions) > 0: + errorMessages.append('ERROR: Missing revision(s) in %s: %s' % (doapFile, ', '.join(sorted(missingDoapRevisions)))) + if (len(errorMessages) > 0): + raise RuntimeError('\n%s\n(Hint: copy/paste from the stable branch version of the file(s).)' + % '\n'.join(errorMessages)) + +def normalizeVersion(tup): + suffix = '' + if tup[-1] is not None and tup[-1].lower() == '-alpha': + tup = tup[:(len(tup) - 1)] + suffix = '-ALPHA' + elif tup[-1] is not None and tup[-1].lower() == '-beta': + tup = tup[:(len(tup) - 1)] + suffix = '-BETA' + while tup[-1] in ('', None): + tup = tup[:(len(tup) - 1)] + while len(tup) < 3: + tup = tup + ('0',) + return '.'.join(tup) + suffix + def pushLocal(version, root, rev, rcNum, localDir): print('Push local [%s]...' % localDir) os.makedirs(localDir) From 4bc75dbf235145fad5ec1001004c663e15449523 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Mon, 20 Mar 2017 08:21:54 +0700 Subject: [PATCH 039/563] SOLR-9835: Fix OnlyLeaderIndexesTest failure, inplace updates is not copied over properly --- solr/core/src/java/org/apache/solr/update/UpdateLog.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) 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 bd0adbb4252..daa50a9621f 100644 --- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java +++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java @@ -1197,10 +1197,7 @@ public static final int VERSION_IDX = 1; 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); + AddUpdateCommand cmd = convertTlogEntryToAddUpdateCommand(req, entry, oper, version); cmd.setFlags(UpdateCommand.IGNORE_AUTOCOMMIT); add(cmd); break; From e09273943b41330ca224377e2e6b6ca54dce84eb Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Mon, 20 Mar 2017 09:18:54 +0700 Subject: [PATCH 040/563] SOLR-10079: Speedup TestInPlaceUpdatesDistrib in new replication mode --- .../test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java | 3 --- 1 file changed, 3 deletions(-) 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 4538e900e7e..cbd7b0260c5 100644 --- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java +++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java @@ -323,9 +323,6 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { } private void docValuesUpdateTest() throws Exception { - clearIndex(); - commit(); - // number of docs we're testing (0 <= id), index may contain additional random docs (id < 0) final int numDocs = atLeast(100); log.info("Trying num docs = " + numDocs); From 21adce4a2fa6d9a207cd8abed11ba050ff57c933 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Mon, 20 Mar 2017 15:21:36 +0700 Subject: [PATCH 041/563] SOLR-9992: Add support for grouping with PointFIelds --- .../handler/component/QueryComponent.java | 5 +++- .../TopGroupsShardRequestFactory.java | 4 ++-- .../SearchGroupsResultTransformer.java | 24 +++++++++++++++---- .../TopGroupsResultTransformer.java | 4 +++- .../apache/solr/TestDistributedGrouping.java | 2 -- .../org/apache/solr/TestGroupingSearch.java | 8 ------- .../cloud/SegmentTerminateEarlyTestState.java | 4 +--- .../apache/solr/cloud/TestSegmentSorting.java | 2 -- 8 files changed, 29 insertions(+), 24 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java index deff25bf232..08a0e842e08 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java @@ -49,6 +49,7 @@ import org.apache.lucene.search.grouping.GroupDocs; import org.apache.lucene.search.grouping.SearchGroup; import org.apache.lucene.search.grouping.TopGroups; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.InPlaceMergeSorter; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.common.SolrDocument; @@ -445,7 +446,9 @@ public class QueryComponent extends SearchComponent for (String topGroup : topGroupsParam) { SearchGroup searchGroup = new SearchGroup<>(); if (!topGroup.equals(TopGroupsShardRequestFactory.GROUP_NULL_VALUE)) { - searchGroup.groupValue = new BytesRef(schemaField.getType().readableToIndexed(topGroup)); + BytesRefBuilder builder = new BytesRefBuilder(); + schemaField.getType().readableToIndexed(topGroup, builder); + searchGroup.groupValue = builder.get(); } topGroups.add(searchGroup); } diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/requestfactory/TopGroupsShardRequestFactory.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/requestfactory/TopGroupsShardRequestFactory.java index ca73a30e164..5067c518ed5 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/requestfactory/TopGroupsShardRequestFactory.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/requestfactory/TopGroupsShardRequestFactory.java @@ -19,6 +19,7 @@ package org.apache.solr.search.grouping.distributed.requestfactory; import org.apache.lucene.analysis.reverse.ReverseStringFilter; import org.apache.lucene.search.grouping.SearchGroup; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CharsRefBuilder; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.GroupParams; import org.apache.solr.common.params.ModifiableSolrParams; @@ -117,9 +118,8 @@ public class TopGroupsShardRequestFactory implements ShardRequestFactory { for (SearchGroup searchGroup : entry.getValue()) { String groupValue; if (searchGroup.groupValue != null) { - String rawGroupValue = searchGroup.groupValue.utf8ToString(); FieldType fieldType = schema.getField(entry.getKey()).getType(); - groupValue = fieldType.indexedToReadable(rawGroupValue); + groupValue = fieldType.indexedToReadable(searchGroup.groupValue, new CharsRefBuilder()).toString(); } else { groupValue = GROUP_NULL_VALUE; } diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java index c5c1e811970..2602221a654 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/SearchGroupsResultTransformer.java @@ -19,6 +19,8 @@ package org.apache.solr.search.grouping.distributed.shardresultserializer; import org.apache.lucene.search.Sort; import org.apache.lucene.search.grouping.SearchGroup; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.CharsRefBuilder; import org.apache.solr.common.util.NamedList; import org.apache.solr.schema.SchemaField; import org.apache.solr.search.SolrIndexSearcher; @@ -56,7 +58,7 @@ public class SearchGroupsResultTransformer implements ShardResultTransformer> searchGroups = fieldCommandResult.getSearchGroups(); if (searchGroups != null) { - commandResult.add(TOP_GROUPS, serializeSearchGroup(searchGroups, fieldCommand.getGroupSort())); + commandResult.add(TOP_GROUPS, serializeSearchGroup(searchGroups, fieldCommand)); } final Integer groupedCount = fieldCommandResult.getGroupCount(); if (groupedCount != null) { @@ -85,7 +87,17 @@ public class SearchGroupsResultTransformer implements ShardResultTransformer> rawSearchGroup : rawSearchGroups){ SearchGroup searchGroup = new SearchGroup<>(); - searchGroup.groupValue = rawSearchGroup.getKey() != null ? new BytesRef(rawSearchGroup.getKey()) : null; + SchemaField groupField = rawSearchGroup.getKey() != null? searcher.getSchema().getFieldOrNull(command.getKey()) : null; + searchGroup.groupValue = null; + if (rawSearchGroup.getKey() != null) { + if (groupField != null) { + BytesRefBuilder builder = new BytesRefBuilder(); + groupField.getType().readableToIndexed(rawSearchGroup.getKey(), builder); + searchGroup.groupValue = builder.get(); + } else { + searchGroup.groupValue = new BytesRef(rawSearchGroup.getKey()); + } + } searchGroup.sortValues = rawSearchGroup.getValue().toArray(new Comparable[rawSearchGroup.getValue().size()]); for (int i = 0; i < searchGroup.sortValues.length; i++) { SchemaField field = groupSort.getSort()[i].getField() != null ? searcher.getSchema().getFieldOrNull(groupSort.getSort()[i].getField()) : null; @@ -101,17 +113,19 @@ public class SearchGroupsResultTransformer implements ShardResultTransformer> data, Sort groupSort) { + private NamedList serializeSearchGroup(Collection> data, SearchGroupsFieldCommand command) { final NamedList result = new NamedList<>(data.size()); for (SearchGroup searchGroup : data) { Object[] convertedSortValues = new Object[searchGroup.sortValues.length]; for (int i = 0; i < searchGroup.sortValues.length; i++) { Object sortValue = searchGroup.sortValues[i]; - SchemaField field = groupSort.getSort()[i].getField() != null ? searcher.getSchema().getFieldOrNull(groupSort.getSort()[i].getField()) : null; + SchemaField field = command.getGroupSort().getSort()[i].getField() != null ? + searcher.getSchema().getFieldOrNull(command.getGroupSort().getSort()[i].getField()) : null; convertedSortValues[i] = ShardResultTransformerUtils.marshalSortValue(sortValue, field); } - String groupValue = searchGroup.groupValue != null ? searchGroup.groupValue.utf8ToString() : null; + SchemaField field = searcher.getSchema().getFieldOrNull(command.getKey()); + String groupValue = searchGroup.groupValue != null ? field.getType().indexedToReadable(searchGroup.groupValue, new CharsRefBuilder()).toString() : null; result.add(groupValue, convertedSortValues); } diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java index 3ff87bd83ed..415e513d8a8 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java @@ -33,6 +33,7 @@ import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.search.grouping.GroupDocs; import org.apache.lucene.search.grouping.TopGroups; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CharsRefBuilder; import org.apache.solr.common.util.NamedList; import org.apache.solr.handler.component.ResponseBuilder; import org.apache.solr.handler.component.ShardDoc; @@ -232,7 +233,8 @@ public class TopGroupsResultTransformer implements ShardResultTransformer Date: Mon, 20 Mar 2017 17:30:40 +0700 Subject: [PATCH 042/563] SOLR-9992: Update changes.txt --- solr/CHANGES.txt | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index b7e55cc9a4f..e628694cca6 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -101,8 +101,13 @@ Apache UIMA 2.3.1 Apache ZooKeeper 3.4.6 Jetty 9.3.14.v20161028 +Detailed Change List +---------------------- -(No Changes) +New Features +---------------------- + +* SOLR-9992: Add support for grouping with PointFIelds. (Cao Manh Dat) ================== 6.5.0 ================== From a3e4f57e1b520787fc332bf7471d68331af65e25 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Mon, 20 Mar 2017 19:03:55 +0100 Subject: [PATCH 043/563] SOLR-10319 SolrCore "instanceDir" metric not visible in JMX. --- solr/CHANGES.txt | 2 ++ solr/core/src/java/org/apache/solr/core/SolrCore.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index e628694cca6..b4196edd847 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -313,6 +313,8 @@ Bug Fixes * SOLR-10218: The Schema API commands "add-field-type" and "replace-field-type" improperly specify SimilarityFactory params. (Benjamin Deininger, Troy Mohl, Steve Rowe) +* SOLR-10319: SolrCore "instanceDir" metric not visible in JMX. (ab) + Optimizations ---------------------- 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 6e25280c653..9d77c7e2651 100644 --- a/solr/core/src/java/org/apache/solr/core/SolrCore.java +++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java @@ -1134,7 +1134,7 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea 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, () -> resourceLoader.getInstancePath().toString(), 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()); From ffaa2345b40cff5287fcf8edaf9059cf3a8d1892 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Mon, 20 Mar 2017 19:00:33 +0000 Subject: [PATCH 044/563] SOLR-10046: move from 6.5.0 to 6.6.0 CHANGES.txt (backport yet to be completed) --- solr/CHANGES.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index b4196edd847..7767453c9d9 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -109,6 +109,8 @@ New Features * SOLR-9992: Add support for grouping with PointFIelds. (Cao Manh Dat) +* SOLR-10046: Add UninvertDocValuesMergePolicyFactory class. (Keith Laban, Christine Poerschke) + ================== 6.5.0 ================== @@ -213,8 +215,6 @@ New Features * SOLR-10224: Add disk total and disk free metrics. (ab) -* SOLR-10046: Add UninvertDocValuesMergePolicyFactory class. (Keith Laban, Christine Poerschke) - * SOLR-10085: SQL result set fields should be ordered by the field list (Joel Bernstein) * SOLR-10254: significantTerms Streaming Expression should work in non-SolrCloud mode (Joel Bernstein) From 92297b58605104106b5b31d3dae5c2daed1886ba Mon Sep 17 00:00:00 2001 From: Dennis Gove Date: Mon, 20 Mar 2017 16:36:05 -0400 Subject: [PATCH 045/563] SOLR-10292: Adds CartesianProductStream to turn multivalued fields into multiple tuples --- solr/CHANGES.txt | 3 + .../apache/solr/handler/StreamHandler.java | 5 + .../client/solrj/io/comp/FieldComparator.java | 5 + .../io/comp/MultipleFieldComparator.java | 22 ++ .../solrj/io/comp/StreamComparator.java | 1 + .../client/solrj/io/eval/FieldEvaluator.java | 28 +- .../io/stream/CartesianProductStream.java | 301 ++++++++++++++++++ .../solr/configsets/streaming/conf/schema.xml | 1 + .../solrj/io/stream/StreamExpressionTest.java | 132 ++++++++ .../io/stream/eval/FieldEvaluatorTest.java | 114 +++++++ 10 files changed, 611 insertions(+), 1 deletion(-) create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CartesianProductStream.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/FieldEvaluatorTest.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 7767453c9d9..f61c4c2b125 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -58,6 +58,9 @@ New Features * SOLR-9835: Create another replication mode for SolrCloud +* SOLR-10292: Adds CartesianProductStream which turns a single tuple with a multi-valued field into N + tuples, one for each value in the multi-valued field. (Dennis Gove) + Bug Fixes ---------------------- * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509. diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java index e69f52be17b..dfae5cd69e3 100644 --- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java @@ -154,6 +154,9 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, .withFunctionName("gatherNodes", GatherNodesStream.class) .withFunctionName("nodes", GatherNodesStream.class) .withFunctionName("select", SelectStream.class) + .withFunctionName("shortestPath", ShortestPathStream.class) + .withFunctionName("gatherNodes", GatherNodesStream.class) + .withFunctionName("nodes", GatherNodesStream.class) .withFunctionName("scoreNodes", ScoreNodesStream.class) .withFunctionName("model", ModelStream.class) .withFunctionName("classify", ClassifyStream.class) @@ -162,6 +165,8 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, .withFunctionName("null", NullStream.class) .withFunctionName("priority", PriorityStream.class) .withFunctionName("significantTerms", SignificantTermsStream.class) + .withFunctionName("cartesianProduct", CartesianProductStream.class) + // metrics .withFunctionName("min", MinMetric.class) .withFunctionName("max", MaxMetric.class) diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/FieldComparator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/FieldComparator.java index 15af57aef93..f2be53e8dfa 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/FieldComparator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/FieldComparator.java @@ -166,4 +166,9 @@ public class FieldComparator implements StreamComparator { order ); } + + @Override + public StreamComparator append(StreamComparator other){ + return new MultipleFieldComparator(this).append(other); + } } \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/MultipleFieldComparator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/MultipleFieldComparator.java index c709f4d0d47..09532e19f39 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/MultipleFieldComparator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/MultipleFieldComparator.java @@ -17,6 +17,8 @@ package org.apache.solr.client.solrj.io.comp; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.UUID; @@ -113,4 +115,24 @@ public class MultipleFieldComparator implements StreamComparator { return new MultipleFieldComparator(aliasedComps); } + + @Override + public StreamComparator append(StreamComparator other){ + List newComps = new ArrayList<>(); + + for(StreamComparator comp : comps){ + newComps.add(comp); + } + + if(other instanceof FieldComparator){ + newComps.add(other); + } + else if(other instanceof MultipleFieldComparator){ + for(StreamComparator comp : ((MultipleFieldComparator)other).comps){ + newComps.add(comp); + } + } + + return new MultipleFieldComparator(newComps.toArray(new StreamComparator[newComps.size()])); + } } \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/StreamComparator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/StreamComparator.java index 156a19cd1a3..70bd51d801b 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/StreamComparator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/comp/StreamComparator.java @@ -27,4 +27,5 @@ import org.apache.solr.client.solrj.io.stream.expr.Expressible; public interface StreamComparator extends Comparator, Expressible, Serializable { public boolean isDerivedFrom(StreamComparator base); public StreamComparator copyAliased(Map aliases); + public StreamComparator append(StreamComparator other); } \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FieldEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FieldEvaluator.java index 0ebe72913c1..32514982a84 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FieldEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FieldEvaluator.java @@ -20,6 +20,8 @@ package org.apache.solr.client.solrj.io.eval; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.stream.expr.Explanation; @@ -43,7 +45,31 @@ public class FieldEvaluator extends SimpleEvaluator { @Override public Object evaluate(Tuple tuple) { - return tuple.get(fieldName); // returns null if field doesn't exist in tuple + Object value = tuple.get(fieldName); + + // if we have an array then convert to an ArrayList + // if we have an iterable that is not a list then convert to ArrayList + // lists are good to go + if(null != value){ + if(value instanceof Object[]){ + Object[] array = (Object[])value; + List list = new ArrayList(array.length); + for(Object obj : array){ + list.add(obj); + } + return list; + } + else if(value instanceof Iterable && !(value instanceof List)){ + Iterable iter = (Iterable)value; + List list = new ArrayList(); + for(Object obj : iter){ + list.add(obj); + } + return list; + } + } + + return value; } @Override diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CartesianProductStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CartesianProductStream.java new file mode 100644 index 00000000000..feb10c797a8 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CartesianProductStream.java @@ -0,0 +1,301 @@ +/* + * 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.client.solrj.io.stream; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.comp.FieldComparator; +import org.apache.solr.client.solrj.io.comp.StreamComparator; +import org.apache.solr.client.solrj.io.eval.FieldEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.Explanation; +import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType; +import org.apache.solr.client.solrj.io.stream.expr.Expressible; +import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class CartesianProductStream extends TupleStream implements Expressible { + + private static final long serialVersionUID = 1; + + private TupleStream stream; + private List evaluators; + private StreamComparator orderBy; + + // Used to contain the sorted queue of generated tuples + private LinkedList generatedTuples; + + public CartesianProductStream(StreamExpression expression,StreamFactory factory) throws IOException { + String functionName = factory.getFunctionName(getClass()); + + // grab all parameters out + List streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class); + List evaluateAsExpressions = factory.getOperandsOfType(expression, StreamExpressionValue.class); + StreamExpressionNamedParameter orderByExpression = factory.getNamedOperand(expression, "productSort"); + + // validate expression contains only what we want. + if(expression.getParameters().size() != streamExpressions.size() + evaluateAsExpressions.size() + (null == orderByExpression ? 0 : 1)){ + throw new IOException(String.format(Locale.ROOT,"Invalid %s expression %s - unknown operands found", functionName, expression)); + } + + if(1 != streamExpressions.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid %s expression %s - expecting single stream but found %d (must be TupleStream types)", functionName, expression, streamExpressions.size())); + } + + stream = factory.constructStream(streamExpressions.get(0)); + orderBy = null == orderByExpression ? null : factory.constructComparator(((StreamExpressionValue)orderByExpression.getParameter()).getValue(), FieldComparator.class); + + evaluators = new ArrayList<>(); + for(StreamExpressionParameter evaluateAsExpression : evaluateAsExpressions){ + String fullString = ((StreamExpressionValue)evaluateAsExpression).getValue().trim(); + String originalFullString = fullString; // used for error messages + + // remove possible wrapping quotes + if(fullString.length() > 2 && fullString.startsWith("\"") && fullString.endsWith("\"")){ + fullString = fullString.substring(1, fullString.length() - 1).trim(); + } + + String evaluatorPart = null; + String asNamePart = null; + + if(fullString.toLowerCase(Locale.ROOT).contains(" as ")){ + String[] parts = fullString.split("(?i) as "); // ensure we are splitting in a case-insensitive way + if(2 != parts.length){ + throw new IOException(String.format(Locale.ROOT,"Invalid %s expression %s - expecting evaluator of form 'fieldA' or 'fieldA as alias' but found %s", functionName, expression, originalFullString)); + } + + evaluatorPart = parts[0].trim(); + asNamePart = parts[1].trim(); + } + else{ + evaluatorPart = fullString; + // no rename + } + + boolean wasHandledAsEvaluatorFunction = false; + StreamEvaluator evaluator = null; + if(evaluatorPart.contains("(")){ + // is a possible evaluator + try{ + StreamExpression asValueExpression = StreamExpressionParser.parse(evaluatorPart); + if(factory.doesRepresentTypes(asValueExpression, StreamEvaluator.class)){ + evaluator = factory.constructEvaluator(asValueExpression); + wasHandledAsEvaluatorFunction = true; + } + } + catch(Throwable e){ + // it was not handled, so treat as a non-evaluator + } + } + if(!wasHandledAsEvaluatorFunction){ + // treat as a straight field evaluator + evaluator = new FieldEvaluator(evaluatorPart); + if(null == asNamePart){ + asNamePart = evaluatorPart; // just use the field name + } + } + + if(null == evaluator || null == asNamePart){ + throw new IOException(String.format(Locale.ROOT,"Invalid %s expression %s - failed to parse evaluator '%s'", functionName, expression, originalFullString)); + } + + evaluators.add(new NamedEvaluator(asNamePart, evaluator)); + } + } + + @Override + public StreamExpression toExpression(StreamFactory factory) throws IOException{ + return toExpression(factory, true); + } + + private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException { + // function name + StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass())); + + if(includeStreams){ + // we know stream is expressible + expression.addParameter(((Expressible)stream).toExpression(factory)); + } + else{ + expression.addParameter(""); + } + + // selected evaluators + for(NamedEvaluator evaluator : evaluators) { + expression.addParameter(String.format(Locale.ROOT, "%s as %s", evaluator.getEvaluator().toExpression(factory), evaluator.getName())); + } + + expression.addParameter(new StreamExpressionNamedParameter("productSort", orderBy.toExpression(factory))); + + return expression; + } + + @Override + public Explanation toExplanation(StreamFactory factory) throws IOException { + + Explanation explanation = new StreamExplanation(getStreamNodeId().toString()) + .withChildren(new Explanation[]{ + stream.toExplanation(factory) + }) + .withFunctionName(factory.getFunctionName(this.getClass())) + .withImplementingClass(this.getClass().getName()) + .withExpressionType(ExpressionType.STREAM_DECORATOR) + .withExpression(toExpression(factory, false).toString()); + + for(NamedEvaluator evaluator : evaluators){ + explanation.addHelper(evaluator.getEvaluator().toExplanation(factory)); + } + + explanation.addHelper(orderBy.toExplanation(factory)); + + return explanation; + } + + public Tuple read() throws IOException { + if(generatedTuples.isEmpty()){ + Tuple tuple = stream.read(); + + if(tuple.EOF){ + return tuple; + } + + // returns tuples in desired sorted order + generatedTuples = generateTupleList(tuple); + } + + return generatedTuples.pop(); + } + + private LinkedList generateTupleList(Tuple original) throws IOException{ + Map evaluatedValues = new HashMap<>(); + + for(NamedEvaluator evaluator : evaluators){ + evaluatedValues.put(evaluator.getName(), evaluator.getEvaluator().evaluate(original)); + } + + // use an array list internally because it has better sort performance + // in Java 8. We do pay a conversion to a linked list but ..... oh well + ArrayList generatedTupleList = new ArrayList<>(); + + int[] workingIndexes = new int[evaluators.size()]; // java language spec ensures all values are 0 + do{ + Tuple generated = original.clone(); + for(int offset = 0; offset < workingIndexes.length; ++offset){ + String fieldName = evaluators.get(offset).getName(); + Object evaluatedValue = evaluatedValues.get(fieldName); + if(evaluatedValue instanceof Collection){ + // because of the way a FieldEvaluator works we know that + // any collection is a list. + generated.put(fieldName, ((List)evaluatedValue).get(workingIndexes[offset])); + } + } + generatedTupleList.add(generated); + }while(iterate(evaluators, workingIndexes, evaluatedValues)); + + // order if we need to + if(null != orderBy){ + generatedTupleList.sort(orderBy); + } + + return new LinkedList<>(generatedTupleList); + } + + private boolean iterate(List evaluators, int[] indexes, Map evaluatedValues){ + // this assumes evaluators and indexes are the same length, which is ok cause we created it so we know it is + // go right to left and increment, returning true if we're not at the end + for(int offset = indexes.length - 1; offset >= 0; --offset){ + Object evaluatedValue = evaluatedValues.get(evaluators.get(offset).getName()); + if(evaluatedValue instanceof Collection){ + int currentIndexValue = indexes[offset]; + if(currentIndexValue < ((Collection)evaluatedValue).size() - 1){ + indexes[offset] = currentIndexValue + 1; + return true; + } + else if(0 != offset){ + indexes[offset] = 0; + // move to the left + } + } + } + + // no more + return false; + } + + /** Return the incoming sort + the sort applied to the generated tuples */ + public StreamComparator getStreamSort(){ + if(null != orderBy){ + return stream.getStreamSort().append(orderBy); + } + return stream.getStreamSort(); + } + + public void setStreamContext(StreamContext context) { + this.stream.setStreamContext(context); + } + + public List children() { + List l = new ArrayList<>(); + l.add(stream); + return l; + } + + public void open() throws IOException { + stream.open(); + generatedTuples = new LinkedList<>(); + } + + public void close() throws IOException { + stream.close(); + generatedTuples.clear(); + } + + public int getCost() { + return 0; + } + + class NamedEvaluator{ + private String name; + private StreamEvaluator evaluator; + + public NamedEvaluator(String name, StreamEvaluator evaluator){ + this.name = name; + this.evaluator = evaluator; + } + + public String getName(){ + return name; + } + + public StreamEvaluator getEvaluator(){ + return evaluator; + } + } +} \ No newline at end of file diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml index 7d3173addcd..b61a2e93ee8 100644 --- a/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml +++ b/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml @@ -526,6 +526,7 @@ + diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java index c61e443576b..6c96025e640 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java @@ -4959,6 +4959,138 @@ public class StreamExpressionTest extends SolrCloudTestCase { } + @Test + public void testCartesianProductStream() throws Exception { + + new UpdateRequest() + .add(id, "0", "a_ss", "a", "a_ss", "b", "a_ss", "c", "a_ss", "d", "a_ss", "e", "b_ls", "1", "b_ls", "2", "b_ls", "3") + .add(id, "1", "a_ss", "a", "a_ss", "b", "a_ss", "c", "a_ss", "d", "a_ss", "e") + .commit(cluster.getSolrClient(), COLLECTIONORALIAS); + + StreamExpression expression; + TupleStream stream; + List tuples; + + StreamFactory factory = new StreamFactory() + .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress()) + .withFunctionName("search", CloudSolrStream.class) + .withFunctionName("cartesian", CartesianProductStream.class); + + // single selection, no sort + stream = factory.constructStream("cartesian(" + + "search(collection1, q=*:*, fl=\"id,a_ss\", sort=\"id asc\")," + + "a_ss" + + ")"); + tuples = getTuples(stream); + + assertEquals(10, tuples.size()); + assertOrder(tuples, 0,0,0,0,0,1,1,1,1,1); + assertEquals("a", tuples.get(0).get("a_ss")); + assertEquals("c", tuples.get(2).get("a_ss")); + assertEquals("a", tuples.get(5).get("a_ss")); + assertEquals("c", tuples.get(7).get("a_ss")); + + // single selection, sort + stream = factory.constructStream("cartesian(" + + "search(collection1, q=*:*, fl=\"id,a_ss\", sort=\"id asc\")," + + "a_ss," + + "productSort=\"a_ss DESC\"" + + ")"); + tuples = getTuples(stream); + + assertEquals(10, tuples.size()); + assertOrder(tuples, 0,0,0,0,0,1,1,1,1,1); + assertEquals("e", tuples.get(0).get("a_ss")); + assertEquals("c", tuples.get(2).get("a_ss")); + assertEquals("e", tuples.get(5).get("a_ss")); + assertEquals("c", tuples.get(7).get("a_ss")); + + // multi selection, sort + stream = factory.constructStream("cartesian(" + + "search(collection1, q=*:*, fl=\"id,a_ss,b_ls\", sort=\"id asc\")," + + "a_ss," + + "b_ls," + + "productSort=\"a_ss ASC\"" + + ")"); + tuples = getTuples(stream); + + assertEquals(20, tuples.size()); // (5 * 3) + 5 + assertOrder(tuples, 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,1,1,1,1); + assertEquals("a", tuples.get(0).get("a_ss")); + assertEquals(1L, tuples.get(0).get("b_ls")); + assertEquals("a", tuples.get(1).get("a_ss")); + assertEquals(2L, tuples.get(1).get("b_ls")); + assertEquals("a", tuples.get(2).get("a_ss")); + assertEquals(3L, tuples.get(2).get("b_ls")); + + assertEquals("b", tuples.get(3).get("a_ss")); + assertEquals(1L, tuples.get(3).get("b_ls")); + assertEquals("b", tuples.get(4).get("a_ss")); + assertEquals(2L, tuples.get(4).get("b_ls")); + assertEquals("b", tuples.get(5).get("a_ss")); + assertEquals(3L, tuples.get(5).get("b_ls")); + + // multi selection, sort + stream = factory.constructStream("cartesian(" + + "search(collection1, q=*:*, fl=\"id,a_ss,b_ls\", sort=\"id asc\")," + + "a_ss," + + "b_ls," + + "productSort=\"a_ss ASC, b_ls DESC\"" + + ")"); + tuples = getTuples(stream); + + assertEquals(20, tuples.size()); // (5 * 3) + 5 + assertOrder(tuples, 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,1,1,1,1); + assertEquals("a", tuples.get(0).get("a_ss")); + assertEquals(3L, tuples.get(0).get("b_ls")); + assertEquals("a", tuples.get(1).get("a_ss")); + assertEquals(2L, tuples.get(1).get("b_ls")); + assertEquals("a", tuples.get(2).get("a_ss")); + assertEquals(1L, tuples.get(2).get("b_ls")); + + assertEquals("b", tuples.get(3).get("a_ss")); + assertEquals(3L, tuples.get(3).get("b_ls")); + assertEquals("b", tuples.get(4).get("a_ss")); + assertEquals(2L, tuples.get(4).get("b_ls")); + assertEquals("b", tuples.get(5).get("a_ss")); + assertEquals(1L, tuples.get(5).get("b_ls")); + + // multi selection, sort + stream = factory.constructStream("cartesian(" + + "search(collection1, q=*:*, fl=\"id,a_ss,b_ls\", sort=\"id asc\")," + + "a_ss," + + "b_ls," + + "productSort=\"b_ls DESC\"" + + ")"); + tuples = getTuples(stream); + + assertEquals(20, tuples.size()); // (5 * 3) + 5 + assertOrder(tuples, 0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,1,1,1,1); + assertEquals("a", tuples.get(0).get("a_ss")); + assertEquals(3L, tuples.get(0).get("b_ls")); + assertEquals("b", tuples.get(1).get("a_ss")); + assertEquals(3L, tuples.get(1).get("b_ls")); + assertEquals("c", tuples.get(2).get("a_ss")); + assertEquals(3L, tuples.get(2).get("b_ls")); + assertEquals("d", tuples.get(3).get("a_ss")); + assertEquals(3L, tuples.get(3).get("b_ls")); + assertEquals("e", tuples.get(4).get("a_ss")); + assertEquals(3L, tuples.get(4).get("b_ls")); + + assertEquals("a", tuples.get(5).get("a_ss")); + assertEquals(2L, tuples.get(5).get("b_ls")); + assertEquals("b", tuples.get(6).get("a_ss")); + assertEquals(2L, tuples.get(6).get("b_ls")); + assertEquals("c", tuples.get(7).get("a_ss")); + assertEquals(2L, tuples.get(7).get("b_ls")); + assertEquals("d", tuples.get(8).get("a_ss")); + assertEquals(2L, tuples.get(8).get("b_ls")); + assertEquals("e", tuples.get(9).get("a_ss")); + assertEquals(2L, tuples.get(9).get("b_ls")); + + } + + @Test public void testParallelComplementStream() throws Exception { diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/FieldEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/FieldEvaluatorTest.java new file mode 100644 index 00000000000..b1c67c7a319 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/FieldEvaluatorTest.java @@ -0,0 +1,114 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.PriorityQueue; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.FieldEvaluator; +import org.junit.Test; + +import junit.framework.Assert; + +public class FieldEvaluatorTest extends LuceneTestCase { + + Map values; + + public FieldEvaluatorTest() { + super(); + + values = new HashMap(); + } + + @SuppressWarnings("serial") + @Test + public void listTypes() throws Exception{ + values.clear(); + values.put("a", new ArrayList(){{ add(true); add(false); }}); + values.put("b", new ArrayList(){{ add(0.0); add(1.1); }}); + values.put("c", new ArrayList(){{ add(0); add(1); }}); + values.put("d", new ArrayList(){{ add(0L); add(1L); }}); + values.put("e", new ArrayList(){{ add("first"); add("second"); }}); + + Tuple tuple = new Tuple(values); + + for(String fieldName : new String[]{ "a", "b", "c", "d", "e" }){ + Assert.assertTrue(new FieldEvaluator(fieldName).evaluate(tuple) instanceof Collection); + Assert.assertEquals(2, ((Collection)new FieldEvaluator(fieldName).evaluate(tuple)).size()); + } + + Assert.assertEquals(false, ((Collection)new FieldEvaluator("a").evaluate(tuple)).toArray()[1]); + Assert.assertEquals(1.1, ((Collection)new FieldEvaluator("b").evaluate(tuple)).toArray()[1]); + Assert.assertEquals(1, ((Collection)new FieldEvaluator("c").evaluate(tuple)).toArray()[1]); + Assert.assertEquals(1L, ((Collection)new FieldEvaluator("d").evaluate(tuple)).toArray()[1]); + Assert.assertEquals("second", ((Collection)new FieldEvaluator("e").evaluate(tuple)).toArray()[1]); + } + + @Test + public void arrayTypes() throws Exception{ + values.clear(); + values.put("a", new Boolean[]{ true, false }); + values.put("b", new Double[]{ 0.0, 1.1 }); + values.put("c", new Integer[]{ 0, 1 }); + values.put("d", new Long[]{ 0L, 1L }); + values.put("e", new String[]{ "first", "second" }); + + Tuple tuple = new Tuple(values); + + for(String fieldName : new String[]{ "a", "b", "c", "d", "e" }){ + Assert.assertTrue(new FieldEvaluator(fieldName).evaluate(tuple) instanceof Collection); + Assert.assertEquals(2, ((Collection)new FieldEvaluator(fieldName).evaluate(tuple)).size()); + } + + Assert.assertEquals(false, ((Collection)new FieldEvaluator("a").evaluate(tuple)).toArray()[1]); + Assert.assertEquals(1.1, ((Collection)new FieldEvaluator("b").evaluate(tuple)).toArray()[1]); + Assert.assertEquals(1, ((Collection)new FieldEvaluator("c").evaluate(tuple)).toArray()[1]); + Assert.assertEquals(1L, ((Collection)new FieldEvaluator("d").evaluate(tuple)).toArray()[1]); + Assert.assertEquals("second", ((Collection)new FieldEvaluator("e").evaluate(tuple)).toArray()[1]); + } + + @SuppressWarnings("serial") + @Test + public void iterableTypes() throws Exception{ + values.clear(); + + values.put("a", new PriorityQueue(){{ add(true); add(false); }}); + values.put("b", new PriorityQueue(){{ add(0.0); add(1.1); }}); + values.put("c", new PriorityQueue(){{ add(0); add(1); }}); + values.put("d", new PriorityQueue(){{ add(0L); add(1L); }}); + values.put("e", new PriorityQueue(){{ add("first"); add("second"); }}); + + Tuple tuple = new Tuple(values); + + for(String fieldName : new String[]{ "a", "b", "c", "d", "e" }){ + Assert.assertTrue(new FieldEvaluator(fieldName).evaluate(tuple) instanceof Collection); + Assert.assertEquals(2, ((Collection)new FieldEvaluator(fieldName).evaluate(tuple)).size()); + } + + // the priority queue is doing natural ordering, so false is first + Assert.assertEquals(true, ((Collection)new FieldEvaluator("a").evaluate(tuple)).toArray()[1]); + Assert.assertEquals(1.1, ((Collection)new FieldEvaluator("b").evaluate(tuple)).toArray()[1]); + Assert.assertEquals(1, ((Collection)new FieldEvaluator("c").evaluate(tuple)).toArray()[1]); + Assert.assertEquals(1L, ((Collection)new FieldEvaluator("d").evaluate(tuple)).toArray()[1]); + Assert.assertEquals("second", ((Collection)new FieldEvaluator("e").evaluate(tuple)).toArray()[1]); + } +} From 4171ef79b404829f5bb7c03de3855313d3ea9c86 Mon Sep 17 00:00:00 2001 From: Dennis Gove Date: Tue, 21 Mar 2017 08:40:40 -0400 Subject: [PATCH 046/563] SOLR-10333: Fixes use of HashedMap in StreamEvaluator tests --- .../io/stream/eval/AbsoluteValueEvaluatorTest.java | 8 ++++---- .../client/solrj/io/stream/eval/AddEvaluatorTest.java | 8 ++++---- .../client/solrj/io/stream/eval/AndEvaluatorTest.java | 8 ++++---- .../solrj/io/stream/eval/CompoundEvaluatorTest.java | 8 ++++---- .../solrj/io/stream/eval/DivideEvaluatorTest.java | 10 +++++----- .../solrj/io/stream/eval/EqualsEvaluatorTest.java | 8 ++++---- .../solrj/io/stream/eval/ExclusiveOrEvaluatorTest.java | 8 ++++---- .../stream/eval/GreaterThanEqualToEvaluatorTest.java | 8 ++++---- .../solrj/io/stream/eval/GreaterThanEvaluatorTest.java | 8 ++++---- .../io/stream/eval/LessThanEqualToEvaluatorTest.java | 8 ++++---- .../solrj/io/stream/eval/LessThanEvaluatorTest.java | 8 ++++---- .../solrj/io/stream/eval/MultiplyEvaluatorTest.java | 8 ++++---- .../solrj/io/stream/eval/NaturalLogEvaluatorTest.java | 8 ++++---- .../client/solrj/io/stream/eval/NotEvaluatorTest.java | 10 +++++----- .../client/solrj/io/stream/eval/OrEvaluatorTest.java | 8 ++++---- .../solrj/io/stream/eval/RawValueEvaluatorTest.java | 8 ++++---- .../solrj/io/stream/eval/SubtractEvaluatorTest.java | 8 ++++---- 17 files changed, 70 insertions(+), 70 deletions(-) diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AbsoluteValueEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AbsoluteValueEvaluatorTest.java index 88d34470a85..ff2384c6d92 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AbsoluteValueEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AbsoluteValueEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.AbsoluteValueEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class AbsoluteValueEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -39,7 +39,7 @@ public class AbsoluteValueEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("abs", AbsoluteValueEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AddEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AddEvaluatorTest.java index 7115452f2de..ac31accd140 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AddEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AddEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.AddEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class AddEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -39,7 +39,7 @@ public class AddEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("add", AddEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AndEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AndEvaluatorTest.java index 9daa9283a13..4e94758cd6e 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AndEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/AndEvaluatorTest.java @@ -16,11 +16,9 @@ */ package org.apache.solr.client.solrj.io.stream.eval; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.AndEvaluator; @@ -28,6 +26,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class AndEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -38,7 +38,7 @@ public class AndEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("and", AndEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CompoundEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CompoundEvaluatorTest.java index 8ae5657f546..74b44b35293 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CompoundEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CompoundEvaluatorTest.java @@ -16,11 +16,9 @@ */ package org.apache.solr.client.solrj.io.stream.eval; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.AddEvaluator; @@ -34,6 +32,8 @@ import org.apache.solr.client.solrj.io.eval.SubtractEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class CompoundEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -50,7 +50,7 @@ public class CompoundEvaluatorTest extends LuceneTestCase { .withFunctionName("sub", SubtractEvaluator.class) .withFunctionName("mult", MultiplyEvaluator.class) .withFunctionName("if", IfThenElseEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DivideEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DivideEvaluatorTest.java index 680be63beb4..b33c89626bf 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DivideEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/DivideEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.DivideEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class DivideEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -39,7 +39,7 @@ public class DivideEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("div", DivideEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test @@ -122,7 +122,7 @@ public class DivideEvaluatorTest extends LuceneTestCase { @Test(expected = IOException.class) public void divManyFieldsWithValues() throws Exception{ - StreamEvaluator evaluator = factory.constructEvaluator("div(a,b,c,d)"); + factory.constructEvaluator("div(a,b,c,d)"); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/EqualsEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/EqualsEvaluatorTest.java index 2f9dd9c95db..690e2fdfecc 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/EqualsEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/EqualsEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.EqualsEvaluator; @@ -30,6 +28,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class EqualsEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -42,7 +42,7 @@ public class EqualsEvaluatorTest extends LuceneTestCase { .withFunctionName("eq", EqualsEvaluator.class) .withFunctionName("val", RawValueEvaluator.class) ; - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ExclusiveOrEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ExclusiveOrEvaluatorTest.java index c1cc6770515..659bfdd85cb 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ExclusiveOrEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ExclusiveOrEvaluatorTest.java @@ -16,11 +16,9 @@ */ package org.apache.solr.client.solrj.io.stream.eval; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.ExclusiveOrEvaluator; @@ -28,6 +26,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class ExclusiveOrEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -38,7 +38,7 @@ public class ExclusiveOrEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("eor", ExclusiveOrEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/GreaterThanEqualToEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/GreaterThanEqualToEvaluatorTest.java index 5968a154732..26e6ac50d82 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/GreaterThanEqualToEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/GreaterThanEqualToEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.GreaterThanEqualToEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class GreaterThanEqualToEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -39,7 +39,7 @@ public class GreaterThanEqualToEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("gte", GreaterThanEqualToEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/GreaterThanEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/GreaterThanEvaluatorTest.java index d31a79c33bf..41a97636a82 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/GreaterThanEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/GreaterThanEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.GreaterThanEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class GreaterThanEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -39,7 +39,7 @@ public class GreaterThanEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("gt", GreaterThanEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/LessThanEqualToEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/LessThanEqualToEvaluatorTest.java index 114ea2d09e0..e62a3e9a3ce 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/LessThanEqualToEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/LessThanEqualToEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.LessThanEqualToEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class LessThanEqualToEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -39,7 +39,7 @@ public class LessThanEqualToEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("lte", LessThanEqualToEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/LessThanEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/LessThanEvaluatorTest.java index 5cc0274a329..da8c46a3772 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/LessThanEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/LessThanEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.LessThanEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class LessThanEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -39,7 +39,7 @@ public class LessThanEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("lt", LessThanEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/MultiplyEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/MultiplyEvaluatorTest.java index a2a6616ddb2..ef473fcee92 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/MultiplyEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/MultiplyEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.MultiplyEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class MultiplyEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -39,7 +39,7 @@ public class MultiplyEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("mult", MultiplyEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NaturalLogEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NaturalLogEvaluatorTest.java index c4ae127f4ee..97867fedbe1 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NaturalLogEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NaturalLogEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.AddEvaluator; @@ -30,6 +28,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class NaturalLogEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -40,7 +40,7 @@ public class NaturalLogEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("log", NaturalLogEvaluator.class).withFunctionName("add", AddEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NotEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NotEvaluatorTest.java index 6116163916d..56cb4fd5d43 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NotEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/NotEvaluatorTest.java @@ -16,18 +16,18 @@ */ package org.apache.solr.client.solrj.io.stream.eval; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; -import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.eval.NotEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class NotEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -38,7 +38,7 @@ public class NotEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("not", NotEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/OrEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/OrEvaluatorTest.java index 00c6b7a5c5e..09206af77fc 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/OrEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/OrEvaluatorTest.java @@ -16,11 +16,9 @@ */ package org.apache.solr.client.solrj.io.stream.eval; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.OrEvaluator; @@ -28,6 +26,8 @@ import org.apache.solr.client.solrj.io.eval.StreamEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class OrEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -38,7 +38,7 @@ public class OrEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("or", OrEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/RawValueEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/RawValueEvaluatorTest.java index 0d637e14b24..1419ec2b905 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/RawValueEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/RawValueEvaluatorTest.java @@ -16,11 +16,9 @@ */ package org.apache.solr.client.solrj.io.stream.eval; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.AddEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.RawValueEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class RawValueEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -42,7 +42,7 @@ public class RawValueEvaluatorTest extends LuceneTestCase { .withFunctionName("add", AddEvaluator.class) .withFunctionName("and", AndEvaluator.class) ; - values = new HashedMap(); + values = new HashMap(); } @Test diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SubtractEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SubtractEvaluatorTest.java index 58cef8dabf4..3da169e3172 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SubtractEvaluatorTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SubtractEvaluatorTest.java @@ -17,11 +17,9 @@ package org.apache.solr.client.solrj.io.stream.eval; import java.io.IOException; +import java.util.HashMap; import java.util.Map; -import junit.framework.Assert; - -import org.apache.commons.collections.map.HashedMap; import org.apache.lucene.util.LuceneTestCase; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.eval.StreamEvaluator; @@ -29,6 +27,8 @@ import org.apache.solr.client.solrj.io.eval.SubtractEvaluator; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.junit.Test; +import junit.framework.Assert; + public class SubtractEvaluatorTest extends LuceneTestCase { StreamFactory factory; @@ -39,7 +39,7 @@ public class SubtractEvaluatorTest extends LuceneTestCase { factory = new StreamFactory() .withFunctionName("sub", SubtractEvaluator.class); - values = new HashedMap(); + values = new HashMap(); } @Test From 6786089b0bc8be50287c2527874ca4503114addd Mon Sep 17 00:00:00 2001 From: yonik Date: Tue, 21 Mar 2017 08:42:33 -0400 Subject: [PATCH 047/563] SOLR-7452: json facet API, refine/skip through buckets already visited --- .../search/facet/FacetFieldProcessor.java | 52 ++++++++++++++----- .../FacetFieldProcessorByEnumTermsStream.java | 2 +- .../apache/solr/search/facet/FacetModule.java | 1 + .../solr/search/facet/FacetProcessor.java | 10 ++-- .../apache/solr/search/facet/FacetQuery.java | 2 +- .../apache/solr/search/facet/FacetRange.java | 4 +- .../search/facet/TestJsonFacetRefinement.java | 22 ++++---- 7 files changed, 62 insertions(+), 31 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java index fb44f62f47b..1ba252e3a75 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java @@ -19,6 +19,7 @@ package org.apache.solr.search.facet; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -311,7 +312,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { if (freq.missing) { // TODO: it would be more efficient to build up a missing DocSet if we need it here anyway. SimpleOrderedMap missingBucket = new SimpleOrderedMap<>(); - fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null, false); + fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null, false, null); res.add("missing", missingBucket); } @@ -379,7 +380,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { } } - processSubs(target, filter, subDomain, false); + processSubs(target, filter, subDomain, false, null); } @Override @@ -513,31 +514,43 @@ abstract class FacetFieldProcessor extends FacetProcessor { } + /* + "qfacet":{"cat2":{"_l":["A"]}}, + "all":{"_s":[[ + "all", + {"cat3":{"_l":["A"]}}]]}, + "cat1":{"_l":["A"]}}} + + */ + + static List asList(Object list) { + return list != null ? (List)list : Collections.EMPTY_LIST; + } protected SimpleOrderedMap refineFacets() throws IOException { - List leaves = (List)fcontext.facetInfo.get("_l"); + List leaves = asList(fcontext.facetInfo.get("_l")); + List skip = asList(fcontext.facetInfo.get("_s")); + List missing = asList(fcontext.facetInfo.get("_m")); // For leaf refinements, we do full faceting for each leaf bucket. Any sub-facets of these buckets will be fully evaluated. Because of this, we should never // encounter leaf refinements that have sub-facets that return partial results. SimpleOrderedMap res = new SimpleOrderedMap<>(); - List bucketList = new ArrayList<>(leaves.size()); + List bucketList = new ArrayList<>( leaves.size() + skip.size() + missing.size() ); res.add("buckets", bucketList); // TODO: an alternate implementations can fill all accs at once createAccs(-1, 1); - FieldType ft = sf.getType(); for (Object bucketVal : leaves) { - SimpleOrderedMap bucket = new SimpleOrderedMap<>(); - bucketList.add(bucket); - bucket.add("val", bucketVal); + bucketList.add( refineBucket(bucketVal, false, null) ); + } + for (List bucketAndFacetInfo : skip) { + assert bucketAndFacetInfo.size() == 2; + Object bucketVal = bucketAndFacetInfo.get(0); + Map facetInfo = (Map) bucketAndFacetInfo.get(1); - // String internal = ft.toInternal( tobj.toString() ); // TODO - we need a better way to get from object to query... - - Query domainQ = ft.getFieldQuery(null, sf, bucketVal.toString()); - - fillBucket(bucket, domainQ, null, false); + bucketList.add( refineBucket(bucketVal, true, facetInfo ) ); } // If there are just a couple of leaves, and if the domain is large, then @@ -548,4 +561,17 @@ abstract class FacetFieldProcessor extends FacetProcessor { return res; } + private SimpleOrderedMap refineBucket(Object bucketVal, boolean skip, Map facetInfo) throws IOException { + SimpleOrderedMap bucket = new SimpleOrderedMap<>(); + FieldType ft = sf.getType(); + bucket.add("val", bucketVal); + // String internal = ft.toInternal( tobj.toString() ); // TODO - we need a better way to get from object to query... + + Query domainQ = ft.getFieldQuery(null, sf, bucketVal.toString()); + + fillBucket(bucket, domainQ, null, skip, facetInfo); + + return bucket; + } + } diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java index 94f3b2d4c1f..d28e024c7f5 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessorByEnumTermsStream.java @@ -333,7 +333,7 @@ class FacetFieldProcessorByEnumTermsStream extends FacetFieldProcessor implement bucket.add("val", bucketVal); addStats(bucket, 0); if (hasSubFacets) { - processSubs(bucket, bucketQuery, termSet, false); + processSubs(bucket, bucketQuery, termSet, false, null); } // TODO... termSet needs to stick around for streaming sub-facets? diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java index 630e96856f6..bf1379162ef 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java @@ -235,6 +235,7 @@ public class FacetModule extends SearchComponent { Map finfo = new HashMap<>(1); finfo.put(FACET_REFINE, refinement); String finfoStr = JSONUtil.toJSON(finfo); + // System.err.println("##################### REFINE=" + finfoStr); shardsRefineRequest.params.add(FACET_INFO, finfoStr); if (newRequest) { diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java index de6dd722d02..cf4d0fe9cb1 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java @@ -367,7 +367,7 @@ public abstract class FacetProcessor { } // TODO: rather than just have a raw "response", perhaps we should model as a bucket object that contains the response plus extra info? - void fillBucket(SimpleOrderedMap bucket, Query q, DocSet result, boolean skip) throws IOException { + void fillBucket(SimpleOrderedMap bucket, Query q, DocSet result, boolean skip, Map facetInfo) throws IOException { // TODO: we don't need the DocSet if we've already calculated everything during the first phase boolean needDocSet = freq.getFacetStats().size() > 0 || freq.getSubFacets().size() > 0; @@ -398,7 +398,7 @@ public abstract class FacetProcessor { if (!skip) { processStats(bucket, result, count); } - processSubs(bucket, q, result, skip); + processSubs(bucket, q, result, skip, facetInfo); } finally { if (result != null) { // result.decref(); // OFF-HEAP @@ -407,7 +407,7 @@ public abstract class FacetProcessor { } } - void processSubs(SimpleOrderedMap response, Query filter, DocSet domain, boolean skip) throws IOException { + void processSubs(SimpleOrderedMap response, Query filter, DocSet domain, boolean skip, Map facetInfo) throws IOException { boolean emptyDomain = domain == null || domain.size() == 0; @@ -423,8 +423,8 @@ public abstract class FacetProcessor { } MapfacetInfoSub = null; - if (fcontext.facetInfo != null) { - facetInfoSub = (Map)fcontext.facetInfo.get(sub.getKey()); + if (facetInfo != null) { + facetInfoSub = (Map)facetInfo.get(sub.getKey()); } // If we're skipping this node, then we only need to process sub-facets that have facet info specified. diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java b/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java index 584bec344c4..a6782bf7091 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetQuery.java @@ -61,7 +61,7 @@ class FacetQueryProcessor extends FacetProcessor { // FIXME - what needs to be done here? } response = new SimpleOrderedMap<>(); - fillBucket(response, freq.q, null, (fcontext.flags & FacetContext.SKIP_FACET)!=0); + fillBucket(response, freq.q, null, (fcontext.flags & FacetContext.SKIP_FACET)!=0, fcontext.facetInfo); } 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 5d0989bd8d7..682dc19036f 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 @@ -350,7 +350,7 @@ class FacetRangeProcessor extends FacetProcessor { if (freq.getSubFacets().size() > 0) { DocSet subBase = intersections[slot]; try { - processSubs(bucket, filters[slot], subBase, false); + processSubs(bucket, filters[slot], subBase, false, null); } finally { // subContext.base.decref(); // OFF-HEAP // subContext.base = null; // do not modify context after creation... there may be deferred execution (i.e. streaming) @@ -367,7 +367,7 @@ class FacetRangeProcessor extends FacetProcessor { } Query rangeQ = sf.getType().getRangeQuery(null, sf, range.low == null ? null : calc.formatValue(range.low), range.high==null ? null : calc.formatValue(range.high), range.includeLower, range.includeUpper); - fillBucket(bucket, rangeQ, null, false); + fillBucket(bucket, rangeQ, null, false, null); return bucket; } diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java index f23ae8c297a..869c90bf1da 100644 --- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java +++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java @@ -227,16 +227,16 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { String cat_s = p.get("cat_s"); String num_d = p.get("num_d"); - clients.get(0).add( sdoc("id", "01", cat_s, "A", num_d, -1) ); // A wins count tie - clients.get(0).add( sdoc("id", "02", cat_s, "B", num_d, 3) ); + clients.get(0).add( sdoc("id", "01", "all_s","all", cat_s, "A", num_d, -1) ); // A wins count tie + clients.get(0).add( sdoc("id", "02", "all_s","all", cat_s, "B", num_d, 3) ); - clients.get(1).add( sdoc("id", "11", cat_s, "B", num_d, -5) ); // B highest count - clients.get(1).add( sdoc("id", "12", cat_s, "B", num_d, -11) ); - clients.get(1).add( sdoc("id", "13", cat_s, "A", num_d, 7) ); + clients.get(1).add( sdoc("id", "11", "all_s","all", cat_s, "B", num_d, -5) ); // B highest count + clients.get(1).add( sdoc("id", "12", "all_s","all", cat_s, "B", num_d, -11) ); + clients.get(1).add( sdoc("id", "13", "all_s","all", cat_s, "A", num_d, 7) ); - clients.get(2).add( sdoc("id", "21", cat_s, "A", num_d, 17) ); // A highest count - clients.get(2).add( sdoc("id", "22", cat_s, "A", num_d, -19) ); - clients.get(2).add( sdoc("id", "23", cat_s, "B", num_d, 11) ); + clients.get(2).add( sdoc("id", "21", "all_s","all", cat_s, "A", num_d, 17) ); // A highest count + clients.get(2).add( sdoc("id", "22", "all_s","all", cat_s, "A", num_d, -19) ); + clients.get(2).add( sdoc("id", "23", "all_s","all", cat_s, "B", num_d, 11) ); client.commit(); @@ -291,12 +291,16 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "json.facet", "{" + " cat0:{type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:false, facet:{ min1:'min(${num_d})'} }" + ",cat1:{type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} }" + + ",qfacet:{type:query, q:'*:*', facet:{ cat2:{type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} } }}" + // refinement needed through a query facet + ",allf:{type:terms, field:all_s, facet:{ cat3:{type:terms, field:${cat_s}, sort:'min1 asc', limit:1, overrequest:0, refine:true, facet:{ min1:'min(${num_d})'} } }}" + // refinement needed through field facet ",sum1:'sum(num_d)'" + // make sure that root bucket stats aren't affected by refinement "}" ) , "facets=={ count:8" + - ", cat0:{ buckets:[ {val:A,count:3, min1:-19.0} ] }" + // B wins in shard2, so we're missing the "A" count for that shar w/o refinement. + ", cat0:{ buckets:[ {val:A,count:3, min1:-19.0} ] }" + // B wins in shard2, so we're missing the "A" count for that shard w/o refinement. ", cat1:{ buckets:[ {val:A,count:4, min1:-19.0} ] }" + // with refinement, we get the right count + ", qfacet:{ count:8, cat2:{ buckets:[ {val:A,count:4, min1:-19.0} ] } }" + // just like the previous response, just nested under a query facet + ", allf:{ buckets:[ {cat3:{ buckets:[ {val:A,count:4, min1:-19.0} ] } ,count:8,val:all }] }" + // just like the previous response, just nested under a field facet ", sum1:2.0" + "}" ); From b7042c1f6e449d7eb33a9daaabda0e0d2a53e95b Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Wed, 22 Mar 2017 15:00:33 +0700 Subject: [PATCH 048/563] Add support for CollapseQParser with PointFields --- solr/CHANGES.txt | 1 + .../solr/search/CollapsingQParserPlugin.java | 40 ++-- .../search/TestCollapseQParserPlugin.java | 216 +++++++++--------- .../TestRandomCollapseQParserPlugin.java | 5 - 4 files changed, 131 insertions(+), 131 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index f61c4c2b125..2bfc981e74e 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -114,6 +114,7 @@ New Features * SOLR-10046: Add UninvertDocValuesMergePolicyFactory class. (Keith Laban, Christine Poerschke) +* SOLR-9994: Add support for CollapseQParser with PointFields. (Varun Thacker, Cao Manh Dat) ================== 6.5.0 ================== diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java index 71478aa61d8..22f9f29e12b 100644 --- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java @@ -63,6 +63,9 @@ import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrRequestInfo; import org.apache.solr.schema.FieldType; +import org.apache.solr.schema.FloatPointField; +import org.apache.solr.schema.IntPointField; +import org.apache.solr.schema.LongPointField; import org.apache.solr.schema.StrField; import org.apache.solr.schema.TrieFloatField; import org.apache.solr.schema.TrieIntField; @@ -962,14 +965,14 @@ public class CollapsingQParserPlugin extends QParserPlugin { } else if (funcQuery != null) { this.collapseStrategy = new OrdValueSourceStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, funcQuery, searcher, collapseValues); } else { - if(fieldType instanceof TrieIntField) { + if (fieldType instanceof TrieIntField || fieldType instanceof IntPointField) { this.collapseStrategy = new OrdIntStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, collapseValues); - } else if(fieldType instanceof TrieFloatField) { + } else if (fieldType instanceof TrieFloatField || fieldType instanceof FloatPointField) { this.collapseStrategy = new OrdFloatStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, collapseValues); - } else if(fieldType instanceof TrieLongField) { + } else if (fieldType instanceof TrieLongField || fieldType instanceof LongPointField) { this.collapseStrategy = new OrdLongStrategy(maxDoc, nullPolicy, new int[valueCount], groupHeadSelector, this.needsScores, boostDocs, collapseValues); } else { - throw new IOException("min/max must be either TrieInt, TrieLong, TrieFloat."); + throw new IOException("min/max must be either Int/Long/Float field types"); } } } @@ -1146,12 +1149,12 @@ public class CollapsingQParserPlugin extends QParserPlugin { } else if (funcQuery != null) { this.collapseStrategy = new IntValueSourceStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap, funcQuery, searcher); } else { - if(fieldType instanceof TrieIntField) { + if (fieldType instanceof TrieIntField || fieldType instanceof IntPointField) { this.collapseStrategy = new IntIntStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap); - } else if(fieldType instanceof TrieFloatField) { + } else if (fieldType instanceof TrieFloatField || fieldType instanceof FloatPointField) { this.collapseStrategy = new IntFloatStrategy(maxDoc, size, collapseField, nullValue, nullPolicy, groupHeadSelector, this.needsScores, boostDocsMap); } else { - throw new IOException("min/max must be TrieInt or TrieFloat when collapsing on numeric fields ."); + throw new IOException("min/max must be Int or Float field types when collapsing on numeric fields"); } } } @@ -1259,6 +1262,15 @@ public class CollapsingQParserPlugin extends QParserPlugin { private static class CollectorFactory { + private boolean isNumericCollapsible(FieldType collapseFieldType) { + if (collapseFieldType instanceof TrieIntField || collapseFieldType instanceof IntPointField || + collapseFieldType instanceof TrieFloatField || collapseFieldType instanceof FloatPointField) { + return true; + } else { + return false; + } + } + public DelegatingCollector getCollector(String collapseField, GroupHeadSelector groupHeadSelector, SortSpec sortSpec, @@ -1335,19 +1347,18 @@ public class CollapsingQParserPlugin extends QParserPlugin { return new OrdScoreCollector(maxDoc, leafCount, docValuesProducer, nullPolicy, boostDocs); - } else if (collapseFieldType instanceof TrieIntField || - collapseFieldType instanceof TrieFloatField) { + } else if (isNumericCollapsible(collapseFieldType)) { int nullValue = 0; - if(collapseFieldType instanceof TrieFloatField) { - if(defaultValue != null) { + if (collapseFieldType instanceof TrieFloatField || collapseFieldType instanceof FloatPointField) { + if (defaultValue != null) { nullValue = Float.floatToIntBits(Float.parseFloat(defaultValue)); } else { nullValue = Float.floatToIntBits(0.0f); } } else { - if(defaultValue != null) { + if (defaultValue != null) { nullValue = Integer.parseInt(defaultValue); } } @@ -1374,12 +1385,11 @@ public class CollapsingQParserPlugin extends QParserPlugin { funcQuery, searcher); - } else if((collapseFieldType instanceof TrieIntField || - collapseFieldType instanceof TrieFloatField)) { + } else if(isNumericCollapsible(collapseFieldType)) { int nullValue = 0; - if(collapseFieldType instanceof TrieFloatField) { + if (collapseFieldType instanceof TrieFloatField || collapseFieldType instanceof FloatPointField) { if(defaultValue != null) { nullValue = Float.floatToIntBits(Float.parseFloat(defaultValue)); } else { 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 ea4114a08b0..026e194800d 100644 --- a/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java +++ b/solr/core/src/test/org/apache/solr/search/TestCollapseQParserPlugin.java @@ -16,16 +16,14 @@ */ package org.apache.solr.search; -import java.util.List; import java.util.ArrayList; import java.util.Arrays; -import java.util.Set; import java.util.HashSet; import java.util.Iterator; +import java.util.List; +import java.util.Set; -import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.SolrTestCaseJ4.SuppressPointFields; import org.apache.solr.common.SolrException; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.SolrParams; @@ -35,11 +33,7 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -//We want codecs that support DocValues, and ones supporting blank/empty values. -@SuppressCodecs({"Appending","Lucene3x","Lucene40","Lucene41","Lucene42"}) -@SuppressPointFields public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { - @BeforeClass public static void beforeClass() throws Exception { initCore("solrconfig-collapseqparser.xml", "schema11.xml"); @@ -56,17 +50,17 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { } public void testMultiSort() throws Exception { - assertU(adoc("id", "1", "group_s", "group1", "test_ti", "5", "test_tl", "10")); + assertU(adoc("id", "1", "group_s", "group1", "test_i", "5", "test_l", "10")); assertU(commit()); - assertU(adoc("id", "2", "group_s", "group1", "test_ti", "5", "test_tl", "1000")); - assertU(adoc("id", "3", "group_s", "group1", "test_ti", "5", "test_tl", "1000")); - assertU(adoc("id", "4", "group_s", "group1", "test_ti", "10", "test_tl", "100")); + assertU(adoc("id", "2", "group_s", "group1", "test_i", "5", "test_l", "1000")); + assertU(adoc("id", "3", "group_s", "group1", "test_i", "5", "test_l", "1000")); + assertU(adoc("id", "4", "group_s", "group1", "test_i", "10", "test_l", "100")); // - assertU(adoc("id", "5", "group_s", "group2", "test_ti", "5", "test_tl", "10", "term_s", "YYYY")); + assertU(adoc("id", "5", "group_s", "group2", "test_i", "5", "test_l", "10", "term_s", "YYYY")); assertU(commit()); - assertU(adoc("id", "6", "group_s", "group2", "test_ti", "5", "test_tl","1000")); - assertU(adoc("id", "7", "group_s", "group2", "test_ti", "5", "test_tl","1000", "term_s", "XXXX")); - assertU(adoc("id", "8", "group_s", "group2", "test_ti", "10","test_tl", "100")); + assertU(adoc("id", "6", "group_s", "group2", "test_i", "5", "test_l","1000")); + assertU(adoc("id", "7", "group_s", "group2", "test_i", "5", "test_l","1000", "term_s", "XXXX")); + assertU(adoc("id", "8", "group_s", "group2", "test_i", "10","test_l", "100")); assertU(commit()); ModifiableSolrParams params; @@ -75,7 +69,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params = new ModifiableSolrParams(); params.add("q", "*:*"); params.add("fq", "{!collapse field=group_s sort=$sort}"); - params.add("sort", "test_ti asc, test_tl desc, id desc"); + params.add("sort", "test_i asc, test_l desc, id desc"); assertQ(req(params) , "*[count(//doc)=2]" ,"//result/doc[1]/float[@name='id'][.='7.0']" @@ -85,7 +79,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // group heads are selected using a complex sort, simpler sort used for final groups params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=group_s sort='test_ti asc, test_tl desc, id desc'}"); + params.add("fq", "{!collapse field=group_s sort='test_i asc, test_l desc, id desc'}"); params.add("sort", "id asc"); assertQ(req(params) , "*[count(//doc)=2]" @@ -96,7 +90,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // diff up the sort directions, only first clause matters with our data params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=group_s sort='test_ti desc, test_tl asc, id asc'}"); + params.add("fq", "{!collapse field=group_s sort='test_i desc, test_l asc, id asc'}"); params.add("sort", "id desc"); assertQ(req(params) , "*[count(//doc)=2]" @@ -107,7 +101,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // tie broken by index order params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=group_s sort='test_tl desc'}"); + params.add("fq", "{!collapse field=group_s sort='test_l desc'}"); params.add("sort", "id desc"); assertQ(req(params) , "*[count(//doc)=2]" @@ -118,7 +112,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // score, then tiebreakers; note top level sort by score ASCENDING (just for weirdness) params = new ModifiableSolrParams(); params.add("q", "*:* term_s:YYYY"); - params.add("fq", "{!collapse field=group_s sort='score desc, test_tl desc, test_ti asc, id asc'}"); + params.add("fq", "{!collapse field=group_s sort='score desc, test_l desc, test_i asc, id asc'}"); params.add("sort", "score asc"); assertQ(req(params) , "*[count(//doc)=2]" @@ -129,7 +123,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // score, then tiebreakers; note no score in top level sort/fl to check needsScores logic params = new ModifiableSolrParams(); params.add("q", "*:* term_s:YYYY"); - params.add("fq", "{!collapse field=group_s sort='score desc, test_tl desc, test_ti asc, id asc'}"); + params.add("fq", "{!collapse field=group_s sort='score desc, test_l desc, test_i asc, id asc'}"); params.add("sort", "id desc"); assertQ(req(params) , "*[count(//doc)=2]" @@ -140,7 +134,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // term_s desc -- term_s is missing from many docs, and uses sortMissingLast=true params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=group_s sort='term_s desc, test_tl asc'}"); + params.add("fq", "{!collapse field=group_s sort='term_s desc, test_l asc'}"); params.add("sort", "id asc"); assertQ(req(params) , "*[count(//doc)=2]" @@ -151,7 +145,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // term_s asc -- term_s is missing from many docs, and uses sortMissingLast=true params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=group_s sort='term_s asc, test_tl asc'}"); + params.add("fq", "{!collapse field=group_s sort='term_s asc, test_l asc'}"); params.add("sort", "id asc"); assertQ(req(params) , "*[count(//doc)=2]" @@ -162,7 +156,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // collapse on int field params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=test_ti sort='term_s asc, group_s asc'}"); + params.add("fq", "{!collapse field=test_i sort='term_s asc, group_s asc'}"); params.add("sort", "id asc"); assertQ(req(params) , "*[count(//doc)=2]" @@ -173,8 +167,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // collapse on term_s (very sparse) with nullPolicy=collapse params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=term_s nullPolicy=collapse sort='test_ti asc, test_tl desc, id asc'}"); - params.add("sort", "test_tl asc, id asc"); + params.add("fq", "{!collapse field=term_s nullPolicy=collapse sort='test_i asc, test_l desc, id asc'}"); + params.add("sort", "test_l asc, id asc"); assertQ(req(params) , "*[count(//doc)=3]" ,"//result/doc[1]/float[@name='id'][.='5.0']" @@ -185,8 +179,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // sort local param + elevation params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=group_s sort='term_s desc, test_tl asc'}"); - params.add("sort", "test_tl asc"); + params.add("fq", "{!collapse field=group_s sort='term_s desc, test_l asc'}"); + params.add("sort", "test_l asc"); params.add("qt", "/elevate"); params.add("forceElevation", "true"); params.add("elevateIds", "4.0"); @@ -197,8 +191,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=group_s sort='term_s desc, test_tl asc'}"); - params.add("sort", "test_tl asc"); + params.add("fq", "{!collapse field=group_s sort='term_s desc, test_l asc'}"); + params.add("sort", "test_l asc"); params.add("qt", "/elevate"); params.add("forceElevation", "true"); params.add("elevateIds", "7.0"); @@ -228,38 +222,38 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { @Test public void testFieldValueCollapseWithNegativeMinMax() throws Exception { - String[] doc = {"id","1", "group_i", "-1000", "test_ti", "5", "test_tl", "-10", "test_tf", "2000.32"}; + String[] doc = {"id","1", "group_i", "-1000", "test_i", "5", "test_l", "-10", "test_f", "2000.32"}; assertU(adoc(doc)); assertU(commit()); - String[] doc1 = {"id","2", "group_i", "-1000", "test_ti", "50", "test_tl", "-100", "test_tf", "2000.33"}; + String[] doc1 = {"id","2", "group_i", "-1000", "test_i", "50", "test_l", "-100", "test_f", "2000.33"}; assertU(adoc(doc1)); - String[] doc2 = {"id","3", "group_i", "-1000", "test_tl", "100", "test_tf", "200"}; + String[] doc2 = {"id","3", "group_i", "-1000", "test_l", "100", "test_f", "200"}; assertU(adoc(doc2)); assertU(commit()); - String[] doc3 = {"id","4", "test_ti", "500", "test_tl", "1000", "test_tf", "2000"}; + String[] doc3 = {"id","4", "test_i", "500", "test_l", "1000", "test_f", "2000"}; assertU(adoc(doc3)); - String[] doc4 = {"id","5", "group_i", "-1000", "test_ti", "4", "test_tl", "10", "test_tf", "2000.31"}; + String[] doc4 = {"id","5", "group_i", "-1000", "test_i", "4", "test_l", "10", "test_f", "2000.31"}; assertU(adoc(doc4)); assertU(commit()); - String[] doc5 = {"id","6", "group_i", "-1000", "test_ti", "10", "test_tl", "100", "test_tf", "-2000.12"}; + String[] doc5 = {"id","6", "group_i", "-1000", "test_i", "10", "test_l", "100", "test_f", "-2000.12"}; assertU(adoc(doc5)); assertU(commit()); - String[] doc6 = {"id","7", "group_i", "-1000", "test_ti", "8", "test_tl", "-50", "test_tf", "-100.2"}; + String[] doc6 = {"id","7", "group_i", "-1000", "test_i", "8", "test_l", "-50", "test_f", "-100.2"}; assertU(adoc(doc6)); assertU(commit()); ModifiableSolrParams params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=group_i min=test_tf}"); + params.add("fq", "{!collapse field=group_i min=test_f}"); assertQ(req(params), "*[count(//doc)=1]", "//result/doc[1]/float[@name='id'][.='6.0']"); params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field=group_i max=test_tf}"); + params.add("fq", "{!collapse field=group_i max=test_f}"); assertQ(req(params), "*[count(//doc)=1]", "//result/doc[1]/float[@name='id'][.='2.0']"); @@ -349,29 +343,29 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { private void testCollapseQueries(String group, String hint, boolean numeric) throws Exception { - String[] doc = {"id","1", "term_s", "YYYY", group, "1", "test_ti", "5", "test_tl", "10", "test_tf", "2000"}; + String[] doc = {"id","1", "term_s", "YYYY", group, "1", "test_i", "5", "test_l", "10", "test_f", "2000"}; assertU(adoc(doc)); assertU(commit()); - String[] doc1 = {"id","2", "term_s","YYYY", group, "1", "test_ti", "50", "test_tl", "100", "test_tf", "200"}; + String[] doc1 = {"id","2", "term_s","YYYY", group, "1", "test_i", "50", "test_l", "100", "test_f", "200"}; assertU(adoc(doc1)); - String[] doc2 = {"id","3", "term_s", "YYYY", "test_ti", "5000", "test_tl", "100", "test_tf", "200"}; + String[] doc2 = {"id","3", "term_s", "YYYY", "test_i", "5000", "test_l", "100", "test_f", "200"}; assertU(adoc(doc2)); assertU(commit()); - String[] doc3 = {"id","4", "term_s", "YYYY", "test_ti", "500", "test_tl", "1000", "test_tf", "2000"}; + String[] doc3 = {"id","4", "term_s", "YYYY", "test_i", "500", "test_l", "1000", "test_f", "2000"}; assertU(adoc(doc3)); - String[] doc4 = {"id","5", "term_s", "YYYY", group, "2", "test_ti", "4", "test_tl", "10", "test_tf", "2000"}; + String[] doc4 = {"id","5", "term_s", "YYYY", group, "2", "test_i", "4", "test_l", "10", "test_f", "2000"}; assertU(adoc(doc4)); assertU(commit()); - String[] doc5 = {"id","6", "term_s","YYYY", group, "2", "test_ti", "10", "test_tl", "100", "test_tf", "200"}; + String[] doc5 = {"id","6", "term_s","YYYY", group, "2", "test_i", "10", "test_l", "100", "test_f", "200"}; assertU(adoc(doc5)); assertU(commit()); - String[] doc6 = {"id","7", "term_s", "YYYY", group, "1", "test_ti", "8", "test_tl", "50", "test_tf", "300"}; + String[] doc6 = {"id","7", "term_s", "YYYY", group, "1", "test_i", "8", "test_l", "50", "test_f", "300"}; assertU(adoc(doc6)); assertU(commit()); @@ -381,7 +375,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params.add("q", "*:*"); params.add("fq", "{!collapse field="+group+""+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); assertQ(req(params, "indent", "on"), "*[count(//doc)=2]", "//result/doc[1]/float[@name='id'][.='2.0']", "//result/doc[2]/float[@name='id'][.='6.0']" @@ -391,9 +385,9 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // SOLR-5544 test ordering with empty sort param params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" nullPolicy=expand min=test_tf"+hint+"}"); + params.add("fq", "{!collapse field="+group+" nullPolicy=expand min=test_f"+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); params.add("sort",""); assertQ(req(params), "*[count(//doc)=4]", "//result/doc[1]/float[@name='id'][.='3.0']", @@ -405,8 +399,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // Test value source collapse criteria params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" nullPolicy=collapse min=field(test_ti)"+hint+"}"); - params.add("sort", "test_ti desc"); + params.add("fq", "{!collapse field="+group+" nullPolicy=collapse min=field(test_i)"+hint+"}"); + params.add("sort", "test_i desc"); assertQ(req(params), "*[count(//doc)=3]", "//result/doc[1]/float[@name='id'][.='4.0']", "//result/doc[2]/float[@name='id'][.='1.0']", @@ -418,7 +412,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params.add("q", "*:*"); params.add("fq", "{!collapse field="+group+" nullPolicy=collapse min=cscore()"+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); assertQ(req(params), "*[count(//doc)=3]", "//result/doc[1]/float[@name='id'][.='4.0']", "//result/doc[2]/float[@name='id'][.='1.0']", @@ -430,7 +424,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params.add("q", "*:*"); params.add("fq", "{!collapse field="+group+" nullPolicy=collapse min=cscore()"+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); params.add("fl", "id"); params.add("sort", "id desc"); assertQ(req(params), "*[count(//doc)=3]", @@ -442,9 +436,9 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // Test value source collapse criteria with compound cscore function params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" nullPolicy=collapse min=sum(cscore(),field(test_ti))"+hint+"}"); + params.add("fq", "{!collapse field="+group+" nullPolicy=collapse min=sum(cscore(),field(test_i))"+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); assertQ(req(params), "*[count(//doc)=3]", "//result/doc[1]/float[@name='id'][.='4.0']", "//result/doc[2]/float[@name='id'][.='1.0']", @@ -457,7 +451,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params.add("q", "YYYY"); params.add("fq", "{!collapse field="+group+" nullPolicy=collapse"+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); params.add("qf", "term_s"); params.add("qt", "/elevate"); assertQ(req(params), "*[count(//doc)=4]", @@ -473,7 +467,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params.add("q", "YYYY"); params.add("fq", "{!collapse field="+group + maxscore + " nullPolicy=collapse"+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); params.add("qf", "term_s"); params.add("qt", "/elevate"); params.add("elevateIds", "1,5"); @@ -485,12 +479,12 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test SOLR-5773 with max field collapse criteria // try both max & sort localparams as alternate ways to ask for max group head - for (String max : new String[] {" max=test_ti ", " sort='test_ti desc' "}) { + for (String max : new String[] {" max=test_i ", " sort='test_i desc' "}) { params = new ModifiableSolrParams(); params.add("q", "YYYY"); params.add("fq", "{!collapse field=" + group + max + "nullPolicy=collapse"+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); params.add("qf", "term_s"); params.add("qt", "/elevate"); params.add("elevateIds", "1,5"); @@ -502,12 +496,12 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test SOLR-5773 with min field collapse criteria // try both min & sort localparams as alternate ways to ask for min group head - for (String min : new String[] {" min=test_ti ", " sort='test_ti asc' "}) { + for (String min : new String[] {" min=test_i ", " sort='test_i asc' "}) { params = new ModifiableSolrParams(); params.add("q", "YYYY"); params.add("fq", "{!collapse field=" + group + min + "nullPolicy=collapse"+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); params.add("qf", "term_s"); params.add("qt", "/elevate"); params.add("elevateIds", "1,5"); @@ -522,7 +516,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params.add("q", "YYYY"); params.add("fq", "{!collapse field="+group+""+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); params.add("qf", "term_s"); params.add("qt", "/elevate"); params.add("elevateIds", "3,4"); @@ -536,7 +530,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // Non trivial sort local param for picking group head params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" nullPolicy=collapse sort='term_s asc, test_ti asc' "+hint+"}"); + params.add("fq", "{!collapse field="+group+" nullPolicy=collapse sort='term_s asc, test_i asc' "+hint+"}"); params.add("sort", "id desc"); assertQ(req(params), "*[count(//doc)=3]", @@ -547,7 +541,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" nullPolicy=collapse sort='term_s asc, test_ti desc' "+hint+"}"); + params.add("fq", "{!collapse field="+group+" nullPolicy=collapse sort='term_s asc, test_i desc' "+hint+"}"); params.add("sort", "id desc"); assertQ(req(params), "*[count(//doc)=3]", @@ -560,7 +554,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // Test collapse by min int field and top level sort // try both min & sort localparams as alternate ways to ask for min group head - for (String min : new String[] {" min=test_ti ", " sort='test_ti asc' "}) { + for (String min : new String[] {" min=test_i ", " sort='test_i asc' "}) { params = new ModifiableSolrParams(); params.add("q", "*:*"); params.add("fq", "{!collapse field="+group + min + hint+"}"); @@ -582,7 +576,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params = new ModifiableSolrParams(); params.add("q", "*:*"); params.add("fq", "{!collapse field="+group + min + hint+"}"); - params.add("sort", "test_tl asc,id desc"); + params.add("sort", "test_l asc,id desc"); assertQ(req(params), "*[count(//doc)=2]", "//result/doc[1]/float[@name='id'][.='5.0']", @@ -604,8 +598,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test collapse by max int field params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" max=test_ti"+hint+"}"); - params.add("sort", "test_ti asc"); + params.add("fq", "{!collapse field="+group+" max=test_i"+hint+"}"); + params.add("sort", "test_i asc"); assertQ(req(params), "*[count(//doc)=2]", "//result/doc[1]/float[@name='id'][.='6.0']", "//result/doc[2]/float[@name='id'][.='2.0']" @@ -615,8 +609,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test collapse by min long field params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" min=test_tl"+hint+"}"); - params.add("sort", "test_ti desc"); + params.add("fq", "{!collapse field="+group+" min=test_l"+hint+"}"); + params.add("sort", "test_i desc"); assertQ(req(params), "*[count(//doc)=2]", "//result/doc[1]/float[@name='id'][.='1.0']", "//result/doc[2]/float[@name='id'][.='5.0']"); @@ -625,8 +619,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test collapse by max long field params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" max=test_tl"+hint+"}"); - params.add("sort", "test_ti desc"); + params.add("fq", "{!collapse field="+group+" max=test_l"+hint+"}"); + params.add("sort", "test_i desc"); assertQ(req(params), "*[count(//doc)=2]", "//result/doc[1]/float[@name='id'][.='2.0']", "//result/doc[2]/float[@name='id'][.='6.0']"); @@ -640,8 +634,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test collapse by min float field params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" min=test_tf"+hint+"}"); - params.add("sort", "test_ti desc"); + params.add("fq", "{!collapse field="+group+" min=test_f"+hint+"}"); + params.add("sort", "test_i desc"); assertQ(req(params), "*[count(//doc)=2]", "//result/doc[1]/float[@name='id'][.='2.0']", "//result/doc[2]/float[@name='id'][.='6.0']"); @@ -649,8 +643,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test collapse by min float field params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" max=test_tf"+hint+"}"); - params.add("sort", "test_ti asc"); + params.add("fq", "{!collapse field="+group+" max=test_f"+hint+"}"); + params.add("sort", "test_i asc"); assertQ(req(params), "*[count(//doc)=2]", "//result/doc[1]/float[@name='id'][.='5.0']", "//result/doc[2]/float[@name='id'][.='1.0']"); @@ -658,7 +652,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test collapse by min float field sort by score params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" max=test_tf"+hint+"}"); + params.add("fq", "{!collapse field="+group+" max=test_f"+hint+"}"); params.add("defType", "edismax"); params.add("bf", "field(id)"); params.add("fl", "score, id"); @@ -673,8 +667,8 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // Test collapse using selector field in no docs // tie selector in all of these cases for (String selector : new String[] { - " min=bogus_ti ", " sort='bogus_ti asc' ", - " max=bogus_ti ", " sort='bogus_ti desc' ", + " min=bogus_i ", " sort='bogus_i asc' ", + " max=bogus_i ", " sort='bogus_i desc' ", " min=bogus_tf ", " sort='bogus_tf asc' ", " max=bogus_tf ", " sort='bogus_tf desc' ", " sort='bogus_td asc' ", " sort='bogus_td desc' ", @@ -695,16 +689,16 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // attempting to use cscore() in sort local param should fail assertQEx("expected error trying to sort on a function that includes cscore()", - req(params("q", "{!func}sub(sub(test_tl,1000),id)", + req(params("q", "{!func}sub(sub(test_l,1000),id)", "fq", "{!collapse field="+group+" sort='abs(cscore()) asc, id asc'}", "sort", "score asc")), SolrException.ErrorCode.BAD_REQUEST); // multiple params for picking groupHead should all fail for (String bad : new String[] { - "{!collapse field="+group+" min=test_tf max=test_tf}", - "{!collapse field="+group+" min=test_tf sort='test_tf asc'}", - "{!collapse field="+group+" max=test_tf sort='test_tf asc'}" }) { + "{!collapse field="+group+" min=test_f max=test_f}", + "{!collapse field="+group+" min=test_f sort='test_f asc'}", + "{!collapse field="+group+" max=test_f sort='test_f asc'}" }) { assertQEx("Expected error: " + bad, req(params("q", "*:*", "fq", bad)), SolrException.ErrorCode.BAD_REQUEST); } @@ -713,15 +707,15 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // sort used for (SolrParams collapse : new SolrParams[] { // these should all be equivilently valid - params("fq", "{!collapse field="+group+" nullPolicy=collapse sort='test_ti asc'"+hint+"}"), - params("fq", "{!collapse field="+group+" nullPolicy=collapse min='' sort='test_ti asc'"+hint+"}"), - params("fq", "{!collapse field="+group+" nullPolicy=collapse max='' sort='test_ti asc'"+hint+"}"), - params("fq", "{!collapse field="+group+" nullPolicy=collapse min=$x sort='test_ti asc'"+hint+"}"), - params("fq", "{!collapse field="+group+" nullPolicy=collapse min=$x sort='test_ti asc'"+hint+"}", + params("fq", "{!collapse field="+group+" nullPolicy=collapse sort='test_i asc'"+hint+"}"), + params("fq", "{!collapse field="+group+" nullPolicy=collapse min='' sort='test_i asc'"+hint+"}"), + params("fq", "{!collapse field="+group+" nullPolicy=collapse max='' sort='test_i asc'"+hint+"}"), + params("fq", "{!collapse field="+group+" nullPolicy=collapse min=$x sort='test_i asc'"+hint+"}"), + params("fq", "{!collapse field="+group+" nullPolicy=collapse min=$x sort='test_i asc'"+hint+"}", "x",""), }) { - assertQ(req(collapse, "q", "*:*", "sort", "test_ti desc"), + assertQ(req(collapse, "q", "*:*", "sort", "test_i desc"), "*[count(//doc)=3]", "//result/doc[1]/float[@name='id'][.='4.0']", "//result/doc[2]/float[@name='id'][.='1.0']", @@ -732,7 +726,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test nullPolicy expand params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" max=test_tf nullPolicy=expand"+hint+"}"); + params.add("fq", "{!collapse field="+group+" max=test_f nullPolicy=expand"+hint+"}"); params.add("sort", "id desc"); assertQ(req(params), "*[count(//doc)=4]", "//result/doc[1]/float[@name='id'][.='5.0']", @@ -743,7 +737,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { //Test nullPolicy collapse params = new ModifiableSolrParams(); params.add("q", "*:*"); - params.add("fq", "{!collapse field="+group+" max=test_tf nullPolicy=collapse"+hint+"}"); + params.add("fq", "{!collapse field="+group+" max=test_f nullPolicy=collapse"+hint+"}"); params.add("sort", "id desc"); assertQ(req(params), "*[count(//doc)=3]", "//result/doc[1]/float[@name='id'][.='5.0']", @@ -755,12 +749,12 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params.add("q", "*:*"); params.add("fq", "{!collapse field="+group+hint+"}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); - params.add("fq","{!tag=test_ti}id:5"); + params.add("bf", "field(test_i)"); + params.add("fq","{!tag=test_i}id:5"); params.add("facet","true"); - params.add("facet.field","{!ex=test_ti}test_ti"); + params.add("facet.field","{!ex=test_i}test_i"); params.add("facet.mincount", "1"); - assertQ(req(params), "*[count(//doc)=1]", "*[count(//lst[@name='facet_fields']/lst[@name='test_ti']/int)=2]"); + assertQ(req(params), "*[count(//doc)=1]", "*[count(//lst[@name='facet_fields']/lst[@name='test_i']/int)=2]"); // SOLR-5230 - ensure CollapsingFieldValueCollector.finish() is called params = new ModifiableSolrParams(); @@ -779,7 +773,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { params.add("q", "YYYY"); params.add("fq", "{!collapse field="+group+hint+" nullPolicy=collapse}"); params.add("defType", "edismax"); - params.add("bf", "field(test_ti)"); + params.add("bf", "field(test_i)"); params.add("qf", "term_s"); params.add("qt", "/elevate"); assertQ(req(params), "*[count(//doc)=3]", @@ -805,7 +799,7 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { String group = (random().nextBoolean() ? "group_s" : "group_s_dv"); // min-or-max is for CollapsingScoreCollector vs. CollapsingFieldValueCollector - String optional_min_or_max = (random().nextBoolean() ? "" : (random().nextBoolean() ? "min=field(test_ti)" : "max=field(test_ti)")); + String optional_min_or_max = (random().nextBoolean() ? "" : (random().nextBoolean() ? "min=field(test_i)" : "max=field(test_i)")); ModifiableSolrParams params = new ModifiableSolrParams(); params.add("q", "*:*"); @@ -817,17 +811,17 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { // as unlikely as this test seems, it's important for the possibility that a segment exists w/o // any live docs that have DocValues for the group field -- ie: every doc in segment is in null group. - assertU(adoc("id", "1", "group_s", "group1", "test_ti", "5", "test_tl", "10")); + assertU(adoc("id", "1", "group_s", "group1", "test_i", "5", "test_l", "10")); assertU(commit()); - assertU(adoc("id", "2", "group_s", "group1", "test_ti", "5", "test_tl", "1000")); - assertU(adoc("id", "3", "group_s", "group1", "test_ti", "5", "test_tl", "1000")); - assertU(adoc("id", "4", "group_s", "group1", "test_ti", "10", "test_tl", "100")); + assertU(adoc("id", "2", "group_s", "group1", "test_i", "5", "test_l", "1000")); + assertU(adoc("id", "3", "group_s", "group1", "test_i", "5", "test_l", "1000")); + assertU(adoc("id", "4", "group_s", "group1", "test_i", "10", "test_l", "100")); // - assertU(adoc("id", "5", "group_s", "group2", "test_ti", "5", "test_tl", "10", "term_s", "YYYY")); + assertU(adoc("id", "5", "group_s", "group2", "test_i", "5", "test_l", "10", "term_s", "YYYY")); assertU(commit()); - assertU(adoc("id", "6", "group_s", "group2", "test_ti", "5", "test_tl","1000")); - assertU(adoc("id", "7", "group_s", "group2", "test_ti", "5", "test_tl","1000", "term_s", "XXXX")); - assertU(adoc("id", "8", "group_s", "group2", "test_ti", "10","test_tl", "100")); + assertU(adoc("id", "6", "group_s", "group2", "test_i", "5", "test_l","1000")); + assertU(adoc("id", "7", "group_s", "group2", "test_i", "5", "test_l","1000", "term_s", "XXXX")); + assertU(adoc("id", "8", "group_s", "group2", "test_i", "10","test_l", "100")); assertU(commit()); // none of these grouping fields are in any doc @@ -835,17 +829,17 @@ public class TestCollapseQParserPlugin extends SolrTestCaseJ4 { "field=bogus_s", "field=bogus_s_dv", "field=bogus_s hint=top_fc", // alternative docvalues codepath w/ hint "field=bogus_s_dv hint=top_fc", // alternative docvalues codepath w/ hint - "field=bogus_ti", "field=bogus_tf" }) { + "field=bogus_i", "field=bogus_tf" }) { // for any of these selectors, behavior of these checks should be consistent for (String selector : new String[] { "", " sort='score desc' ", - " min=test_ti ", " max=test_ti ", " sort='test_ti asc' ", " sort='test_ti desc' ", - " min=test_tf ", " max=test_tf ", " sort='test_tf asc' ", " sort='test_tf desc' ", + " min=test_i ", " max=test_i ", " sort='test_i asc' ", " sort='test_i desc' ", + " min=test_f ", " max=test_f ", " sort='test_f asc' ", " sort='test_f desc' ", " sort='group_s asc' ", " sort='group_s desc' ", // fields that don't exist - " min=bogus_sort_ti ", " max=bogus_sort_ti ", - " sort='bogus_sort_ti asc' ", " sort='bogus_sort_ti desc' ", + " min=bogus_sort_i ", " max=bogus_sort_i ", + " sort='bogus_sort_i asc' ", " sort='bogus_sort_i desc' ", " sort='bogus_sort_s asc' ", " sort='bogus_sort_s desc' ", }) { diff --git a/solr/core/src/test/org/apache/solr/search/TestRandomCollapseQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/TestRandomCollapseQParserPlugin.java index 7d135e2344e..bbb26236c17 100644 --- a/solr/core/src/test/org/apache/solr/search/TestRandomCollapseQParserPlugin.java +++ b/solr/core/src/test/org/apache/solr/search/TestRandomCollapseQParserPlugin.java @@ -20,11 +20,9 @@ import java.util.List; import java.util.ArrayList; import java.util.Arrays; -import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.TestUtil; import org.apache.solr.CursorPagingTest; import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.SolrTestCaseJ4.SuppressPointFields; import org.apache.solr.client.solrj.SolrClient; import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer; import org.apache.solr.client.solrj.response.QueryResponse; @@ -37,9 +35,6 @@ import static org.apache.solr.search.CollapsingQParserPlugin.NULL_EXPAND; import org.junit.AfterClass; import org.junit.BeforeClass; -//We want codecs that support DocValues, and ones supporting blank/empty values. -@SuppressCodecs({"Appending","Lucene3x","Lucene40","Lucene41","Lucene42"}) -@SuppressPointFields public class TestRandomCollapseQParserPlugin extends SolrTestCaseJ4 { /** Full SolrServer instance for arbitrary introspection of response data and adding fqs */ From 8a996753920170ac1e6e8960d6b63848ccc1ea44 Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Wed, 22 Mar 2017 10:52:14 -0700 Subject: [PATCH 049/563] SOLR-9986: Add javadoc to DatePointField class --- .../apache/solr/schema/DatePointField.java | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) diff --git a/solr/core/src/java/org/apache/solr/schema/DatePointField.java b/solr/core/src/java/org/apache/solr/schema/DatePointField.java index 377f571f299..50f85e30d30 100644 --- a/solr/core/src/java/org/apache/solr/schema/DatePointField.java +++ b/solr/core/src/java/org/apache/solr/schema/DatePointField.java @@ -36,8 +36,70 @@ import org.apache.lucene.util.mutable.MutableValueDate; import org.apache.lucene.util.mutable.MutableValueLong; import org.apache.solr.search.QParser; import org.apache.solr.uninverting.UninvertingReader; +import org.apache.solr.update.processor.TimestampUpdateProcessorFactory; import org.apache.solr.util.DateMathParser; +/** + * FieldType that can represent any Date/Time with millisecond precision. + *

+ * Date Format for the XML, incoming and outgoing: + *

+ *
+ * A date field shall be of the form 1995-12-31T23:59:59Z + * The trailing "Z" designates UTC time and is mandatory + * (See below for an explanation of UTC). + * Optional fractional seconds are allowed, as long as they do not end + * in a trailing 0 (but any precision beyond milliseconds will be ignored). + * All other parts are mandatory. + *
+ *

+ * This format was derived to be standards compliant (ISO 8601) and is a more + * restricted form of the + * canonical + * representation of dateTime from XML schema part 2. Examples... + *

+ *
    + *
  • 1995-12-31T23:59:59Z
  • + *
  • 1995-12-31T23:59:59.9Z
  • + *
  • 1995-12-31T23:59:59.99Z
  • + *
  • 1995-12-31T23:59:59.999Z
  • + *
+ *

+ * Note that DatePointField is lenient with regards to parsing fractional + * seconds that end in trailing zeros and will ensure that those values + * are indexed in the correct canonical format. + *

+ *

+ * This FieldType also supports incoming "Date Math" strings for computing + * values by adding/rounding internals of time relative either an explicit + * datetime (in the format specified above) or the literal string "NOW", + * ie: "NOW+1YEAR", "NOW/DAY", "1995-12-31T23:59:59.999Z+5MINUTES", etc... + * -- see {@link DateMathParser} for more examples. + *

+ *

+ * NOTE: Although it is possible to configure a DatePointField + * instance with a default value of "NOW" to compute a timestamp + * of when the document was indexed, this is not advisable when using SolrCloud + * since each replica of the document may compute a slightly different value. + * {@link TimestampUpdateProcessorFactory} is recommended instead. + *

+ * + *

+ * Explanation of "UTC"... + *

+ *
+ * "In 1970 the Coordinated Universal Time system was devised by an + * international advisory group of technical experts within the International + * Telecommunication Union (ITU). The ITU felt it was best to designate a + * single abbreviation for use in all languages in order to minimize + * confusion. Since unanimous agreement could not be achieved on using + * either the English word order, CUT, or the French word order, TUC, the + * acronym UTC was chosen as a compromise." + *
+ * + * @see TrieDateField + * @see PointField + */ public class DatePointField extends PointField implements DateValueFieldType { public DatePointField() { From 725cd4e2f546a71ccf43218ffc88739a3e05a260 Mon Sep 17 00:00:00 2001 From: yonik Date: Wed, 22 Mar 2017 19:53:50 -0400 Subject: [PATCH 050/563] SOLR-7452: facet refinement - don't generate domain if skipping bucket --- .../org/apache/solr/search/facet/FacetFieldProcessor.java | 6 +++--- .../java/org/apache/solr/search/facet/FacetProcessor.java | 6 +----- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java index 1ba252e3a75..e8b234d6ed0 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java @@ -528,9 +528,9 @@ abstract class FacetFieldProcessor extends FacetProcessor { } protected SimpleOrderedMap refineFacets() throws IOException { - List leaves = asList(fcontext.facetInfo.get("_l")); - List skip = asList(fcontext.facetInfo.get("_s")); - List missing = asList(fcontext.facetInfo.get("_m")); + List leaves = asList(fcontext.facetInfo.get("_l")); // We have not seen this bucket: do full faceting for this bucket, including all sub-facets + List skip = asList(fcontext.facetInfo.get("_s")); // We have seen this bucket, so skip stats on it, and skip sub-facets except for the specified sub-facets that should calculate specified buckets. + List missing = asList(fcontext.facetInfo.get("_m")); // We have not seen this bucket, do full faceting for this bucket, and most sub-facets... but some sub-facets should only visit specified buckets. // For leaf refinements, we do full faceting for each leaf bucket. Any sub-facets of these buckets will be fully evaluated. Because of this, we should never // encounter leaf refinements that have sub-facets that return partial results. diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java index cf4d0fe9cb1..9f05d8e56f6 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetProcessor.java @@ -366,13 +366,9 @@ public abstract class FacetProcessor { } } - // TODO: rather than just have a raw "response", perhaps we should model as a bucket object that contains the response plus extra info? void fillBucket(SimpleOrderedMap bucket, Query q, DocSet result, boolean skip, Map facetInfo) throws IOException { - // TODO: we don't need the DocSet if we've already calculated everything during the first phase - boolean needDocSet = freq.getFacetStats().size() > 0 || freq.getSubFacets().size() > 0; - - // TODO: put info in for the merger (like "skip=true"?) Maybe we don't need to if we leave out all extraneous info? + boolean needDocSet = (skip==false && freq.getFacetStats().size() > 0) || freq.getSubFacets().size() > 0; int count; From eb587772ddecaea371b20feb955a197e80699f22 Mon Sep 17 00:00:00 2001 From: Noble Paul Date: Thu, 23 Mar 2017 11:45:50 +1030 Subject: [PATCH 051/563] SOLR-6615: use constants for 'id', '_route_', '_version_' --- .../apache/solr/cloud/ElectionContext.java | 4 ++- .../java/org/apache/solr/cloud/Overseer.java | 8 +++-- .../solr/cloud/OverseerNodePrioritizer.java | 7 ++-- .../solr/cloud/OverseerTaskProcessor.java | 7 ++-- .../org/apache/solr/core/JmxMonitoredMap.java | 4 +-- .../org/apache/solr/handler/BlobHandler.java | 12 ++++--- .../apache/solr/handler/CdcrReplicator.java | 15 ++++---- .../apache/solr/handler/StreamHandler.java | 34 +++++++++++-------- .../handler/admin/LukeRequestHandler.java | 2 +- .../solr/handler/admin/ThreadDumpHandler.java | 5 +-- .../component/QueryElevationComponent.java | 4 ++- .../component/RealTimeGetComponent.java | 16 +++++---- .../solr/handler/loader/JavabinLoader.java | 3 +- .../solr/handler/loader/JsonLoader.java | 10 ++++-- .../apache/solr/handler/loader/XMLLoader.java | 14 ++++---- .../apache/solr/handler/sql/SolrTable.java | 1 - .../reporters/solr/SolrClusterReporter.java | 4 ++- .../TopGroupsResultTransformer.java | 8 +++-- .../solr/search/mlt/CloudMLTQParser.java | 5 ++- .../solr/spelling/SpellCheckCollator.java | 4 ++- .../apache/solr/update/AddUpdateCommand.java | 3 +- .../java/org/apache/solr/update/PeerSync.java | 3 +- .../org/apache/solr/update/VersionInfo.java | 6 ++-- .../processor/AtomicUpdateDocumentMerger.java | 15 ++++---- .../update/processor/CdcrUpdateProcessor.java | 3 +- .../processor/DistributedUpdateProcessor.java | 25 +++++++------- ...sedVersionConstraintsProcessorFactory.java | 4 +-- .../cloud/CdcrVersionReplicationTest.java | 4 +-- .../cloud/FullSolrCloudDistribCmdsTest.java | 18 +++++----- .../solr/update/DirectUpdateHandlerTest.java | 6 ++-- .../update/TestInPlaceUpdatesStandalone.java | 6 ++-- .../org/apache/solr/update/UpdateLogTest.java | 6 ++-- .../client/solrj/impl/CloudSolrClient.java | 3 +- .../solr/client/solrj/io/ModelCache.java | 7 ++-- .../client/solrj/io/stream/DaemonStream.java | 9 +++-- .../solrj/io/stream/ExecutorStream.java | 4 ++- .../io/stream/FeaturesSelectionStream.java | 4 ++- .../client/solrj/io/stream/FetchStream.java | 6 ++-- .../client/solrj/io/stream/ModelStream.java | 6 ++-- .../solrj/io/stream/TextLogitStream.java | 4 ++- .../client/solrj/io/stream/TopicStream.java | 17 ++++++---- .../client/solrj/io/stream/UpdateStream.java | 4 ++- .../request/JavaBinUpdateRequestCodec.java | 5 +-- .../client/solrj/request/UpdateRequest.java | 9 ++--- .../solr/common/ToleratedUpdateError.java | 10 +++--- .../solr/common/cloud/HashBasedRouter.java | 8 +++-- .../solr/common/params/CommonParams.java | 4 +++ .../solr/common/params/ShardParams.java | 24 ++++++------- 48 files changed, 231 insertions(+), 159 deletions(-) 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 223a5395201..6e8dbdabf81 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java +++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java @@ -58,6 +58,8 @@ import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; + public abstract class ElectionContext implements Closeable { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); final String electionPath; @@ -753,7 +755,7 @@ final class OverseerElectionContext extends ElectionContext { log.info("I am going to be the leader {}", id); final String id = leaderSeqPath .substring(leaderSeqPath.lastIndexOf("/") + 1); - ZkNodeProps myProps = new ZkNodeProps("id", id); + ZkNodeProps myProps = new ZkNodeProps(ID, id); zkClient.makePath(leaderPath, Utils.toJSON(myProps), CreateMode.EPHEMERAL, true); diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java index 61f15fca6eb..e0449b4a0f1 100644 --- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java +++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java @@ -55,6 +55,8 @@ import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; + /** * Cluster leader. Responsible for processing state updates, node assignments, creating/deleting * collections, shards, replicas and setting various properties. @@ -292,7 +294,7 @@ public class Overseer implements Closeable { } try { Map m = (Map) Utils.fromJSON(data); - String id = (String) m.get("id"); + String id = (String) m.get(ID); if(overseerCollectionConfigSetProcessor.getId().equals(id)){ try { log.warn("I'm exiting, but I'm still the leader"); @@ -372,7 +374,7 @@ public class Overseer implements Closeable { case UPDATESHARDSTATE: return Collections.singletonList(new SliceMutator(getZkStateReader()).updateShardState(clusterState, message)); case QUIT: - if (myId.equals(message.get("id"))) { + if (myId.equals(message.get(ID))) { log.info("Quit command received {} {}", message, LeaderElector.getNodeName(myId)); overseerCollectionConfigSetProcessor.close(); close(); @@ -396,7 +398,7 @@ public class Overseer implements Closeable { try { ZkNodeProps props = ZkNodeProps.load(zkClient.getData( OVERSEER_ELECT + "/leader", null, null, true)); - if (myId.equals(props.getStr("id"))) { + if (myId.equals(props.getStr(ID))) { return LeaderStatus.YES; } } catch (KeeperException e) { diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerNodePrioritizer.java b/solr/core/src/java/org/apache/solr/cloud/OverseerNodePrioritizer.java index 798eca39950..7db13156638 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerNodePrioritizer.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerNodePrioritizer.java @@ -17,14 +17,13 @@ package org.apache.solr.cloud; import java.lang.invoke.MethodHandles; - import java.util.List; import java.util.Map; +import org.apache.solr.cloud.overseer.OverseerAction; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkStateReader; -import org.apache.solr.cloud.overseer.OverseerAction; import org.apache.solr.common.params.CoreAdminParams; import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction; import org.apache.solr.common.params.ModifiableSolrParams; @@ -37,6 +36,8 @@ import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; + /** * Responsible for prioritization of Overseer nodes, for example with the * ADDROLE collection command. @@ -90,7 +91,7 @@ public class OverseerNodePrioritizer { //now ask the current leader to QUIT , so that the designate can takeover Overseer.getStateUpdateQueue(zkStateReader.getZkClient()).offer( Utils.toJSON(new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.QUIT.toLower(), - "id", OverseerTaskProcessor.getLeaderId(zkStateReader.getZkClient())))); + ID, OverseerTaskProcessor.getLeaderId(zkStateReader.getZkClient())))); } diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java index bed71a63734..4dffb21905e 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java @@ -34,8 +34,8 @@ import com.codahale.metrics.Timer; import com.google.common.collect.ImmutableSet; import org.apache.commons.io.IOUtils; import org.apache.solr.client.solrj.SolrResponse; -import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent; import org.apache.solr.cloud.Overseer.LeaderStatus; +import org.apache.solr.cloud.OverseerTaskQueue.QueueEvent; import org.apache.solr.common.SolrException; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkNodeProps; @@ -50,6 +50,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.solr.common.params.CommonAdminParams.ASYNC; +import static org.apache.solr.common.params.CommonParams.ID; /** * A generic processor run in the Overseer, used for handling items added @@ -375,7 +376,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable { return null; } Map m = (Map) Utils.fromJSON(data); - return (String) m.get("id"); + return (String) m.get(ID); } protected LeaderStatus amILeader() { @@ -385,7 +386,7 @@ public class OverseerTaskProcessor implements Runnable, Closeable { try { ZkNodeProps props = ZkNodeProps.load(zkStateReader.getZkClient().getData( Overseer.OVERSEER_ELECT + "/leader", null, null, true)); - if (myId.equals(props.getStr("id"))) { + if (myId.equals(props.getStr(ID))) { return LeaderStatus.YES; } } catch (KeeperException e) { diff --git a/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java b/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java index 8bfa662d548..4fb0dcd4d63 100644 --- a/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java +++ b/solr/core/src/java/org/apache/solr/core/JmxMonitoredMap.java @@ -38,7 +38,6 @@ import javax.management.openmbean.SimpleType; import javax.management.remote.JMXConnectorServer; import javax.management.remote.JMXConnectorServerFactory; import javax.management.remote.JMXServiceURL; - import java.lang.invoke.MethodHandles; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -58,6 +57,7 @@ import org.apache.solr.metrics.reporters.JmxObjectNameFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; import static org.apache.solr.common.params.CommonParams.NAME; /** @@ -269,7 +269,7 @@ public class JmxMonitoredMap extends Hashtable map = new Hashtable<>(); map.put("type", key); if (infoBean.getName() != null && !"".equals(infoBean.getName())) { - map.put("id", infoBean.getName()); + map.put(ID, infoBean.getName()); } return ObjectName.getInstance(jmxRootName, map); } diff --git a/solr/core/src/java/org/apache/solr/handler/BlobHandler.java b/solr/core/src/java/org/apache/solr/handler/BlobHandler.java index 177af9ec0c4..7e97f5953f4 100644 --- a/solr/core/src/java/org/apache/solr/handler/BlobHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/BlobHandler.java @@ -64,7 +64,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static java.util.Collections.singletonMap; +import static org.apache.solr.common.params.CommonParams.ID; import static org.apache.solr.common.params.CommonParams.JSON; +import static org.apache.solr.common.params.CommonParams.VERSION; import static org.apache.solr.common.util.Utils.makeMap; public class BlobHandler extends RequestHandlerBase implements PluginInfoInitialized { @@ -131,15 +133,15 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial version++; String id = blobName + "/" + version; Map doc = makeMap( - "id", id, + ID, id, "md5", md5, "blobName", blobName, - "version", version, + VERSION, version, "timestamp", new Date(), "size", payload.limit(), "blob", payload); verifyWithRealtimeGet(blobName, version, req, doc); - log.info(StrUtils.formatString("inserting new blob {0} ,size {1}, md5 {2}", doc.get("id"), String.valueOf(payload.limit()), md5)); + log.info(StrUtils.formatString("inserting new blob {0} ,size {1}, md5 {2}", doc.get(ID), String.valueOf(payload.limit()), md5)); indexMap(req, rsp, doc); log.info(" Successfully Added and committed a blob with id {} and size {} ", id, payload.limit()); @@ -212,7 +214,7 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial for (; ; ) { SolrQueryResponse response = new SolrQueryResponse(); String id = blobName + "/" + version; - forward(req, "/get", new MapSolrParams(singletonMap("id", id)), response); + forward(req, "/get", new MapSolrParams(singletonMap(ID, id)), response); if (response.getValues().get("doc") == null) { //ensure that the version does not exist return; @@ -221,7 +223,7 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial version++; doc.put("version", version); id = blobName + "/" + version; - doc.put("id", id); + doc.put(ID, id); } } diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java index 8519815f418..75a787b8804 100644 --- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java +++ b/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java @@ -16,6 +16,11 @@ */ package org.apache.solr.handler; +import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.nio.charset.Charset; +import java.util.List; + import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.request.UpdateRequest; @@ -25,14 +30,10 @@ import org.apache.solr.common.SolrInputDocument; import org.apache.solr.update.CdcrUpdateLog; import org.apache.solr.update.UpdateLog; import org.apache.solr.update.processor.CdcrUpdateProcessor; -import org.apache.solr.update.processor.DistributedUpdateProcessor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.nio.charset.Charset; -import java.util.List; +import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; /** * The replication logic. Given a {@link org.apache.solr.handler.CdcrReplicatorState}, it reads all the new entries @@ -183,14 +184,14 @@ public class CdcrReplicator implements Runnable { case UpdateLog.DELETE: { byte[] idBytes = (byte[]) entry.get(2); req.deleteById(new String(idBytes, Charset.forName("UTF-8"))); - req.setParam(DistributedUpdateProcessor.VERSION_FIELD, Long.toString(version)); + req.setParam(VERSION_FIELD, Long.toString(version)); return req; } case UpdateLog.DELETE_BY_QUERY: { String query = (String) entry.get(2); req.deleteByQuery(query); - req.setParam(DistributedUpdateProcessor.VERSION_FIELD, Long.toString(version)); + req.setParam(VERSION_FIELD, Long.toString(version)); return req; } diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java index dfae5cd69e3..a25ede46e17 100644 --- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java @@ -80,6 +80,8 @@ import org.apache.solr.util.plugin.SolrCoreAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; + public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, PermissionNameProvider { static SolrClientCache clientCache = new SolrClientCache(); @@ -284,7 +286,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, private void handleAdmin(SolrQueryRequest req, SolrQueryResponse rsp, SolrParams params) { String action = params.get("action"); if("stop".equalsIgnoreCase(action)) { - String id = params.get("id"); + String id = params.get(ID); DaemonStream d = daemons.get(id); if(d != null) { d.close(); @@ -292,21 +294,23 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, } else { rsp.add("result-set", new DaemonResponseStream("Deamon:" + id + " not found on " + coreName)); } - } else if("start".equalsIgnoreCase(action)) { - String id = params.get("id"); - DaemonStream d = daemons.get(id); - d.open(); - rsp.add("result-set", new DaemonResponseStream("Deamon:" + id + " started on " + coreName)); - } else if("list".equalsIgnoreCase(action)) { - Collection vals = daemons.values(); - rsp.add("result-set", new DaemonCollectionStream(vals)); - } else if("kill".equalsIgnoreCase(action)) { - String id = params.get("id"); - DaemonStream d = daemons.remove(id); - if (d != null) { - d.close(); + } else { + if ("start".equalsIgnoreCase(action)) { + String id = params.get(ID); + DaemonStream d = daemons.get(id); + d.open(); + rsp.add("result-set", new DaemonResponseStream("Deamon:" + id + " started on " + coreName)); + } else if ("list".equalsIgnoreCase(action)) { + Collection vals = daemons.values(); + rsp.add("result-set", new DaemonCollectionStream(vals)); + } else if ("kill".equalsIgnoreCase(action)) { + String id = params.get("id"); + DaemonStream d = daemons.remove(id); + if (d != null) { + d.close(); + } + rsp.add("result-set", new DaemonResponseStream("Deamon:" + id + " killed on " + coreName)); } - rsp.add("result-set", new DaemonResponseStream("Deamon:" + id + " killed on " + coreName)); } } diff --git a/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java index 62a50e624a9..8e0b1fb8cc7 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/LukeRequestHandler.java @@ -102,7 +102,7 @@ public class LukeRequestHandler extends RequestHandlerBase public static final String NUMTERMS = "numTerms"; public static final String INCLUDE_INDEX_FIELD_FLAGS = "includeIndexFieldFlags"; public static final String DOC_ID = "docId"; - public static final String ID = "id"; + public static final String ID = CommonParams.ID; public static final int DEFAULT_COUNT = 10; static final int HIST_ARRAY_SIZE = 33; diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ThreadDumpHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ThreadDumpHandler.java index f0e397067ae..bb5b3ee0f5f 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/ThreadDumpHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/ThreadDumpHandler.java @@ -28,6 +28,7 @@ import org.apache.solr.handler.RequestHandlerBase; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.response.SolrQueryResponse; +import static org.apache.solr.common.params.CommonParams.ID; import static org.apache.solr.common.params.CommonParams.NAME; /** @@ -85,7 +86,7 @@ public class ThreadDumpHandler extends RequestHandlerBase SimpleOrderedMap info = new SimpleOrderedMap<>(); long tid = ti.getThreadId(); - info.add( "id", tid ); + info.add( ID, tid ); info.add(NAME, ti.getThreadName()); info.add( "state", ti.getThreadState().toString() ); @@ -107,7 +108,7 @@ public class ThreadDumpHandler extends RequestHandlerBase if (ti.getLockOwnerName() != null) { SimpleOrderedMap owner = new SimpleOrderedMap<>(); owner.add(NAME, ti.getLockOwnerName()); - owner.add( "id", ti.getLockOwnerId() ); + owner.add( ID, ti.getLockOwnerId() ); } // Add the stack trace diff --git a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java index 8482d65cc06..c12902e83b5 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/QueryElevationComponent.java @@ -92,6 +92,8 @@ import org.w3c.dom.Node; import org.w3c.dom.NodeList; import org.xml.sax.InputSource; +import static org.apache.solr.common.params.CommonParams.ID; + /** * A component to elevate some documents to the top of the result set. * @@ -308,7 +310,7 @@ public class QueryElevationComponent extends SearchComponent implements SolrCore ArrayList exclude = new ArrayList<>(); for (int j = 0; j < children.getLength(); j++) { Node child = children.item(j); - String id = DOMUtil.getAttr(child, "id", "missing 'id'"); + String id = DOMUtil.getAttr(child, ID, "missing 'id'"); String e = DOMUtil.getAttr(child, EXCLUDE, null); if (e != null) { if (Boolean.valueOf(e)) { diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java index 900c787e406..12aa4033d14 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java @@ -78,11 +78,13 @@ import org.apache.solr.update.DocumentBuilder; import org.apache.solr.update.IndexFingerprint; import org.apache.solr.update.PeerSync; import org.apache.solr.update.UpdateLog; -import org.apache.solr.update.processor.DistributedUpdateProcessor; import org.apache.solr.util.RefCounted; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; +import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; + public class RealTimeGetComponent extends SearchComponent { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @@ -473,8 +475,8 @@ public class RealTimeGetComponent extends SearchComponent doc = toSolrDoc(luceneDocument, core.getLatestSchema()); searcher.decorateDocValueFields(doc, docid, decorateFields); - long docVersion = (long) doc.getFirstValue(DistributedUpdateProcessor.VERSION_FIELD); - Object partialVersionObj = partialDoc.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD); + long docVersion = (long) doc.getFirstValue(VERSION_FIELD); + Object partialVersionObj = partialDoc.getFieldValue(VERSION_FIELD); long partialDocVersion = partialVersionObj instanceof Field? ((Field) partialVersionObj).numericValue().longValue(): partialVersionObj instanceof Number? ((Number) partialVersionObj).longValue(): Long.parseLong(partialVersionObj.toString()); if (docVersion > partialDocVersion) { @@ -621,8 +623,8 @@ public class RealTimeGetComponent extends SearchComponent } if (versionReturned != null) { - if (sid.containsKey(DistributedUpdateProcessor.VERSION_FIELD)) { - versionReturned.set((long)sid.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD)); + if (sid.containsKey(VERSION_FIELD)) { + versionReturned.set((long)sid.getFieldValue(VERSION_FIELD)); } } return sid; @@ -841,7 +843,7 @@ public class RealTimeGetComponent extends SearchComponent sreq.params.set("distrib",false); sreq.params.remove(ShardParams.SHARDS); - sreq.params.remove("id"); + sreq.params.remove(ID); sreq.params.remove("ids"); sreq.params.set("ids", StrUtils.join(ids, ',')); @@ -1146,7 +1148,7 @@ public class RealTimeGetComponent extends SearchComponent return (IdsRequsted)req.getContext().get(contextKey); } final SolrParams params = req.getParams(); - final String id[] = params.getParams("id"); + final String id[] = params.getParams(ID); final String ids[] = params.getParams("ids"); if (id == null && ids == null) { diff --git a/solr/core/src/java/org/apache/solr/handler/loader/JavabinLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/JavabinLoader.java index 873bcd15aa5..870737a4a4d 100644 --- a/solr/core/src/java/org/apache/solr/handler/loader/JavabinLoader.java +++ b/solr/core/src/java/org/apache/solr/handler/loader/JavabinLoader.java @@ -20,6 +20,7 @@ import org.apache.solr.client.solrj.request.JavaBinUpdateRequestCodec; import org.apache.solr.client.solrj.request.UpdateRequest; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.params.UpdateParams; import org.apache.solr.common.util.ContentStream; @@ -140,7 +141,7 @@ public class JavabinLoader extends ContentStreamLoader { } } if (map != null) { - String route = (String) map.get(UpdateRequest.ROUTE); + String route = (String) map.get(ShardParams._ROUTE_); if (route != null) { delcmd.setRoute(route); } diff --git a/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java index 8650ab82724..b93d5ef18bb 100644 --- a/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java +++ b/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java @@ -15,6 +15,7 @@ * limitations under the License. */ package org.apache.solr.handler.loader; + import java.io.IOException; import java.io.Reader; import java.io.StringReader; @@ -57,8 +58,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static java.util.stream.Collectors.toList; +import static org.apache.solr.common.params.CommonParams.ID; import static org.apache.solr.common.params.CommonParams.JSON; import static org.apache.solr.common.params.CommonParams.PATH; +import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; +import static org.apache.solr.common.params.ShardParams._ROUTE_; /** @@ -364,15 +368,15 @@ public class JsonLoader extends ContentStreamLoader { if (ev == JSONParser.STRING) { String key = parser.getString(); if (parser.wasKey()) { - if ("id".equals(key)) { + if (ID.equals(key)) { cmd.setId(getString(parser.nextEvent())); } else if ("query".equals(key)) { cmd.setQuery(parser.getString()); } else if ("commitWithin".equals(key)) { cmd.commitWithin = (int) parser.getLong(); - } else if ("_version_".equals(key)) { + } else if (VERSION_FIELD.equals(key)) { cmd.setVersion(parser.getLong()); - } else if ("_route_".equals(key)) { + } else if (_ROUTE_.equals(key)) { cmd.setRoute(parser.getString()); } else { throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown key '" + key + "' at [" + parser.getPosition() + "]"); diff --git a/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java index 038ed9f3e66..000edeed4d7 100644 --- a/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java +++ b/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java @@ -15,6 +15,7 @@ * limitations under the License. */ package org.apache.solr.handler.loader; + import javax.xml.parsers.SAXParserFactory; import javax.xml.stream.FactoryConfigurationError; import javax.xml.stream.XMLInputFactory; @@ -39,11 +40,12 @@ import java.util.concurrent.atomic.AtomicBoolean; import com.google.common.collect.Lists; import org.apache.commons.io.IOUtils; -import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.common.EmptyEntityResolver; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.params.UpdateParams; import org.apache.solr.common.util.ContentStream; @@ -60,13 +62,13 @@ import org.apache.solr.update.CommitUpdateCommand; import org.apache.solr.update.DeleteUpdateCommand; import org.apache.solr.update.RollbackUpdateCommand; import org.apache.solr.update.processor.UpdateRequestProcessor; -import org.apache.solr.common.EmptyEntityResolver; import org.apache.solr.util.xslt.TransformerProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.xml.sax.InputSource; import org.xml.sax.XMLReader; +import static org.apache.solr.common.params.CommonParams.ID; import static org.apache.solr.common.params.CommonParams.NAME; @@ -318,7 +320,7 @@ public class XMLLoader extends ContentStreamLoader { switch (event) { case XMLStreamConstants.START_ELEMENT: String mode = parser.getLocalName(); - if (!("id".equals(mode) || "query".equals(mode))) { + if (!(ID.equals(mode) || "query".equals(mode))) { String msg = "XML element has invalid XML child element: " + mode; log.warn(msg); throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, @@ -326,14 +328,14 @@ public class XMLLoader extends ContentStreamLoader { } text.setLength(0); - if ("id".equals(mode)) { + if (ID.equals(mode)) { for (int i = 0; i < parser.getAttributeCount(); i++) { String attrName = parser.getAttributeLocalName(i); String attrVal = parser.getAttributeValue(i); if (UpdateRequestHandler.VERSION.equals(attrName)) { deleteCmd.setVersion(Long.parseLong(attrVal)); } - if (UpdateRequest.ROUTE.equals(attrName)) { + if (ShardParams._ROUTE_.equals(attrName)) { deleteCmd.setRoute(attrVal); } } @@ -342,7 +344,7 @@ public class XMLLoader extends ContentStreamLoader { case XMLStreamConstants.END_ELEMENT: String currTag = parser.getLocalName(); - if ("id".equals(currTag)) { + if (ID.equals(currTag)) { deleteCmd.setId(text.toString()); } else if ("query".equals(currTag)) { deleteCmd.setQuery(text.toString()); 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 644ed97449a..c3b83dbdd83 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 @@ -61,7 +61,6 @@ import java.util.stream.Collectors; */ class SolrTable extends AbstractQueryableTable implements TranslatableTable { private static final String DEFAULT_QUERY = "*:*"; - private static final String DEFAULT_VERSION_FIELD = "_version_"; private final String collection; private final SolrSchema schema; 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 0c3b651a573..a34accd82aa 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 @@ -41,6 +41,8 @@ import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; + /** * This reporter sends selected metrics from local registries to {@link Overseer}. *

The following configuration properties are supported:

@@ -253,7 +255,7 @@ public class SolrClusterReporter extends SolrMetricReporter { if (props == null) { return lastKnownUrl; } - String oid = props.getStr("id"); + String oid = props.getStr(ID); if (oid == null) { return lastKnownUrl; } diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java index 415e513d8a8..adb81de2c1d 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java @@ -47,6 +47,8 @@ import org.apache.solr.search.grouping.distributed.command.TopGroupsFieldCommand import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; + /** * Implementation for transforming {@link TopGroups} and {@link TopDocs} into a {@link NamedList} structure and * visa versa. @@ -158,7 +160,7 @@ public class TopGroupsResultTransformer implements ShardResultTransformer document : documents) { - Object docId = document.get("id"); + Object docId = document.get(ID); if (docId != null) { docId = docId.toString(); } else { @@ -208,7 +210,7 @@ public class TopGroupsResultTransformer implements ShardResultTransformer candidateFields = new HashSet<>(); // if _version_ field is not supported for in-place update, bail out early - SchemaField versionField = schema.getFieldOrNull(DistributedUpdateProcessor.VERSION_FIELD); + SchemaField versionField = schema.getFieldOrNull(CommonParams.VERSION_FIELD); if (versionField == null || !isSupportedFieldForInPlaceUpdate(versionField)) { return Collections.emptySet(); } @@ -169,7 +172,7 @@ public class AtomicUpdateDocumentMerger { // and bail out early if anything is obviously not a valid in-place update for (String fieldName : sdoc.getFieldNames()) { if (fieldName.equals(uniqueKeyFieldName) - || fieldName.equals(DistributedUpdateProcessor.VERSION_FIELD)) { + || fieldName.equals(CommonParams.VERSION_FIELD)) { continue; } Object fieldValue = sdoc.getField(fieldName).getValue(); @@ -245,7 +248,7 @@ public class AtomicUpdateDocumentMerger { SolrInputDocument inputDoc = cmd.getSolrInputDocument(); BytesRef idBytes = cmd.getIndexedId(); - updatedFields.add(DistributedUpdateProcessor.VERSION_FIELD); // add the version field so that it is fetched too + updatedFields.add(CommonParams.VERSION_FIELD); // add the version field so that it is fetched too SolrInputDocument oldDocument = RealTimeGetComponent.getInputDocument (cmd.getReq().getCore(), idBytes, null, // don't want the version to be returned @@ -258,11 +261,11 @@ public class AtomicUpdateDocumentMerger { return false; } - if (oldDocument.containsKey(DistributedUpdateProcessor.VERSION_FIELD) == false) { + if (oldDocument.containsKey(CommonParams.VERSION_FIELD) == false) { throw new SolrException (ErrorCode.INVALID_STATE, "There is no _version_ in previous document. id=" + cmd.getPrintableId()); } - Long oldVersion = (Long) oldDocument.remove(DistributedUpdateProcessor.VERSION_FIELD).getValue(); + Long oldVersion = (Long) oldDocument.remove(CommonParams.VERSION_FIELD).getValue(); // If the oldDocument contains any other field apart from updatedFields (or id/version field), then remove them. // This can happen, despite requesting for these fields in the call to RTGC.getInputDocument, if the document was @@ -270,7 +273,7 @@ public class AtomicUpdateDocumentMerger { if (updatedFields != null) { Collection names = new HashSet(oldDocument.getFieldNames()); for (String fieldName: names) { - if (fieldName.equals(DistributedUpdateProcessor.VERSION_FIELD)==false && fieldName.equals("id")==false && updatedFields.contains(fieldName)==false) { + if (fieldName.equals(CommonParams.VERSION_FIELD)==false && fieldName.equals(ID)==false && updatedFields.contains(fieldName)==false) { oldDocument.remove(fieldName); } } diff --git a/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessor.java index 5bbc4a2a845..ee454670767 100644 --- a/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessor.java +++ b/solr/core/src/java/org/apache/solr/update/processor/CdcrUpdateProcessor.java @@ -19,6 +19,7 @@ package org.apache.solr.update.processor; import java.io.IOException; import java.lang.invoke.MethodHandles; +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.request.SolrQueryRequest; @@ -101,7 +102,7 @@ public class CdcrUpdateProcessor extends DistributedUpdateProcessor { // } else { // log.info("+++ cdcr.update version present, params are: " + params); // } - result.set(DistributedUpdateProcessor.VERSION_FIELD, params.get(DistributedUpdateProcessor.VERSION_FIELD)); + result.set(CommonParams.VERSION_FIELD, params.get(CommonParams.VERSION_FIELD)); } return result; 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 08ede724269..fe71b0b805f 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 @@ -64,6 +64,7 @@ import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZooKeeperException; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.params.SolrParams; @@ -240,8 +241,6 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { private final UpdateRequestProcessor next; private final AtomicUpdateDocumentMerger docMerger; - public static final String VERSION_FIELD = "_version_"; - private final UpdateHandler updateHandler; private final UpdateLog ulog; private final VersionInfo vinfo; @@ -310,7 +309,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { // this should always be used - see filterParams DistributedUpdateProcessorFactory.addParamToDistributedRequestWhitelist - (this.req, UpdateParams.UPDATE_CHAIN, TEST_DISTRIB_SKIP_SERVERS, VERSION_FIELD); + (this.req, UpdateParams.UPDATE_CHAIN, TEST_DISTRIB_SKIP_SERVERS, CommonParams.VERSION_FIELD); CoreDescriptor coreDesc = req.getCore().getCoreDescriptor(); @@ -1031,13 +1030,13 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { long versionOnUpdate = cmd.getVersion(); if (versionOnUpdate == 0) { - SolrInputField versionField = cmd.getSolrInputDocument().getField(VersionInfo.VERSION_FIELD); + SolrInputField versionField = cmd.getSolrInputDocument().getField(CommonParams.VERSION_FIELD); if (versionField != null) { Object o = versionField.getValue(); versionOnUpdate = o instanceof Number ? ((Number) o).longValue() : Long.parseLong(o.toString()); } else { // Find the version - String versionOnUpdateS = req.getParams().get(VERSION_FIELD); + String versionOnUpdateS = req.getParams().get(CommonParams.VERSION_FIELD); versionOnUpdate = versionOnUpdateS == null ? 0 : Long.parseLong(versionOnUpdateS); } } @@ -1084,7 +1083,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { // forwarded from a collection but we are not buffering so strip original version and apply our own // see SOLR-5308 log.info("Removing version field from doc: " + cmd.getPrintableId()); - cmd.solrDoc.remove(VERSION_FIELD); + cmd.solrDoc.remove(CommonParams.VERSION_FIELD); versionOnUpdate = 0; } @@ -1114,7 +1113,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { long version = vinfo.getNewClock(); cmd.setVersion(version); - cmd.getSolrInputDocument().setField(VersionInfo.VERSION_FIELD, version); + cmd.getSolrInputDocument().setField(CommonParams.VERSION_FIELD, version); bucket.updateHighest(version); } else { // The leader forwarded us this update. @@ -1152,7 +1151,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { // Make this update to become a non-inplace update containing the full document obtained from the leader cmd.solrDoc = ((AddUpdateCommand)fetchedFromLeader).solrDoc; cmd.prevVersion = -1; - cmd.setVersion((long)cmd.solrDoc.getFieldValue(VERSION_FIELD)); + cmd.setVersion((long)cmd.solrDoc.getFieldValue(CommonParams.VERSION_FIELD)); assert cmd.isInPlaceUpdate() == false; } } else { @@ -1354,7 +1353,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { AddUpdateCommand cmd = new AddUpdateCommand(req); cmd.solrDoc = leaderDoc; - cmd.setVersion((long)leaderDoc.getFieldValue(VERSION_FIELD)); + cmd.setVersion((long)leaderDoc.getFieldValue(CommonParams.VERSION_FIELD)); return cmd; } @@ -1386,7 +1385,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { throw new SolrException(ErrorCode.CONFLICT, "Document not found for update. id=" + cmd.getPrintableId()); } } else { - oldDoc.remove(VERSION_FIELD); + oldDoc.remove(CommonParams.VERSION_FIELD); } @@ -1598,7 +1597,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { if (zkEnabled) { // forward to all replicas ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams())); - params.set(VERSION_FIELD, Long.toString(cmd.getVersion())); + params.set(CommonParams.VERSION_FIELD, Long.toString(cmd.getVersion())); params.set(DISTRIB_UPDATE_PARAM, DistribPhase.FROMLEADER.toString()); params.set(DISTRIB_FROM, ZkCoreNodeProps.getCoreUrl( zkController.getBaseUrl(), req.getCore().getName())); @@ -1667,7 +1666,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { // Find the version long versionOnUpdate = cmd.getVersion(); if (versionOnUpdate == 0) { - String versionOnUpdateS = req.getParams().get(VERSION_FIELD); + String versionOnUpdateS = req.getParams().get(CommonParams.VERSION_FIELD); versionOnUpdate = versionOnUpdateS == null ? 0 : Long.parseLong(versionOnUpdateS); } versionOnUpdate = Math.abs(versionOnUpdate); // normalize to positive version @@ -1776,7 +1775,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { // Find the version long versionOnUpdate = cmd.getVersion(); if (versionOnUpdate == 0) { - String versionOnUpdateS = req.getParams().get(VERSION_FIELD); + String versionOnUpdateS = req.getParams().get(CommonParams.VERSION_FIELD); versionOnUpdate = versionOnUpdateS == null ? 0 : Long.parseLong(versionOnUpdateS); } long signedVersionOnUpdate = versionOnUpdate; diff --git a/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java index b089c94523f..a8b331c8429 100644 --- a/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java +++ b/solr/core/src/java/org/apache/solr/update/processor/DocBasedVersionConstraintsProcessorFactory.java @@ -22,6 +22,7 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.core.SolrCore; import org.apache.solr.handler.component.RealTimeGetComponent; @@ -33,7 +34,6 @@ import org.apache.solr.search.SolrIndexSearcher; import org.apache.solr.update.AddUpdateCommand; import org.apache.solr.update.DeleteUpdateCommand; import org.apache.solr.update.UpdateCommand; -import org.apache.solr.update.VersionInfo; import org.apache.solr.util.RefCounted; import org.apache.solr.util.plugin.SolrCoreAware; import org.slf4j.Logger; @@ -202,7 +202,7 @@ public class DocBasedVersionConstraintsProcessorFactory extends UpdateRequestPro this.core = req.getCore(); this.versionFieldName = versionField; this.userVersionField = core.getLatestSchema().getField(versionField); - this.solrVersionField = core.getLatestSchema().getField(VersionInfo.VERSION_FIELD); + this.solrVersionField = core.getLatestSchema().getField(CommonParams.VERSION_FIELD); this.useFieldCache = useFieldCache; for (UpdateRequestProcessor proc = next ;proc != null; proc = proc.next) { diff --git a/solr/core/src/test/org/apache/solr/cloud/CdcrVersionReplicationTest.java b/solr/core/src/test/org/apache/solr/cloud/CdcrVersionReplicationTest.java index 367bbafc9ef..59d3818b3d5 100644 --- a/solr/core/src/test/org/apache/solr/cloud/CdcrVersionReplicationTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/CdcrVersionReplicationTest.java @@ -23,9 +23,9 @@ 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.SolrException; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.util.StrUtils; import org.apache.solr.update.processor.CdcrUpdateProcessor; -import org.apache.solr.update.processor.DistributedUpdateProcessor; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,7 +40,7 @@ public class CdcrVersionReplicationTest extends BaseCdcrDistributedZkTest { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private static final String vfield = DistributedUpdateProcessor.VERSION_FIELD; + private static final String vfield = CommonParams.VERSION_FIELD; SolrClient solrServer; public CdcrVersionReplicationTest() { diff --git a/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java b/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java index e15ab0d4cba..7f3ab9691c4 100644 --- a/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/FullSolrCloudDistribCmdsTest.java @@ -16,6 +16,11 @@ */ package org.apache.solr.cloud; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + import org.apache.lucene.util.LuceneTestCase.Slow; import org.apache.solr.SolrTestCaseJ4.SuppressSSL; import org.apache.solr.client.solrj.SolrClient; @@ -38,16 +43,11 @@ import org.apache.solr.common.params.CollectionParams.CollectionAction; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.Utils; -import org.apache.solr.update.VersionInfo; -import org.apache.solr.update.processor.DistributedUpdateProcessor; import org.apache.zookeeper.CreateMode; import org.junit.BeforeClass; import org.junit.Test; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; +import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; /** * Super basic testing, no shard restarting or anything. @@ -718,12 +718,12 @@ public class FullSolrCloudDistribCmdsTest extends AbstractFullDistribZkTestBase private void testOptimisticUpdate(QueryResponse results) throws Exception { SolrDocument doc = results.getResults().get(0); - Long version = (Long) doc.getFieldValue(VersionInfo.VERSION_FIELD); + Long version = (Long) doc.getFieldValue(VERSION_FIELD); Integer theDoc = (Integer) doc.getFieldValue("id"); UpdateRequest uReq = new UpdateRequest(); SolrInputDocument doc1 = new SolrInputDocument(); uReq.setParams(new ModifiableSolrParams()); - uReq.getParams().set(DistributedUpdateProcessor.VERSION_FIELD, Long.toString(version)); + uReq.getParams().set(VERSION_FIELD, Long.toString(version)); addFields(doc1, "id", theDoc, t1, "theupdatestuff"); uReq.add(doc1); @@ -736,7 +736,7 @@ public class FullSolrCloudDistribCmdsTest extends AbstractFullDistribZkTestBase SolrInputDocument doc2 = new SolrInputDocument(); uReq = new UpdateRequest(); uReq.setParams(new ModifiableSolrParams()); - uReq.getParams().set(DistributedUpdateProcessor.VERSION_FIELD, Long.toString(version)); + uReq.getParams().set(VERSION_FIELD, Long.toString(version)); addFields(doc2, "id", theDoc, t1, "thenewupdatestuff"); uReq.add(doc2); 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 462241a3902..7d97ee4240c 100644 --- a/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java @@ -26,8 +26,8 @@ import java.util.concurrent.atomic.AtomicLong; import com.codahale.metrics.Gauge; import com.codahale.metrics.Meter; import com.codahale.metrics.Metric; -import org.apache.lucene.index.TieredMergePolicy; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.TieredMergePolicy; import org.apache.lucene.store.Directory; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.params.CommonParams; @@ -46,6 +46,8 @@ import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; + /** * * @@ -128,7 +130,7 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 { assertNull("This test requires a schema that has no version field, " + "it appears the schema file in use has been edited to violate " + "this requirement", - h.getCore().getLatestSchema().getFieldOrNull(VersionInfo.VERSION_FIELD)); + h.getCore().getLatestSchema().getFieldOrNull(VERSION_FIELD)); assertU(adoc("id","5")); assertU(adoc("id","6")); diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java index 877467e1e5a..aa075a46cb1 100644 --- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java +++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesStandalone.java @@ -43,9 +43,9 @@ import org.apache.solr.common.SolrDocumentList; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.SolrInputField; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.index.NoMergePolicyFactory; import org.apache.solr.request.SolrQueryRequest; -import org.apache.solr.update.processor.DistributedUpdateProcessor; import org.apache.solr.schema.IndexSchema; import org.apache.solr.schema.SchemaField; import org.apache.solr.search.SolrIndexSearcher; @@ -1119,8 +1119,8 @@ public class TestInPlaceUpdatesStandalone extends SolrTestCaseJ4 { try (SolrQueryRequest req = req()) { AddUpdateCommand cmd = new AddUpdateCommand(req); cmd.solrDoc = sdoc; - assertTrue(cmd.solrDoc.containsKey(DistributedUpdateProcessor.VERSION_FIELD)); - cmd.setVersion(Long.parseLong(cmd.solrDoc.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD).toString())); + assertTrue(cmd.solrDoc.containsKey(CommonParams.VERSION_FIELD)); + cmd.setVersion(Long.parseLong(cmd.solrDoc.getFieldValue(CommonParams.VERSION_FIELD).toString())); return AtomicUpdateDocumentMerger.computeInPlaceUpdatableFields(cmd); } } diff --git a/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java b/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java index 8abfe2ab285..9e7d977a03d 100644 --- a/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java +++ b/solr/core/src/test/org/apache/solr/update/UpdateLogTest.java @@ -26,11 +26,11 @@ import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.handler.component.RealTimeGetComponent; import org.apache.solr.request.SolrQueryRequest; -import org.apache.solr.update.processor.DistributedUpdateProcessor; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; +import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; import static org.junit.internal.matchers.StringContains.containsString; public class UpdateLogTest extends SolrTestCaseJ4 { @@ -265,8 +265,8 @@ public class UpdateLogTest extends SolrTestCaseJ4 { public static AddUpdateCommand buildAddUpdateCommand(final SolrQueryRequest req, final SolrInputDocument sdoc) { AddUpdateCommand cmd = new AddUpdateCommand(req); cmd.solrDoc = sdoc; - assertTrue("", cmd.solrDoc.containsKey(DistributedUpdateProcessor.VERSION_FIELD)); - cmd.setVersion(Long.parseLong(cmd.solrDoc.getFieldValue(DistributedUpdateProcessor.VERSION_FIELD).toString())); + assertTrue("", cmd.solrDoc.containsKey(VERSION_FIELD)); + cmd.setVersion(Long.parseLong(cmd.solrDoc.getFieldValue(VERSION_FIELD).toString())); return cmd; } } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java index 6941a77663d..83c6326af4b 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java @@ -89,6 +89,7 @@ import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH; import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH; import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH; import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH; +import static org.apache.solr.common.params.CommonParams.ID; /** * SolrJ client class to communicate with SolrCloud. @@ -121,7 +122,7 @@ public class CloudSolrClient extends SolrClient { private ExecutorService threadPool = ExecutorUtil .newMDCAwareCachedThreadPool(new SolrjNamedThreadFactory( "CloudSolrClient ThreadPool")); - private String idField = "id"; + private String idField = ID; public static final String STATE_VERSION = "_stateVer_"; private long retryExpiryTime = TimeUnit.NANOSECONDS.convert(3, TimeUnit.SECONDS);//3 seconds or 3 million nanos private final Set NON_ROUTABLE_PARAMS; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java index 521ffec7dac..9a204e26808 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java @@ -20,10 +20,13 @@ import java.io.IOException; import java.util.Date; import java.util.LinkedHashMap; import java.util.Map; + import org.apache.solr.client.solrj.io.stream.CloudSolrStream; import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.common.params.ModifiableSolrParams; +import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; + /** * The Model cache keeps a local in-memory copy of models @@ -92,8 +95,8 @@ public class ModelCache { Model m = models.get(modelID); if (m != null) { Tuple t = m.getTuple(); - long v = t.getLong("_version_"); - if (v >= tuple.getLong("_version_")) { + long v = t.getLong(VERSION_FIELD); + if (v >= tuple.getLong(VERSION_FIELD)) { return t; } else { models.put(modelID, new Model(tuple, currentTime)); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DaemonStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DaemonStream.java index 8214f9a3805..2d8aa340384 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DaemonStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/DaemonStream.java @@ -15,6 +15,7 @@ * limitations under the License. */ package org.apache.solr.client.solrj.io.stream; + import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.ArrayList; @@ -38,6 +39,8 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; + public class DaemonStream extends TupleStream implements Expressible { @@ -63,7 +66,7 @@ public class DaemonStream extends TupleStream implements Expressible { TupleStream tupleStream = factory.constructStream(streamExpressions.get(0)); - StreamExpressionNamedParameter idExpression = factory.getNamedOperand(expression, "id"); + StreamExpressionNamedParameter idExpression = factory.getNamedOperand(expression, ID); StreamExpressionNamedParameter runExpression = factory.getNamedOperand(expression, "runInterval"); StreamExpressionNamedParameter queueExpression = factory.getNamedOperand(expression, "queueSize"); StreamExpressionNamedParameter terminateExpression = factory.getNamedOperand(expression, "terminate"); @@ -130,7 +133,7 @@ public class DaemonStream extends TupleStream implements Expressible { expression.addParameter(""); } - expression.addParameter(new StreamExpressionNamedParameter("id", id)); + expression.addParameter(new StreamExpressionNamedParameter(ID, id)); expression.addParameter(new StreamExpressionNamedParameter("runInterval", Long.toString(runInterval))); expression.addParameter(new StreamExpressionNamedParameter("queueSize", Integer.toString(queueSize))); expression.addParameter(new StreamExpressionNamedParameter("terminate", Boolean.toString(terminate))); @@ -230,7 +233,7 @@ public class DaemonStream extends TupleStream implements Expressible { public synchronized Tuple getInfo() { Tuple tuple = new Tuple(new HashMap()); - tuple.put("id", id); + tuple.put(ID, id); tuple.put("startTime", startTime); tuple.put("stopTime", stopTime); tuple.put("iterations", iterations); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ExecutorStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ExecutorStream.java index 6765f7271db..e2f5b823214 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ExecutorStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ExecutorStream.java @@ -41,6 +41,8 @@ import org.apache.solr.common.util.SolrjNamedThreadFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.ID; + /** * The executor function wraps a stream with Tuples containing Streaming Expressions * and executes them in parallel. Sample syntax: @@ -197,7 +199,7 @@ public class ExecutorStream extends TupleStream implements Expressible { } String expr = tuple.getString("expr_s"); - Object id = tuple.get("id"); + Object id = tuple.get(ID); TupleStream stream = null; try { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java index cfb3941f8ae..f15e2a7c530 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java @@ -59,6 +59,8 @@ import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SolrjNamedThreadFactory; +import static org.apache.solr.common.params.CommonParams.ID; + public class FeaturesSelectionStream extends TupleStream implements Expressible{ private static final long serialVersionUID = 1; @@ -355,7 +357,7 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{ if (tuples.size() == numTerms) break; index++; Map map = new HashMap(); - map.put("id", featureSet + "_" + index); + map.put(ID, featureSet + "_" + index); map.put("index_i", index); map.put("term_s", termScore.getKey()); map.put("score_f", termScore.getValue()); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java index 463ab4ae5be..55ca51a8277 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java @@ -19,11 +19,11 @@ package org.apache.solr.client.solrj.io.stream; import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.HashMap; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.comp.StreamComparator; @@ -37,6 +37,8 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.apache.solr.common.params.ModifiableSolrParams; +import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; + /** * Iterates over a stream and fetches additional fields from a specified collection. * Fetches are done in batches. @@ -139,7 +141,7 @@ public class FetchStream extends TupleStream implements Expressible { for(int i=0; i checkpoints = new HashMap(); + private Map checkpoints = new HashMap<>(); private String checkpointCollection; private long initialCheckpoint = -1; @@ -143,7 +146,7 @@ public class TopicStream extends CloudSolrStream implements Expressible { List namedParams = factory.getNamedOperands(expression); StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost"); - StreamExpressionNamedParameter idParam = factory.getNamedOperand(expression, "id"); + StreamExpressionNamedParameter idParam = factory.getNamedOperand(expression, ID); if(null == idParam) { throw new IOException("invalid TopicStream id cannot be null"); } @@ -186,7 +189,7 @@ public class TopicStream extends CloudSolrStream implements Expressible { ModifiableSolrParams params = new ModifiableSolrParams(); for(StreamExpressionNamedParameter namedParam : namedParams){ if(!namedParam.getName().equals("zkHost") && - !namedParam.getName().equals("id") && + !namedParam.getName().equals(ID) && !namedParam.getName().equals("checkpointEvery")) { params.set(namedParam.getName(), namedParam.getParameter().toString().trim()); } @@ -240,7 +243,7 @@ public class TopicStream extends CloudSolrStream implements Expressible { // zkHost expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost)); - expression.addParameter(new StreamExpressionNamedParameter("id", id)); + expression.addParameter(new StreamExpressionNamedParameter(ID, id)); if(initialCheckpoint > -1) { expression.addParameter(new StreamExpressionNamedParameter("initialCheckpoint", Long.toString(initialCheckpoint))); } @@ -391,7 +394,7 @@ public class TopicStream extends CloudSolrStream implements Expressible { persistCheckpoints(); } - long version = tuple.getLong("_version_"); + long version = tuple.getLong(VERSION_FIELD); String slice = tuple.getString("_SLICE_"); checkpoints.put(slice, version); @@ -469,7 +472,7 @@ public class TopicStream extends CloudSolrStream implements Expressible { UpdateRequest request = new UpdateRequest(); request.setParam("collection", checkpointCollection); SolrInputDocument doc = new SolrInputDocument(); - doc.addField("id", id); + doc.addField(ID, id); for(Map.Entry entry : checkpoints.entrySet()) { doc.addField("checkpoint_ss", entry.getKey()+"~"+entry.getValue()); @@ -523,7 +526,7 @@ public class TopicStream extends CloudSolrStream implements Expressible { mParams.set("distrib", "false"); // We are the aggregator. String fl = mParams.get("fl"); mParams.set("sort", "_version_ asc"); - if(!fl.contains("_version_")) { + if(!fl.contains(VERSION_FIELD)) { fl += ",_version_"; } mParams.set("fl", fl); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/UpdateStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/UpdateStream.java index 55291bf9ed9..0b29e4f2932 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/UpdateStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/UpdateStream.java @@ -42,6 +42,8 @@ import org.apache.solr.common.SolrInputDocument; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; + /** * Sends tuples emitted by a wrapped {@link TupleStream} as updates to a SolrCloud collection. */ @@ -268,7 +270,7 @@ public class UpdateStream extends TupleStream implements Expressible { private SolrInputDocument convertTupleToSolrDocument(Tuple tuple) { SolrInputDocument doc = new SolrInputDocument(); for (Object field : tuple.fields.keySet()) { - if (! ((String)field).equals("_version_")) { + if (! field.equals(VERSION_FIELD)) { Object value = tuple.get(field); if (value instanceof List) { addMultivaluedField(doc, (String)field, (List)value); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/JavaBinUpdateRequestCodec.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/JavaBinUpdateRequestCodec.java index 7776a13663e..afa746e96a4 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/JavaBinUpdateRequestCodec.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/JavaBinUpdateRequestCodec.java @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.DataInputInputStream; import org.apache.solr.common.util.JavaBinCodec; @@ -226,8 +227,8 @@ public class JavaBinUpdateRequestCodec { Map params = entry.getValue(); if (params != null) { Long version = (Long) params.get(UpdateRequest.VER); - if (params.containsKey(UpdateRequest.ROUTE)) - updateRequest.deleteById(entry.getKey(), (String) params.get(UpdateRequest.ROUTE)); + if (params.containsKey(ShardParams._ROUTE_)) + updateRequest.deleteById(entry.getKey(), (String) params.get(ShardParams._ROUTE_)); else updateRequest.deleteById(entry.getKey(), version); } else { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java index e7ca0fa32a0..142710a0425 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java @@ -44,6 +44,8 @@ import org.apache.solr.common.params.UpdateParams; import org.apache.solr.common.util.ContentStream; import org.apache.solr.common.util.XML; +import static org.apache.solr.common.params.ShardParams._ROUTE_; + /** * * @@ -54,7 +56,6 @@ public class UpdateRequest extends AbstractUpdateRequest { public static final String REPFACT = "rf"; public static final String MIN_REPFACT = "min_rf"; public static final String VER = "ver"; - public static final String ROUTE = "_route_"; public static final String OVERWRITE = "ow"; public static final String COMMIT_WITHIN = "cw"; private Map> documents = null; @@ -188,7 +189,7 @@ public class UpdateRequest extends AbstractUpdateRequest { if (version != null) params.put(VER, version); if (route != null) - params.put(ROUTE, route); + params.put(_ROUTE_, route); deleteById.put(id, params); return this; } @@ -221,7 +222,7 @@ public class UpdateRequest extends AbstractUpdateRequest { public UpdateRequest withRoute(String route) { if (params == null) params = new ModifiableSolrParams(); - params.set(ROUTE, route); + params.set(_ROUTE_, route); return this; } @@ -461,7 +462,7 @@ public class UpdateRequest extends AbstractUpdateRequest { Map map = entry.getValue(); if (map != null) { Long version = (Long) map.get(VER); - String route = (String)map.get(ROUTE); + String route = (String)map.get(_ROUTE_); if (version != null) { writer.append(" version=\"" + version + "\""); } diff --git a/solr/solrj/src/java/org/apache/solr/common/ToleratedUpdateError.java b/solr/solrj/src/java/org/apache/solr/common/ToleratedUpdateError.java index 55b9b8c7a4d..9c4ac9afa22 100644 --- a/solr/solrj/src/java/org/apache/solr/common/ToleratedUpdateError.java +++ b/solr/solrj/src/java/org/apache/solr/common/ToleratedUpdateError.java @@ -18,9 +18,11 @@ package org.apache.solr.common; import java.util.ArrayList; import java.util.List; -import org.apache.solr.common.util.SimpleOrderedMap; -import org.apache.solr.common.SolrException; + import org.apache.solr.common.SolrException.ErrorCode; +import org.apache.solr.common.util.SimpleOrderedMap; + +import static org.apache.solr.common.params.CommonParams.ID; /** * Models the basic information related to a single "tolerated" error that occured during updates. @@ -74,7 +76,7 @@ public final class ToleratedUpdateError { * @see #getSimpleMap */ public static ToleratedUpdateError parseMap(SimpleOrderedMap data) { - final String id = data.get("id"); + final String id = data.get(ID); final String message = data.get("message"); final String t = data.get("type"); if (null == t || null == id || null == message) { @@ -156,7 +158,7 @@ public final class ToleratedUpdateError { public SimpleOrderedMap getSimpleMap() { SimpleOrderedMap entry = new SimpleOrderedMap(); entry.add("type", type.toString()); - entry.add("id", id); + entry.add(ID, id); entry.add("message", message); return entry; } diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java b/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java index f9ab5d44f51..5e19d38dabc 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/HashBasedRouter.java @@ -16,13 +16,15 @@ */ package org.apache.solr.common.cloud; +import java.util.Collection; +import java.util.Collections; + import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.Hash; -import java.util.Collection; -import java.util.Collections; +import static org.apache.solr.common.params.CommonParams.ID; public abstract class HashBasedRouter extends DocRouter { @@ -51,7 +53,7 @@ public abstract class HashBasedRouter extends DocRouter { } protected String getId(SolrInputDocument sdoc, SolrParams params) { - Object idObj = sdoc.getFieldValue("id"); // blech + Object idObj = sdoc.getFieldValue(ID); // blech String id = idObj != null ? idObj.toString() : "null"; // should only happen on client side return id; } diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java index 7cf27d2ce45..589ef7e7e94 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java @@ -259,5 +259,9 @@ public interface CommonParams { String NAME = "name"; String VALUE_LONG = "val"; + + String VERSION_FIELD="_version_"; + + String ID = "id"; } diff --git a/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java b/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java index 26865737a92..cbc33f41f45 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/ShardParams.java @@ -25,35 +25,35 @@ package org.apache.solr.common.params; */ public interface ShardParams { /** the shards to use (distributed configuration) */ - public static final String SHARDS = "shards"; + String SHARDS = "shards"; /** per-shard start and rows */ - public static final String SHARDS_ROWS = "shards.rows"; - public static final String SHARDS_START = "shards.start"; + String SHARDS_ROWS = "shards.rows"; + String SHARDS_START = "shards.start"; /** IDs of the shard documents */ - public static final String IDS = "ids"; + String IDS = "ids"; /** whether the request goes to a shard */ - public static final String IS_SHARD = "isShard"; + String IS_SHARD = "isShard"; /** The requested URL for this shard */ - public static final String SHARD_URL = "shard.url"; + String SHARD_URL = "shard.url"; /** The Request Handler for shard requests */ - public static final String SHARDS_QT = "shards.qt"; + String SHARDS_QT = "shards.qt"; /** Request detailed match info for each shard (true/false) */ - public static final String SHARDS_INFO = "shards.info"; + String SHARDS_INFO = "shards.info"; /** Should things fail if there is an error? (true/false) */ - public static final String SHARDS_TOLERANT = "shards.tolerant"; + String SHARDS_TOLERANT = "shards.tolerant"; /** query purpose for shard requests */ - public static final String SHARDS_PURPOSE = "shards.purpose"; + String SHARDS_PURPOSE = "shards.purpose"; - public static final String _ROUTE_ = "_route_"; + String _ROUTE_ = "_route_"; /** Force a single-pass distributed query? (true/false) */ - public static final String DISTRIB_SINGLE_PASS = "distrib.singlePass"; + String DISTRIB_SINGLE_PASS = "distrib.singlePass"; } From 583fec1a58b41a0562529e6228a29728a790d87c Mon Sep 17 00:00:00 2001 From: koji Date: Thu, 23 Mar 2017 14:57:45 +0900 Subject: [PATCH 052/563] SOLR-9184: Add a static convenience method ModifiableSolrParams#of(SolrParams) which returns the same instance if it already is modifiable, otherwise creates a new ModifiableSolrParams instance. --- solr/CHANGES.txt | 7 +++++ .../common/params/ModifiableSolrParams.java | 13 ++++++++++ .../params/ModifiableSolrParamsTest.java | 26 +++++++++++++++++++ 3 files changed, 46 insertions(+) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 2bfc981e74e..d6aba02f386 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -116,6 +116,13 @@ New Features * SOLR-9994: Add support for CollapseQParser with PointFields. (Varun Thacker, Cao Manh Dat) +Optimizations +---------------------- + +* SOLR-9184: Add a static convenience method ModifiableSolrParams#of(SolrParams) which returns the same + instance if it already is modifiable, otherwise creates a new ModifiableSolrParams instance. + (Jörg Rathlev via Koji) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/solrj/src/java/org/apache/solr/common/params/ModifiableSolrParams.java b/solr/solrj/src/java/org/apache/solr/common/params/ModifiableSolrParams.java index e3cb499680f..da6bcf15f99 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/ModifiableSolrParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/ModifiableSolrParams.java @@ -54,6 +54,19 @@ public class ModifiableSolrParams extends SolrParams } } + /** + * If the input params are of type MofifiableSolrParams, returns the input, otherwise, constructs a new + * ModifiableSolrParams, copying values from the given params. If params is null, returns an empty + * ModifiableSolrParams instance. + */ + public static ModifiableSolrParams of(SolrParams params) + { + if (params instanceof ModifiableSolrParams) { + return (ModifiableSolrParams) params; + } + return new ModifiableSolrParams(params); + } + public int size() { return vals == null ? 0 : vals.size(); } diff --git a/solr/solrj/src/test/org/apache/solr/common/params/ModifiableSolrParamsTest.java b/solr/solrj/src/test/org/apache/solr/common/params/ModifiableSolrParamsTest.java index b65b607576b..c315b088eca 100644 --- a/solr/solrj/src/test/org/apache/solr/common/params/ModifiableSolrParamsTest.java +++ b/solr/solrj/src/test/org/apache/solr/common/params/ModifiableSolrParamsTest.java @@ -18,6 +18,9 @@ package org.apache.solr.common.params; import org.apache.lucene.util.LuceneTestCase; +import java.util.HashMap; +import java.util.Map; + /** * Unit Test Case for {@link org.apache.solr.common.params.ModifiableSolrParams * ModifiableSolrParams} @@ -39,6 +42,29 @@ public class ModifiableSolrParamsTest extends LuceneTestCase { super.tearDown(); } + public void testOf() throws Exception + { + String key = "key"; + String value = "value"; + + // input is not of type ModifiableSolrParams + Map values = new HashMap<>(); + values.put(key, value); + SolrParams mapParams = new MapSolrParams(values); + ModifiableSolrParams result = ModifiableSolrParams.of(mapParams); + assertNotSame(mapParams, result); + assertEquals(value, result.get(key)); + + // input is of type ModifiableSolrParams + modifiable.add(key, value); + result = ModifiableSolrParams.of(modifiable); + assertSame(result, modifiable); + + // input is null + result = ModifiableSolrParams.of(null); + assertNotNull(result); + assertEquals(0, result.size()); + } public void testAdd() { From 4edfc1a9c7b877a23cffbf5f73a2728173fa663d Mon Sep 17 00:00:00 2001 From: Noble Paul Date: Thu, 23 Mar 2017 18:12:20 +1030 Subject: [PATCH 053/563] SOLR-6615: use constants for 'sort', 'distrib' --- .../solr/response/VelocityResponseWriter.java | 4 ++- .../org/apache/solr/cloud/SyncStrategy.java | 4 ++- .../apache/solr/core/QuerySenderListener.java | 26 ++++++++++--------- .../org/apache/solr/handler/BlobHandler.java | 3 ++- .../org/apache/solr/handler/GraphHandler.java | 4 ++- .../solr/handler/PingRequestHandler.java | 6 +++-- .../apache/solr/handler/StreamHandler.java | 3 ++- .../component/IterativeMergeStrategy.java | 4 ++- .../component/MoreLikeThisComponent.java | 4 ++- .../component/RealTimeGetComponent.java | 3 ++- .../solr/handler/component/SearchHandler.java | 5 ++-- .../apache/solr/handler/sql/SolrTable.java | 11 ++++---- .../solr/index/SortingMergePolicyFactory.java | 7 +++-- .../org/apache/solr/request/SimpleFacets.java | 4 ++- .../apache/solr/request/json/RequestUtil.java | 5 ++-- .../solr/search/CollapsingQParserPlugin.java | 15 ++++++----- .../solr/search/facet/FacetRequest.java | 3 ++- .../apache/solr/search/facet/LegacyFacet.java | 8 +++--- .../java/org/apache/solr/update/PeerSync.java | 11 ++++---- .../processor/DistributedUpdateProcessor.java | 3 ++- .../java/org/apache/solr/util/SolrCLI.java | 3 ++- .../solr/client/solrj/io/ModelCache.java | 3 ++- .../solrj/io/graph/GatherNodesStream.java | 4 ++- .../solrj/io/graph/ShortestPathStream.java | 4 ++- .../client/solrj/io/ops/GroupOperation.java | 6 +++-- .../solrj/io/stream/CloudSolrStream.java | 9 ++++--- .../io/stream/FeaturesSelectionStream.java | 3 ++- .../client/solrj/io/stream/FetchStream.java | 3 ++- .../client/solrj/io/stream/JDBCStream.java | 8 +++--- .../solrj/io/stream/ParallelStream.java | 13 ++++++---- .../client/solrj/io/stream/RandomStream.java | 14 +++++----- .../client/solrj/io/stream/RankStream.java | 6 +++-- .../solrj/io/stream/ScoreNodesStream.java | 6 +++-- .../io/stream/SignificantTermsStream.java | 4 ++- .../solrj/io/stream/TextLogitStream.java | 3 ++- .../client/solrj/io/stream/TopicStream.java | 10 ++++--- .../solr/common/params/TermsParams.java | 12 +++++---- 37 files changed, 151 insertions(+), 93 deletions(-) diff --git a/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java b/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java index 7a0f2a10552..e7cc9b7fb15 100644 --- a/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java +++ b/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java @@ -58,6 +58,8 @@ import org.apache.velocity.tools.generic.SortTool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.SORT; + public class VelocityResponseWriter implements QueryResponseWriter, SolrCoreAware { // init param names, these are _only_ loaded at init time (no per-request control of these) // - multiple different named writers could be created with different init params @@ -204,7 +206,7 @@ public class VelocityResponseWriter implements QueryResponseWriter, SolrCoreAwar context.put("esc", new EscapeTool()); context.put("date", new ComparisonDateTool()); context.put("list", new ListTool()); - context.put("sort", new SortTool()); + context.put(SORT, new SortTool()); MathTool mathTool = new MathTool(); mathTool.configure(toolConfig); diff --git a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java index 6356da77f00..386fef9e5f7 100644 --- a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java +++ b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java @@ -44,6 +44,8 @@ import org.apache.solr.update.UpdateShardHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; + public class SyncStrategy { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); @@ -258,7 +260,7 @@ public class SyncStrategy { sreq.actualShards = sreq.shards; sreq.params = new ModifiableSolrParams(); sreq.params.set("qt","/get"); - sreq.params.set("distrib",false); + sreq.params.set(DISTRIB,false); sreq.params.set("getVersions",Integer.toString(nUpdates)); sreq.params.set("sync",leaderUrl); diff --git a/solr/core/src/java/org/apache/solr/core/QuerySenderListener.java b/solr/core/src/java/org/apache/solr/core/QuerySenderListener.java index a5cda61bdf6..9037ef3122c 100644 --- a/solr/core/src/java/org/apache/solr/core/QuerySenderListener.java +++ b/solr/core/src/java/org/apache/solr/core/QuerySenderListener.java @@ -16,21 +16,23 @@ */ package org.apache.solr.core; -import org.apache.solr.request.SolrQueryRequest; -import org.apache.solr.request.SolrRequestInfo; -import org.apache.solr.response.ResultContext; -import org.apache.solr.search.SolrIndexSearcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.solr.search.DocList; -import org.apache.solr.search.DocIterator; +import java.lang.invoke.MethodHandles; +import java.util.List; + import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.util.NamedList; import org.apache.solr.request.LocalSolrQueryRequest; +import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.request.SolrRequestInfo; +import org.apache.solr.response.ResultContext; import org.apache.solr.response.SolrQueryResponse; +import org.apache.solr.search.DocIterator; +import org.apache.solr.search.DocList; +import org.apache.solr.search.SolrIndexSearcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import java.lang.invoke.MethodHandles; -import java.util.List; +import static org.apache.solr.common.params.CommonParams.DISTRIB; /** * @@ -55,8 +57,8 @@ public class QuerySenderListener extends AbstractSolrEventListener { // bind the request to a particular searcher (the newSearcher) NamedList params = addEventParms(currentSearcher, nlst); // for this, we default to distrib = false - if (params.get("distrib") == null) { - params.add("distrib", false); + if (params.get(DISTRIB) == null) { + params.add(DISTRIB, false); } req = new LocalSolrQueryRequest(getCore(),params) { @Override public SolrIndexSearcher getSearcher() { return searcher; } diff --git a/solr/core/src/java/org/apache/solr/handler/BlobHandler.java b/solr/core/src/java/org/apache/solr/handler/BlobHandler.java index 7e97f5953f4..052fcfee48e 100644 --- a/solr/core/src/java/org/apache/solr/handler/BlobHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/BlobHandler.java @@ -66,6 +66,7 @@ import org.slf4j.LoggerFactory; import static java.util.Collections.singletonMap; import static org.apache.solr.common.params.CommonParams.ID; import static org.apache.solr.common.params.CommonParams.JSON; +import static org.apache.solr.common.params.CommonParams.SORT; import static org.apache.solr.common.params.CommonParams.VERSION; import static org.apache.solr.common.util.Utils.makeMap; @@ -204,7 +205,7 @@ public class BlobHandler extends RequestHandlerBase implements PluginInfoInitial new MapSolrParams((Map) makeMap( "q", StrUtils.formatString(q, blobName, version), "fl", "id,size,version,timestamp,blobName,md5", - "sort", "version desc")) + SORT, "version desc")) , rsp); } } diff --git a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java index 3b521547c81..6d41d830840 100644 --- a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java @@ -57,6 +57,8 @@ import org.apache.solr.util.plugin.SolrCoreAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.SORT; + public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, PermissionNameProvider { private StreamFactory streamFactory = new StreamFactory(); @@ -118,7 +120,7 @@ public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, P .withFunctionName("shortestPath", ShortestPathStream.class) .withFunctionName("gatherNodes", GatherNodesStream.class) .withFunctionName("nodes", GatherNodesStream.class) - .withFunctionName("sort", SortStream.class) + .withFunctionName(SORT, SortStream.class) .withFunctionName("scoreNodes", ScoreNodesStream.class) .withFunctionName("random", RandomStream.class) diff --git a/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java index 8230bf5d6be..0cd9e1d57ef 100644 --- a/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java @@ -38,6 +38,8 @@ import org.apache.solr.util.plugin.SolrCoreAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; + /** * Ping Request Handler for reporting SolrCore health to a Load Balancer. * @@ -180,10 +182,10 @@ public class PingRequestHandler extends RequestHandlerBase implements SolrCoreAw // in this case, we want to default distrib to false so // we only ping the single node - Boolean distrib = params.getBool("distrib"); + Boolean distrib = params.getBool(DISTRIB); if (distrib == null) { ModifiableSolrParams mparams = new ModifiableSolrParams(params); - mparams.set("distrib", false); + mparams.set(DISTRIB, false); req.setParams(mparams); } diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java index a25ede46e17..5d10664debf 100644 --- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java @@ -81,6 +81,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.solr.common.params.CommonParams.ID; +import static org.apache.solr.common.params.CommonParams.SORT; public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, PermissionNameProvider { @@ -148,7 +149,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, .withFunctionName("outerHashJoin", OuterHashJoinStream.class) .withFunctionName("intersect", IntersectStream.class) .withFunctionName("complement", ComplementStream.class) - .withFunctionName("sort", SortStream.class) + .withFunctionName(SORT, SortStream.class) .withFunctionName("train", TextLogitStream.class) .withFunctionName("features", FeaturesSelectionStream.class) .withFunctionName("daemon", DaemonStream.class) diff --git a/solr/core/src/java/org/apache/solr/handler/component/IterativeMergeStrategy.java b/solr/core/src/java/org/apache/solr/handler/component/IterativeMergeStrategy.java index 2138948f93a..97d4199a114 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/IterativeMergeStrategy.java +++ b/solr/core/src/java/org/apache/solr/handler/component/IterativeMergeStrategy.java @@ -38,6 +38,8 @@ import org.apache.http.client.HttpClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; + public abstract class IterativeMergeStrategy implements MergeStrategy { protected ExecutorService executorService; @@ -89,7 +91,7 @@ public abstract class IterativeMergeStrategy implements MergeStrategy { this.originalShardResponse = originalShardResponse; req.setMethod(SolrRequest.METHOD.POST); ModifiableSolrParams params = (ModifiableSolrParams)req.getParams(); - params.add("distrib", "false"); + params.add(DISTRIB, "false"); } public QueryResponse getResponse() { diff --git a/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java b/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java index 55edc631114..ffb58588907 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java @@ -51,6 +51,8 @@ import org.apache.solr.search.SolrReturnFields; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.SORT; + /** * TODO! * @@ -339,7 +341,7 @@ public class MoreLikeThisComponent extends SearchComponent { String id = rb.req.getSchema().getUniqueKeyField() .getName(); s.params.set(CommonParams.FL, "score," + id); - s.params.set("sort", "score desc"); + s.params.set(SORT, "score desc"); // MLT Query is submitted as normal query to shards. s.params.set(CommonParams.Q, q); diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java index 12aa4033d14..421e74fc917 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java @@ -82,6 +82,7 @@ import org.apache.solr.util.RefCounted; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; import static org.apache.solr.common.params.CommonParams.ID; import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; @@ -840,7 +841,7 @@ public class RealTimeGetComponent extends SearchComponent // TODO: how to avoid hardcoding this and hit the same handler? sreq.params.set(ShardParams.SHARDS_QT,"/get"); - sreq.params.set("distrib",false); + sreq.params.set(DISTRIB,false); sreq.params.remove(ShardParams.SHARDS); sreq.params.remove(ID); 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 c05c6c42670..a00839b1e85 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 @@ -53,6 +53,7 @@ import org.apache.solr.util.plugin.SolrCoreAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; import static org.apache.solr.common.params.CommonParams.PATH; @@ -212,7 +213,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware , CoreContainer cc = req.getCore().getCoreDescriptor().getCoreContainer(); boolean isZkAware = cc.isZooKeeperAware(); - rb.isDistrib = req.getParams().getBool("distrib", isZkAware); + rb.isDistrib = req.getParams().getBool(DISTRIB, isZkAware); if (!rb.isDistrib) { // for back compat, a shards param with URLs like localhost:8983/solr will mean that this // search is distributed. @@ -361,7 +362,7 @@ public class SearchHandler extends RequestHandlerBase implements SolrCoreAware , for (String shard : sreq.actualShards) { ModifiableSolrParams params = new ModifiableSolrParams(sreq.params); params.remove(ShardParams.SHARDS); // not a top-level request - params.set(CommonParams.DISTRIB, "false"); // not a top-level request + params.set(DISTRIB, "false"); // not a top-level request params.remove("indent"); params.remove(CommonParams.HEADER_ECHO_PARAMS); params.set(ShardParams.IS_SHARD, true); // a sub (shard) request 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 c3b83dbdd83..e313b440ce7 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 @@ -55,6 +55,7 @@ import org.apache.solr.common.params.ModifiableSolrParams; import java.io.IOException; import java.util.*; import java.util.stream.Collectors; +import static org.apache.solr.common.params.CommonParams.SORT; /** * Table based on a Solr collection @@ -271,13 +272,13 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { String fl = getFields(fields); if(orders.size() > 0) { - params.add(CommonParams.SORT, getSort(orders)); + params.add(SORT, getSort(orders)); } else { if(limit == null) { - params.add(CommonParams.SORT, "_version_ desc"); + params.add(SORT, "_version_ desc"); fl = fl+",_version_"; } else { - params.add(CommonParams.SORT, "score desc"); + params.add(SORT, "score desc"); if(fl.indexOf("score") == -1) { fl = fl + ",score"; } @@ -460,7 +461,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { params.set("partitionKeys", getPartitionKeys(buckets)); } - params.set("sort", sort); + params.set(SORT, sort); TupleStream tupleStream = null; @@ -699,7 +700,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { params.set("partitionKeys", getPartitionKeys(buckets)); } - params.set("sort", sort); + params.set(SORT, sort); TupleStream tupleStream = null; diff --git a/solr/core/src/java/org/apache/solr/index/SortingMergePolicyFactory.java b/solr/core/src/java/org/apache/solr/index/SortingMergePolicyFactory.java index b22df3b3f97..ea2387d5914 100644 --- a/solr/core/src/java/org/apache/solr/index/SortingMergePolicyFactory.java +++ b/solr/core/src/java/org/apache/solr/index/SortingMergePolicyFactory.java @@ -18,25 +18,24 @@ package org.apache.solr.index; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.search.Sort; - import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.schema.IndexSchema; import org.apache.solr.search.SortSpecParsing; +import static org.apache.solr.common.params.CommonParams.SORT; + /** * A {@link MergePolicyFactory} for {@code SortingMergePolicy} objects. */ public class SortingMergePolicyFactory extends WrapperMergePolicyFactory { - static final String SORT = "sort"; // not private so that test(s) can use it - protected final Sort mergeSort; public SortingMergePolicyFactory(SolrResourceLoader resourceLoader, MergePolicyFactoryArgs args, IndexSchema schema) { super(resourceLoader, args, schema); final String sortArg = (String) args.remove(SORT); if (sortArg == null) { - throw new IllegalArgumentException(SortingMergePolicyFactory.class.getSimpleName()+" requires a '"+SORT+ "' argument."); + throw new IllegalArgumentException(SortingMergePolicyFactory.class.getSimpleName()+" requires a '"+ SORT + "' argument."); } this.mergeSort = SortSpecParsing.parseSortSpec(sortArg, schema).getSort(); } diff --git a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java index 5370dd9f3f9..b452802cad1 100644 --- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java +++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java @@ -96,6 +96,8 @@ import org.apache.solr.util.BoundedTreeSet; import org.apache.solr.util.DefaultSolrThreadFactory; import org.apache.solr.util.RTimer; +import static org.apache.solr.common.params.CommonParams.SORT; + /** * A class that generates simple Facet information for a request. * @@ -531,7 +533,7 @@ public class SimpleFacets { default: sortVal = sort; } - jsonFacet.put("sort", sortVal ); + jsonFacet.put(SORT, sortVal ); Map topLevel = new HashMap<>(); topLevel.put(field, jsonFacet); diff --git a/solr/core/src/java/org/apache/solr/request/json/RequestUtil.java b/solr/core/src/java/org/apache/solr/request/json/RequestUtil.java index 2529e74f589..ac0dc1951ab 100644 --- a/solr/core/src/java/org/apache/solr/request/json/RequestUtil.java +++ b/solr/core/src/java/org/apache/solr/request/json/RequestUtil.java @@ -36,6 +36,7 @@ import org.noggit.JSONParser; import org.noggit.ObjectBuilder; import static org.apache.solr.common.params.CommonParams.JSON; +import static org.apache.solr.common.params.CommonParams.SORT; public class RequestUtil { /** @@ -206,8 +207,8 @@ public class RequestUtil { out = "start"; } else if ("limit".equals(key)) { out = "rows"; - } else if ("sort".equals(key)) { - out = "sort"; + } else if (SORT.equals(key)) { + out = SORT; } else if ("params".equals(key) || "facet".equals(key) ) { // handled elsewhere continue; diff --git a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java index 22f9f29e12b..61756ea9251 100644 --- a/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/CollapsingQParserPlugin.java @@ -25,6 +25,12 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import com.carrotsearch.hppc.FloatArrayList; +import com.carrotsearch.hppc.IntArrayList; +import com.carrotsearch.hppc.IntIntHashMap; +import com.carrotsearch.hppc.IntLongHashMap; +import com.carrotsearch.hppc.cursors.IntIntCursor; +import com.carrotsearch.hppc.cursors.IntLongCursor; import org.apache.commons.lang.StringUtils; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.DocValues; @@ -72,12 +78,7 @@ import org.apache.solr.schema.TrieIntField; import org.apache.solr.schema.TrieLongField; import org.apache.solr.uninverting.UninvertingReader; -import com.carrotsearch.hppc.FloatArrayList; -import com.carrotsearch.hppc.IntArrayList; -import com.carrotsearch.hppc.IntIntHashMap; -import com.carrotsearch.hppc.IntLongHashMap; -import com.carrotsearch.hppc.cursors.IntIntCursor; -import com.carrotsearch.hppc.cursors.IntLongCursor; +import static org.apache.solr.common.params.CommonParams.SORT; /** @@ -187,7 +188,7 @@ public class CollapsingQParserPlugin extends QParserPlugin { * returns a new GroupHeadSelector based on the specified local params */ public static GroupHeadSelector build(final SolrParams localParams) { - final String sortString = StringUtils.defaultIfBlank(localParams.get("sort"), null); + final String sortString = StringUtils.defaultIfBlank(localParams.get(SORT), null); final String max = StringUtils.defaultIfBlank(localParams.get("max"), null); final String min = StringUtils.defaultIfBlank(localParams.get("min"), null); diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java index 9835f7d7165..b446ecea8c6 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequest.java @@ -36,6 +36,7 @@ import org.apache.solr.search.QueryContext; import org.apache.solr.search.SolrIndexSearcher; import org.apache.solr.search.SyntaxError; +import static org.apache.solr.common.params.CommonParams.SORT; import static org.apache.solr.search.facet.FacetRequest.RefineMethod.NONE; @@ -645,7 +646,7 @@ class FacetFieldParser extends FacetParser { Object o = m.get("facet"); parseSubs(o); - parseSort( m.get("sort") ); + parseSort( m.get(SORT) ); } return facet; diff --git a/solr/core/src/java/org/apache/solr/search/facet/LegacyFacet.java b/solr/core/src/java/org/apache/solr/search/facet/LegacyFacet.java index 9457d9c4cc0..7c2bde46948 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/LegacyFacet.java +++ b/solr/core/src/java/org/apache/solr/search/facet/LegacyFacet.java @@ -35,6 +35,8 @@ import org.apache.solr.search.SolrReturnFields; import org.apache.solr.search.StrParser; import org.apache.solr.search.SyntaxError; +import static org.apache.solr.common.params.CommonParams.SORT; + public class LegacyFacet { private SolrParams params; private Map json; @@ -172,7 +174,7 @@ public class LegacyFacet { String sort = params.getFieldParam(f, FacetParams.FACET_SORT, limit>0 ? FacetParams.FACET_SORT_COUNT : FacetParams.FACET_SORT_INDEX); String prefix = params.getFieldParam(f, FacetParams.FACET_PREFIX); - Map cmd = new HashMap(); + Map cmd = new HashMap<>(); cmd.put("field", facetValue); if (offset != 0) cmd.put("offset", offset); if (limit != 10) cmd.put("limit", limit); @@ -182,9 +184,9 @@ public class LegacyFacet { if (sort.equals("count")) { // our default } else if (sort.equals("index")) { - cmd.put("sort", "index asc"); + cmd.put(SORT, "index asc"); } else { - cmd.put("sort", sort); // can be sort by one of our stats + cmd.put(SORT, sort); // can be sort by one of our stats } Map type = new HashMap<>(1); diff --git a/solr/core/src/java/org/apache/solr/update/PeerSync.java b/solr/core/src/java/org/apache/solr/update/PeerSync.java index 425d1dbbe68..9470cca41be 100644 --- a/solr/core/src/java/org/apache/solr/update/PeerSync.java +++ b/solr/core/src/java/org/apache/solr/update/PeerSync.java @@ -60,6 +60,7 @@ import org.apache.solr.update.processor.UpdateRequestProcessorChain; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; import static org.apache.solr.common.params.CommonParams.ID; import static org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase.FROMLEADER; import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM; @@ -405,7 +406,7 @@ public class PeerSync implements SolrMetricProducer { sreq.params = new ModifiableSolrParams(); sreq.params = new ModifiableSolrParams(); sreq.params.set("qt","/get"); - sreq.params.set("distrib",false); + sreq.params.set(DISTRIB,false); sreq.params.set("getFingerprint", String.valueOf(Long.MAX_VALUE)); shardHandler.submit(sreq, replica, sreq.params); @@ -421,7 +422,7 @@ public class PeerSync implements SolrMetricProducer { sreq.actualShards = sreq.shards; sreq.params = new ModifiableSolrParams(); sreq.params.set("qt","/get"); - sreq.params.set("distrib",false); + sreq.params.set(DISTRIB,false); sreq.params.set("getVersions",nUpdates); sreq.params.set("fingerprint",doFingerprint); shardHandler.submit(sreq, replica, sreq.params); @@ -506,7 +507,7 @@ public class PeerSync implements SolrMetricProducer { sreq.actualShards = sreq.shards; sreq.params = new ModifiableSolrParams(); sreq.params.set("qt", "/get"); - sreq.params.set("distrib", false); + sreq.params.set(DISTRIB, false); sreq.params.set("checkCanHandleVersionRanges", false); ShardHandler sh = shardHandlerFactory.getShardHandler(client); @@ -725,7 +726,7 @@ public class PeerSync implements SolrMetricProducer { sreq.purpose = 0; sreq.params = new ModifiableSolrParams(); sreq.params.set("qt", "/get"); - sreq.params.set("distrib", false); + sreq.params.set(DISTRIB, false); sreq.params.set("getUpdates", versionsAndRanges); sreq.params.set("onlyIfActive", onlyIfActive); @@ -890,7 +891,7 @@ public class PeerSync implements SolrMetricProducer { sreq.shards = new String[]{replica}; sreq.params = new ModifiableSolrParams(); sreq.params.set("qt","/get"); - sreq.params.set("distrib", false); + sreq.params.set(DISTRIB, false); sreq.params.set("getVersions",nUpdates); shardHandler.submit(sreq, replica, sreq.params); } 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 fe71b0b805f..55d1fc8c032 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 @@ -16,6 +16,7 @@ */ package org.apache.solr.update.processor; +import static org.apache.solr.common.params.CommonParams.DISTRIB; import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM; import java.io.IOException; @@ -1303,7 +1304,7 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor { String id = inplaceAdd.getPrintableId(); UpdateShardHandler updateShardHandler = inplaceAdd.getReq().getCore().getCoreDescriptor().getCoreContainer().getUpdateShardHandler(); ModifiableSolrParams params = new ModifiableSolrParams(); - params.set("distrib", false); + params.set(DISTRIB, false); params.set("getInputDocument", id); params.set("onlyIfActive", true); SolrRequest ur = new GenericSolrRequest(METHOD.GET, "/get", params); 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 4f6cf8dfae1..6a85422185d 100644 --- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java +++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java @@ -113,6 +113,7 @@ import org.slf4j.LoggerFactory; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.solr.common.SolrException.ErrorCode.FORBIDDEN; import static org.apache.solr.common.SolrException.ErrorCode.UNAUTHORIZED; +import static org.apache.solr.common.params.CommonParams.DISTRIB; import static org.apache.solr.common.params.CommonParams.NAME; /** @@ -1193,7 +1194,7 @@ public class SolrCLI { // query this replica directly to get doc count and assess health q = new SolrQuery("*:*"); q.setRows(0); - q.set("distrib", "false"); + q.set(DISTRIB, "false"); try (HttpSolrClient solr = new HttpSolrClient.Builder(coreUrl).build()) { String solrUrl = solr.getBaseURL(); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java index 9a204e26808..4676594835b 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java @@ -25,6 +25,7 @@ import org.apache.solr.client.solrj.io.stream.CloudSolrStream; import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.common.params.ModifiableSolrParams; +import static org.apache.solr.common.params.CommonParams.SORT; import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; @@ -74,7 +75,7 @@ public class ModelCache { ModifiableSolrParams params = new ModifiableSolrParams(); params.set("q","name_s:"+modelID); params.set("fl", "terms_ss, idfs_ds, weights_ds, iteration_i, _version_"); - params.set("sort", "iteration_i desc"); + params.set(SORT, "iteration_i desc"); StreamContext streamContext = new StreamContext(); streamContext.setSolrClientCache(solrClientCache); CloudSolrStream stream = new CloudSolrStream(zkHost, collection, params); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java index 52a6a1ed392..805967775a0 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java @@ -51,6 +51,8 @@ import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SolrjNamedThreadFactory; +import static org.apache.solr.common.params.CommonParams.SORT; + public class GatherNodesStream extends TupleStream implements Expressible { private String zkHost; @@ -449,7 +451,7 @@ public class GatherNodesStream extends TupleStream implements Expressible { ModifiableSolrParams joinSParams = new ModifiableSolrParams(SolrParams.toMultiMap(new NamedList(queryParams))); joinSParams.set("fl", buf.toString()); joinSParams.set("qt", "/export"); - joinSParams.set("sort", gather + " asc,"+traverseTo +" asc"); + joinSParams.set(SORT, gather + " asc,"+traverseTo +" asc"); StringBuffer nodeQuery = new StringBuffer(); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java index 768ce07ebb5..aa546ae9e0c 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java @@ -52,6 +52,8 @@ import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.SolrjNamedThreadFactory; +import static org.apache.solr.common.params.CommonParams.SORT; + public class ShortestPathStream extends TupleStream implements Expressible { private static final long serialVersionUID = 1; @@ -450,7 +452,7 @@ public class ShortestPathStream extends TupleStream implements Expressible { joinParams.set("fl", fl); joinParams.set("qt", "/export"); - joinParams.set("sort", toField + " asc,"+fromField +" asc"); + joinParams.set(SORT, toField + " asc,"+fromField +" asc"); StringBuffer nodeQuery = new StringBuffer(); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GroupOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GroupOperation.java index 9ed5cbe29be..4a28cc1e0fd 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GroupOperation.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GroupOperation.java @@ -39,6 +39,8 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import static org.apache.solr.common.params.CommonParams.SORT; + public class GroupOperation implements ReduceOperation { private static final long serialVersionUID = 1L; @@ -52,7 +54,7 @@ public class GroupOperation implements ReduceOperation { public GroupOperation(StreamExpression expression, StreamFactory factory) throws IOException { StreamExpressionNamedParameter nParam = factory.getNamedOperand(expression, "n"); - StreamExpressionNamedParameter sortExpression = factory.getNamedOperand(expression, "sort"); + StreamExpressionNamedParameter sortExpression = factory.getNamedOperand(expression, SORT); StreamComparator streamComparator = factory.constructComparator(((StreamExpressionValue) sortExpression.getParameter()).getValue(), FieldComparator.class); String nStr = ((StreamExpressionValue)nParam.getParameter()).getValue(); @@ -87,7 +89,7 @@ public class GroupOperation implements ReduceOperation { expression.addParameter(new StreamExpressionNamedParameter("n", Integer.toString(size))); // sort - expression.addParameter(new StreamExpressionNamedParameter("sort", streamComparator.toExpression(factory))); + expression.addParameter(new StreamExpressionNamedParameter(SORT, streamComparator.toExpression(factory))); return expression; } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java index f177585e2a3..1acd79dff63 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CloudSolrStream.java @@ -63,6 +63,9 @@ import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.SolrjNamedThreadFactory; import org.apache.solr.common.util.StrUtils; +import static org.apache.solr.common.params.CommonParams.DISTRIB; +import static org.apache.solr.common.params.CommonParams.SORT; + /** * Connects to Zookeeper to pick replicas from a specific collection to send the query to. * Under the covers the SolrStream instances send the query to the replicas. @@ -269,10 +272,10 @@ public class CloudSolrStream extends TupleStream implements Expressible { } String fls = String.join(",", params.getParams("fl")); - if (params.getParams("sort") == null) { + if (params.getParams(SORT) == null) { throw new IOException("sort param expected for search function"); } - String sorts = String.join(",", params.getParams("sort")); + String sorts = String.join(",", params.getParams(SORT)); this.comp = parseComp(sorts, fls); } @@ -403,7 +406,7 @@ public class CloudSolrStream extends TupleStream implements Expressible { Collection slices = CloudSolrStream.getSlices(this.collection, zkStateReader, true); ModifiableSolrParams mParams = new ModifiableSolrParams(params); - mParams.set("distrib", "false"); // We are the aggregator. + mParams.set(DISTRIB, "false"); // We are the aggregator. Set liveNodes = clusterState.getLiveNodes(); for(Slice slice : slices) { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java index f15e2a7c530..c5339cbcf0d 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java @@ -59,6 +59,7 @@ import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SolrjNamedThreadFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; import static org.apache.solr.common.params.CommonParams.ID; public class FeaturesSelectionStream extends TupleStream implements Expressible{ @@ -415,7 +416,7 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{ ModifiableSolrParams params = new ModifiableSolrParams(); HttpSolrClient solrClient = cache.getHttpSolrClient(baseUrl); - params.add("distrib", "false"); + params.add(DISTRIB, "false"); params.add("fq","{!igain}"); for(String key : paramsMap.keySet()) { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java index 55ca51a8277..06e6fdcced7 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java @@ -37,6 +37,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.apache.solr.common.params.ModifiableSolrParams; +import static org.apache.solr.common.params.CommonParams.SORT; import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; /** @@ -238,7 +239,7 @@ public class FetchStream extends TupleStream implements Expressible { params.add("q", buf.toString()); params.add("fl", fieldList+appendFields()); params.add("rows", Integer.toString(batchSize)); - params.add("sort", "_version_ desc"); + params.add(SORT, "_version_ desc"); CloudSolrStream cloudSolrStream = new CloudSolrStream(zkHost, collection, params); StreamContext newContext = new StreamContext(); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/JDBCStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/JDBCStream.java index 0f951038b19..172b9ef7994 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/JDBCStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/JDBCStream.java @@ -45,6 +45,8 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import static org.apache.solr.common.params.CommonParams.SORT; + /** * Connects to a datasource using a registered JDBC driver and execute a query. The results of * that query will be returned as tuples. An EOF tuple will indicate that all have been read. @@ -104,7 +106,7 @@ public class JDBCStream extends TupleStream implements Expressible { List namedParams = factory.getNamedOperands(expression); StreamExpressionNamedParameter connectionUrlExpression = factory.getNamedOperand(expression, "connection"); StreamExpressionNamedParameter sqlQueryExpression = factory.getNamedOperand(expression, "sql"); - StreamExpressionNamedParameter definedSortExpression = factory.getNamedOperand(expression, "sort"); + StreamExpressionNamedParameter definedSortExpression = factory.getNamedOperand(expression, SORT); StreamExpressionNamedParameter driverClassNameExpression = factory.getNamedOperand(expression, "driver"); // Validate there are no unknown parameters - zkHost and alias are namedParameter so we don't need to count it twice @@ -115,7 +117,7 @@ public class JDBCStream extends TupleStream implements Expressible { // All named params we don't care about will be passed to the driver on connection Properties connectionProperties = new Properties(); for(StreamExpressionNamedParameter namedParam : namedParams){ - if(!namedParam.getName().equals("driver") && !namedParam.getName().equals("connection") && !namedParam.getName().equals("sql") && !namedParam.getName().equals("sort")){ + if(!namedParam.getName().equals("driver") && !namedParam.getName().equals("connection") && !namedParam.getName().equals("sql") && !namedParam.getName().equals(SORT)){ connectionProperties.put(namedParam.getName(), namedParam.getParameter().toString().trim()); } } @@ -367,7 +369,7 @@ public class JDBCStream extends TupleStream implements Expressible { expression.addParameter(new StreamExpressionNamedParameter("sql", sqlQuery)); // sort - expression.addParameter(new StreamExpressionNamedParameter("sort", definedSort.toExpression(factory))); + expression.addParameter(new StreamExpressionNamedParameter(SORT, definedSort.toExpression(factory))); // driver class if(null != driverClassName){ diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ParallelStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ParallelStream.java index 10e80ad4273..87e13549d05 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ParallelStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ParallelStream.java @@ -21,23 +21,23 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.Set; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Random; +import java.util.Set; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.comp.FieldComparator; import org.apache.solr.client.solrj.io.comp.StreamComparator; import org.apache.solr.client.solrj.io.stream.expr.Explanation; +import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType; import org.apache.solr.client.solrj.io.stream.expr.Expressible; import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation; import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; -import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType; import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Slice; @@ -45,6 +45,9 @@ import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.params.ModifiableSolrParams; +import static org.apache.solr.common.params.CommonParams.DISTRIB; +import static org.apache.solr.common.params.CommonParams.SORT; + /** * The ParallelStream decorates a TupleStream implementation and pushes it to N workers for parallel execution. * Workers are chosen from a SolrCloud collection. @@ -85,7 +88,7 @@ public class ParallelStream extends CloudSolrStream implements Expressible { String collectionName = factory.getValueOperand(expression, 0); StreamExpressionNamedParameter workersParam = factory.getNamedOperand(expression, "workers"); List streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class); - StreamExpressionNamedParameter sortExpression = factory.getNamedOperand(expression, "sort"); + StreamExpressionNamedParameter sortExpression = factory.getNamedOperand(expression, SORT); StreamExpressionNamedParameter zkHostExpression = factory.getNamedOperand(expression, "zkHost"); // validate expression contains only what we want. @@ -188,7 +191,7 @@ public class ParallelStream extends CloudSolrStream implements Expressible { } // sort - expression.addParameter(new StreamExpressionNamedParameter("sort",comp.toExpression(factory))); + expression.addParameter(new StreamExpressionNamedParameter(SORT,comp.toExpression(factory))); // zkHost expression.addParameter(new StreamExpressionNamedParameter("zkHost", zkHost)); @@ -284,7 +287,7 @@ public class ParallelStream extends CloudSolrStream implements Expressible { for(int w=0; w streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class); StreamExpressionNamedParameter nParam = factory.getNamedOperand(expression, "n"); - StreamExpressionNamedParameter sortExpression = factory.getNamedOperand(expression, "sort"); + StreamExpressionNamedParameter sortExpression = factory.getNamedOperand(expression, SORT); // validate expression contains only what we want. if(expression.getParameters().size() != streamExpressions.size() + 2){ @@ -129,7 +131,7 @@ public class RankStream extends TupleStream implements Expressible { } // sort - expression.addParameter(new StreamExpressionNamedParameter("sort",comp.toExpression(factory))); + expression.addParameter(new StreamExpressionNamedParameter(SORT, comp.toExpression(factory))); return expression; } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ScoreNodesStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ScoreNodesStream.java index f394424e7e1..177cee0433a 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ScoreNodesStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/ScoreNodesStream.java @@ -19,11 +19,11 @@ package org.apache.solr.client.solrj.io.stream; import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.HashMap; import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.io.SolrClientCache; @@ -42,6 +42,8 @@ import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.TermsParams; import org.apache.solr.common.util.NamedList; +import static org.apache.solr.common.params.CommonParams.DISTRIB; + /** * Iterates over a gatherNodes() expression and scores the Tuples based on tf-idf. * @@ -211,7 +213,7 @@ public class ScoreNodesStream extends TupleStream implements Expressible params.add(TermsParams.TERMS_STATS, "true"); params.add(TermsParams.TERMS_LIST, builder.toString()); params.add(TermsParams.TERMS_LIMIT, Integer.toString(nodes.size())); - params.add("distrib", "true"); + params.add(DISTRIB, "true"); QueryRequest request = new QueryRequest(params); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java index 101a71d64d2..b4decd528ae 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java @@ -49,6 +49,8 @@ import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SolrjNamedThreadFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; + public class SignificantTermsStream extends TupleStream implements Expressible{ private static final long serialVersionUID = 1; @@ -376,7 +378,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{ ModifiableSolrParams params = new ModifiableSolrParams(); HttpSolrClient solrClient = cache.getHttpSolrClient(baseUrl); - params.add("distrib", "false"); + params.add(DISTRIB, "false"); params.add("fq","{!sigificantTerms}"); for(String key : paramsMap.keySet()) { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java index 7f194f46776..5a70cedeb79 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java @@ -60,6 +60,7 @@ import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SolrjNamedThreadFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; import static org.apache.solr.common.params.CommonParams.ID; public class TextLogitStream extends TupleStream implements Expressible { @@ -615,7 +616,7 @@ public class TextLogitStream extends TupleStream implements Expressible { ModifiableSolrParams params = new ModifiableSolrParams(); HttpSolrClient solrClient = cache.getHttpSolrClient(baseUrl); - params.add("distrib", "false"); + params.add(DISTRIB, "false"); params.add("fq","{!tlogit}"); params.add("feature", feature); params.add("terms", TextLogitStream.toString(terms)); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java index ccbe8c1673c..58063d0c6dd 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java @@ -63,7 +63,9 @@ import org.apache.solr.common.util.SolrjNamedThreadFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.solr.common.params.CommonParams.DISTRIB; import static org.apache.solr.common.params.CommonParams.ID; +import static org.apache.solr.common.params.CommonParams.SORT; import static org.apache.solr.common.params.CommonParams.VERSION_FIELD; public class TopicStream extends CloudSolrStream implements Expressible { @@ -436,8 +438,8 @@ public class TopicStream extends CloudSolrStream implements Expressible { long checkpoint = -1; ModifiableSolrParams params = new ModifiableSolrParams(); params.set("q","*:*"); - params.set("sort", "_version_ desc"); - params.set("distrib", "false"); + params.set(SORT, "_version_ desc"); + params.set(DISTRIB, "false"); params.set("rows", 1); for(Replica replica : replicas) { if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) { @@ -523,9 +525,9 @@ public class TopicStream extends CloudSolrStream implements Expressible { Collection slices = CloudSolrStream.getSlices(this.collection, zkStateReader, false); ModifiableSolrParams mParams = new ModifiableSolrParams(params); - mParams.set("distrib", "false"); // We are the aggregator. + mParams.set(DISTRIB, "false"); // We are the aggregator. String fl = mParams.get("fl"); - mParams.set("sort", "_version_ asc"); + mParams.set(SORT, "_version_ asc"); if(!fl.contains(VERSION_FIELD)) { fl += ",_version_"; } diff --git a/solr/solrj/src/java/org/apache/solr/common/params/TermsParams.java b/solr/solrj/src/java/org/apache/solr/common/params/TermsParams.java index d719500d096..497584667d2 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/TermsParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/TermsParams.java @@ -18,6 +18,8 @@ package org.apache.solr.common.params; import java.util.regex.Pattern; +import static org.apache.solr.common.params.CommonParams.SORT; + /** * * @@ -112,20 +114,20 @@ public interface TermsParams { /** * Optional. The maximum value of docFreq to be returned. -1 by default means no boundary */ - public static final String TERMS_MAXCOUNT = TERMS_PREFIX + "maxcount"; + String TERMS_MAXCOUNT = TERMS_PREFIX + "maxcount"; /** * Optional. If true, return the raw characters of the indexed term, regardless of if it is readable. * For instance, the index form of numeric numbers is not human readable. The default is false. */ - public static final String TERMS_RAW = TERMS_PREFIX + "raw"; + String TERMS_RAW = TERMS_PREFIX + "raw"; /** * Optional. If sorting by frequency is enabled. Defaults to sorting by count. */ - public static final String TERMS_SORT = TERMS_PREFIX + "sort"; + String TERMS_SORT = TERMS_PREFIX + SORT; - public static final String TERMS_SORT_COUNT = "count"; - public static final String TERMS_SORT_INDEX = "index"; + String TERMS_SORT_COUNT = "count"; + String TERMS_SORT_INDEX = "index"; } From c221ef0fdedaa92885746b3073150f0bd558f596 Mon Sep 17 00:00:00 2001 From: Shalin Shekhar Mangar Date: Thu, 23 Mar 2017 19:33:45 +0530 Subject: [PATCH 054/563] SOLR-10281: ADMIN_PATHS is duplicated in two places and inconsistent --- solr/CHANGES.txt | 5 +++++ .../solr/client/solrj/impl/CloudSolrClient.java | 15 +-------------- .../apache/solr/common/params/CommonParams.java | 4 +++- 3 files changed, 9 insertions(+), 15 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index d6aba02f386..e74e4ae3be0 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -123,6 +123,11 @@ Optimizations instance if it already is modifiable, otherwise creates a new ModifiableSolrParams instance. (Jörg Rathlev via Koji) +Bug Fixes +---------------------- +* SOLR-10281: ADMIN_PATHS is duplicated in two places and inconsistent. This can cause automatic + retries to /admin/metrics handler by the CloudSolrClient. (shalin) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java index 83c6326af4b..4c6dd51a781 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java @@ -84,11 +84,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.MDC; -import static org.apache.solr.common.params.CommonParams.AUTHC_PATH; -import static org.apache.solr.common.params.CommonParams.AUTHZ_PATH; -import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH; -import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH; -import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH; +import static org.apache.solr.common.params.CommonParams.ADMIN_PATHS; import static org.apache.solr.common.params.CommonParams.ID; /** @@ -1057,15 +1053,6 @@ public class CloudSolrClient extends SolrClient { collection = (reqParams != null) ? reqParams.get("collection", getDefaultCollection()) : getDefaultCollection(); return requestWithRetryOnStaleState(request, 0, collection); } - private static final Set ADMIN_PATHS = new HashSet<>(Arrays.asList( - CORES_HANDLER_PATH, - COLLECTIONS_HANDLER_PATH, - CONFIGSETS_HANDLER_PATH, - AUTHC_PATH, - AUTHZ_PATH, - "/____v2/cluster/security/authentication", - "/____v2/cluster/security/authorization" - )); /** * As this class doesn't watch external collections on the client side, diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java index 589ef7e7e94..c3a011c3f21 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/CommonParams.java @@ -188,7 +188,9 @@ public interface CommonParams { CONFIGSETS_HANDLER_PATH, AUTHC_PATH, AUTHZ_PATH, - METRICS_PATH)); + METRICS_PATH, + "/____v2/cluster/security/authentication", + "/____v2/cluster/security/authorization")); /** valid values for: echoParams */ enum EchoParamStyle { From 6665aed952f6bdd741ff22638fc7d21d46a5aaf4 Mon Sep 17 00:00:00 2001 From: Chris Hostetter Date: Thu, 23 Mar 2017 18:28:10 -0700 Subject: [PATCH 055/563] Fix test to stop asserting specific order when secondary sort is unspecified, add new checks that do assert an explicit order when secondary sort IS specified. --- .../apache/solr/schema/NumericFieldsTest.java | 108 +++++++++++++++--- 1 file changed, 95 insertions(+), 13 deletions(-) diff --git a/solr/core/src/test/org/apache/solr/schema/NumericFieldsTest.java b/solr/core/src/test/org/apache/solr/schema/NumericFieldsTest.java index fe5975ed058..17ded2f32e3 100644 --- a/solr/core/src/test/org/apache/solr/schema/NumericFieldsTest.java +++ b/solr/core/src/test/org/apache/solr/schema/NumericFieldsTest.java @@ -51,12 +51,15 @@ public class NumericFieldsTest extends SolrTestCaseJ4 { public void testSortMissingFirstLast() { clearIndex(); + // NOTE: segments may be merged in any order, so we can't make any assumptions about + // the relative order of M1 vs M2 unless we have a secondary sort assertU(adoc("id", "M1")); assertU(adoc(getDoc("+4", 4, "2011-04-04T00:00:00Z"))); assertU(adoc(getDoc("+5", 5, "2011-05-05T00:00:00Z"))); assertU(adoc(getDoc("-3", -3, "2011-01-01T00:00:00Z"))); assertU(adoc("id", "M2")); assertU(commit()); + // 'normal' sorting. Missing Values are 0 String suffix = ""; for (String t : types) { @@ -64,8 +67,8 @@ public class NumericFieldsTest extends SolrTestCaseJ4 { assertQ("Sorting Asc: " + t + suffix, req("fl", "id", "q", "*:*", "sort", (t + suffix) + " asc"), "//*[@numFound='5']", - "//result/doc[1]/str[@name='id'][.='M1']", - "//result/doc[2]/str[@name='id'][.='M2']", + "//result/doc[1]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[2]/str[@name='id'][starts-with(.,'M')]", "//result/doc[3]/str[@name='id'][.='-3']", "//result/doc[4]/str[@name='id'][.='+4']", "//result/doc[5]/str[@name='id'][.='+5']" @@ -77,6 +80,26 @@ public class NumericFieldsTest extends SolrTestCaseJ4 { "//result/doc[1]/str[@name='id'][.='+5']", "//result/doc[2]/str[@name='id'][.='+4']", "//result/doc[3]/str[@name='id'][.='-3']", + "//result/doc[4]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[5]/str[@name='id'][starts-with(.,'M')]" + ); + + assertQ("Sorting Asc w/secondary on id desc: " + t + suffix, + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " asc, id desc"), + "//*[@numFound='5']", + "//result/doc[1]/str[@name='id'][.='M2']", + "//result/doc[2]/str[@name='id'][.='M1']", + "//result/doc[3]/str[@name='id'][.='-3']", + "//result/doc[4]/str[@name='id'][.='+4']", + "//result/doc[5]/str[@name='id'][.='+5']" + ); + + assertQ("Sorting Desc w/secondary on id asc: " + t + suffix, + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " desc, id asc"), + "//*[@numFound='5']", + "//result/doc[1]/str[@name='id'][.='+5']", + "//result/doc[2]/str[@name='id'][.='+4']", + "//result/doc[3]/str[@name='id'][.='-3']", "//result/doc[4]/str[@name='id'][.='M1']", "//result/doc[5]/str[@name='id'][.='M2']" ); @@ -85,8 +108,8 @@ public class NumericFieldsTest extends SolrTestCaseJ4 { req("fl", "id", "q", "*:*", "sort", (t + suffix) + " asc"), "//*[@numFound='5']", "//result/doc[1]/str[@name='id'][.='-3']", - "//result/doc[2]/str[@name='id'][.='M1']", - "//result/doc[3]/str[@name='id'][.='M2']", + "//result/doc[2]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[3]/str[@name='id'][starts-with(.,'M')]", "//result/doc[4]/str[@name='id'][.='+4']", "//result/doc[5]/str[@name='id'][.='+5']" ); @@ -96,6 +119,26 @@ public class NumericFieldsTest extends SolrTestCaseJ4 { "//*[@numFound='5']", "//result/doc[1]/str[@name='id'][.='+5']", "//result/doc[2]/str[@name='id'][.='+4']", + "//result/doc[3]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[4]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[5]/str[@name='id'][.='-3']" + ); + + assertQ("Sorting Asc w/secondary on id desc: " + t + suffix, + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " asc, id desc"), + "//*[@numFound='5']", + "//result/doc[1]/str[@name='id'][.='-3']", + "//result/doc[2]/str[@name='id'][.='M2']", + "//result/doc[3]/str[@name='id'][.='M1']", + "//result/doc[4]/str[@name='id'][.='+4']", + "//result/doc[5]/str[@name='id'][.='+5']" + ); + + assertQ("Sorting Desc w/secondary on id asc: " + t + suffix, + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " desc, id asc"), + "//*[@numFound='5']", + "//result/doc[1]/str[@name='id'][.='+5']", + "//result/doc[2]/str[@name='id'][.='+4']", "//result/doc[3]/str[@name='id'][.='M1']", "//result/doc[4]/str[@name='id'][.='M2']", "//result/doc[5]/str[@name='id'][.='-3']" @@ -114,13 +157,32 @@ public class NumericFieldsTest extends SolrTestCaseJ4 { "//result/doc[1]/str[@name='id'][.='-3']", "//result/doc[2]/str[@name='id'][.='+4']", "//result/doc[3]/str[@name='id'][.='+5']", - "//result/doc[4]/str[@name='id'][.='M1']", - "//result/doc[5]/str[@name='id'][.='M2']" + "//result/doc[4]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[5]/str[@name='id'][starts-with(.,'M')]" ); - // This does not match assertQ("Sorting Desc: " + t + suffix, - req("fl", "id", "q", "*:*", "sort", (t + suffix) + " desc", "indent", "on"), + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " desc"), + "//*[@numFound='5']", + "//result/doc[1]/str[@name='id'][.='+5']", + "//result/doc[2]/str[@name='id'][.='+4']", + "//result/doc[3]/str[@name='id'][.='-3']", + "//result/doc[4]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[5]/str[@name='id'][starts-with(.,'M')]" + ); + + assertQ("Sorting Asc w/secondary on id desc: " + t + suffix, + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " asc, id desc"), + "//*[@numFound='5']", + "//result/doc[1]/str[@name='id'][.='-3']", + "//result/doc[2]/str[@name='id'][.='+4']", + "//result/doc[3]/str[@name='id'][.='+5']", + "//result/doc[4]/str[@name='id'][.='M2']", + "//result/doc[5]/str[@name='id'][.='M1']" + ); + + assertQ("Sorting Desc w/secondary on id asc: " + t + suffix, + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " desc, id asc"), "//*[@numFound='5']", "//result/doc[1]/str[@name='id'][.='+5']", "//result/doc[2]/str[@name='id'][.='+4']", @@ -134,18 +196,37 @@ public class NumericFieldsTest extends SolrTestCaseJ4 { suffix = "_first"; for (String t : types) { assertQ("Sorting Asc: " + t + suffix, - req("fl", "id", "q", "*:*", "sort", (t + suffix) + " asc", "indent", "on"), + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " asc"), "//*[@numFound='5']", - "//result/doc[1]/str[@name='id'][.='M1']", - "//result/doc[2]/str[@name='id'][.='M2']", + "//result/doc[1]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[2]/str[@name='id'][starts-with(.,'M')]", "//result/doc[3]/str[@name='id'][.='-3']", "//result/doc[4]/str[@name='id'][.='+4']", "//result/doc[5]/str[@name='id'][.='+5']" ); - // This does not match assertQ("Sorting Desc: " + t + suffix, - req("fl", "id", "q", "*:*", "sort", (t + suffix) + " desc", "indent", "on"), + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " desc"), + "//*[@numFound='5']", + "//result/doc[1]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[2]/str[@name='id'][starts-with(.,'M')]", + "//result/doc[3]/str[@name='id'][.='+5']", + "//result/doc[4]/str[@name='id'][.='+4']", + "//result/doc[5]/str[@name='id'][.='-3']" + ); + + assertQ("Sorting Asc w/secondary on id desc: " + t + suffix, + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " asc, id desc"), + "//*[@numFound='5']", + "//result/doc[1]/str[@name='id'][.='M2']", + "//result/doc[2]/str[@name='id'][.='M1']", + "//result/doc[3]/str[@name='id'][.='-3']", + "//result/doc[4]/str[@name='id'][.='+4']", + "//result/doc[5]/str[@name='id'][.='+5']" + ); + + assertQ("Sorting Desc w/secondary on id asc: " + t + suffix, + req("fl", "id", "q", "*:*", "sort", (t + suffix) + " desc, id asc"), "//*[@numFound='5']", "//result/doc[1]/str[@name='id'][.='M1']", "//result/doc[2]/str[@name='id'][.='M2']", @@ -153,6 +234,7 @@ public class NumericFieldsTest extends SolrTestCaseJ4 { "//result/doc[4]/str[@name='id'][.='+4']", "//result/doc[5]/str[@name='id'][.='-3']" ); + } } } From 53e5f34f66d264c8f0ea2861e77389902b2a36c4 Mon Sep 17 00:00:00 2001 From: Steve Rowe Date: Fri, 24 Mar 2017 12:31:16 -0400 Subject: [PATCH 056/563] SOLR-9221: Remove Solr contribs: map-reduce, morphlines-core and morphlines-cell --- .gitignore | 1 - dev-tools/idea/.idea/ant.xml | 3 - .../Solr_morphlines_cell_library.xml | 10 - .../Solr_morphlines_core_library.xml | 10 - .../Solr_morphlines_core_test_library.xml | 10 - dev-tools/idea/.idea/modules.xml | 3 - dev-tools/idea/.idea/workspace.xml | 37 +- .../solr/contrib/map-reduce/map-reduce.iml | 43 - .../morphlines-cell/morphlines-cell.iml | 29 - .../morphlines-core/morphlines-core.iml | 28 - .../solr/contrib/map-reduce/pom.xml.template | 90 - .../contrib/morphlines-cell/pom.xml.template | 90 - .../contrib/morphlines-core/pom.xml.template | 91 -- dev-tools/maven/solr/contrib/pom.xml.template | 3 - lucene/ivy-ignore-conflicts.properties | 1 - lucene/ivy-versions.properties | 59 - solr/CHANGES.txt | 10 + solr/contrib/map-reduce/README.txt | 20 - solr/contrib/map-reduce/build.xml | 157 -- solr/contrib/map-reduce/ivy.xml | 37 - .../src/java/assembly/hadoop-job.xml | 39 - .../solr/hadoop/AlphaNumericComparator.java | 75 - .../org/apache/solr/hadoop/BatchWriter.java | 243 --- .../solr/hadoop/DataInputInputStream.java | 58 - .../solr/hadoop/DataOutputOutputStream.java | 66 - .../solr/hadoop/DryRunDocumentLoader.java | 57 - .../java/org/apache/solr/hadoop/GoLive.java | 182 --- .../solr/hadoop/HdfsFileFieldNames.java | 41 - .../org/apache/solr/hadoop/HeartBeater.java | 159 -- .../solr/hadoop/LineRandomizerMapper.java | 67 - .../solr/hadoop/LineRandomizerReducer.java | 48 - .../solr/hadoop/MapReduceIndexerTool.java | 1388 ---------------- .../apache/solr/hadoop/PathArgumentType.java | 233 --- .../org/apache/solr/hadoop/PathParts.java | 130 -- .../solr/hadoop/SolrCloudPartitioner.java | 143 -- .../org/apache/solr/hadoop/SolrCounters.java | 53 - .../hadoop/SolrInputDocumentWritable.java | 66 - .../org/apache/solr/hadoop/SolrMapper.java | 39 - .../apache/solr/hadoop/SolrOutputFormat.java | 280 ---- .../apache/solr/hadoop/SolrRecordWriter.java | 479 ------ .../org/apache/solr/hadoop/SolrReducer.java | 188 --- .../solr/hadoop/ToolRunnerHelpFormatter.java | 90 - .../apache/solr/hadoop/TreeMergeMapper.java | 46 - .../solr/hadoop/TreeMergeOutputFormat.java | 201 --- .../UnbufferedDataInputInputStream.java | 114 -- .../java/org/apache/solr/hadoop/Utils.java | 59 - .../solr/hadoop/ZooKeeperInspector.java | 213 --- .../dedup/NoChangeUpdateConflictResolver.java | 36 - .../RejectingUpdateConflictResolver.java | 48 - ...etainMostRecentUpdateConflictResolver.java | 114 -- .../dedup/SolrInputDocumentComparator.java | 84 - .../dedup/SortingUpdateConflictResolver.java | 79 - .../hadoop/dedup/UpdateConflictResolver.java | 71 - .../solr/hadoop/dedup/package-info.java | 25 - .../hadoop/morphline/MorphlineCounters.java | 47 - .../hadoop/morphline/MorphlineMapRunner.java | 268 --- .../hadoop/morphline/MorphlineMapper.java | 193 --- .../solr/hadoop/morphline/package-info.java | 25 - .../org/apache/solr/hadoop/package-info.java | 25 - .../contrib/map-reduce/src/java/overview.html | 21 - .../map-reduce/src/test-files/README.txt | 1 - .../hadoop/AlphaNumericComparatorTest.java | 46 - .../apache/solr/hadoop/IdentityMapper.java | 38 - .../apache/solr/hadoop/IdentityReducer.java | 37 - .../LineRandomizerMapperReducerTest.java | 94 -- .../org/apache/solr/hadoop/MRUnitBase.java | 64 - ...apReduceIndexerToolArgumentParserTest.java | 468 ------ .../solr/hadoop/MorphlineBasicMiniMRTest.java | 415 ----- .../hadoop/MorphlineGoLiveMiniMRTest.java | 881 ---------- .../solr/hadoop/MorphlineMapperTest.java | 76 - .../solr/hadoop/MorphlineReducerTest.java | 131 -- .../org/apache/solr/hadoop/UtilsForTests.java | 57 - .../solr/hadoop/hack/MiniMRClientCluster.java | 41 - .../hack/MiniMRClientClusterFactory.java | 88 - .../solr/hadoop/hack/MiniMRCluster.java | 266 --- .../solr/hadoop/hack/MiniMRYarnCluster.java | 205 --- .../hadoop/hack/MiniMRYarnClusterAdapter.java | 78 - .../solr/hadoop/hack/MiniYARNCluster.java | 409 ----- solr/contrib/morphlines-cell/README.txt | 6 - solr/contrib/morphlines-cell/build.xml | 144 -- solr/contrib/morphlines-cell/ivy.xml | 35 - .../solr/morphlines/cell/SolrCellBuilder.java | 348 ---- ...StripNonCharSolrContentHandlerFactory.java | 81 - .../cell/TrimSolrContentHandlerFactory.java | 58 - .../solr/morphlines/cell/package-info.java | 25 - .../morphlines-cell/src/java/overview.html | 21 - .../morphlines-cell/src/test-files/README.txt | 1 - .../cell/SolrCellMorphlineTest.java | 292 ---- solr/contrib/morphlines-core/README.txt | 6 - solr/contrib/morphlines-core/build.xml | 105 -- solr/contrib/morphlines-core/ivy.xml | 128 -- .../solr/morphlines/solr/DocumentLoader.java | 73 - .../solr/morphlines/solr/FileUtils.java | 140 -- .../solr/GenerateSolrSequenceKeyBuilder.java | 143 -- .../solr/morphlines/solr/LoadSolrBuilder.java | 153 -- .../solr/SafeConcurrentUpdateSolrClient.java | 70 - .../SanitizeUnknownSolrFieldsBuilder.java | 101 -- .../solr/SolrClientDocumentLoader.java | 124 -- .../solr/morphlines/solr/SolrLocator.java | 254 --- .../morphlines/solr/SolrMorphlineContext.java | 80 - .../morphlines/solr/TokenizeTextBuilder.java | 154 -- .../morphlines/solr/ZooKeeperDownloader.java | 142 -- .../solr/morphlines/solr/package-info.java | 25 - .../morphlines-core/src/java/overview.html | 21 - .../src/test-files/custom-mimetypes.xml | 38 - .../src/test-files/log4j.properties | 7 - .../src/test-files/morphlines-core.marker | 1 - .../solr/collection1/conf/currency.xml | 67 - .../solr/collection1/conf/elevate.xml | 42 - .../collection1/conf/lang/contractions_ca.txt | 8 - .../collection1/conf/lang/contractions_fr.txt | 9 - .../collection1/conf/lang/contractions_ga.txt | 5 - .../collection1/conf/lang/contractions_it.txt | 23 - .../collection1/conf/lang/hyphenations_ga.txt | 5 - .../collection1/conf/lang/stemdict_nl.txt | 6 - .../collection1/conf/lang/stoptags_ja.txt | 420 ----- .../collection1/conf/lang/stopwords_ar.txt | 125 -- .../collection1/conf/lang/stopwords_bg.txt | 193 --- .../collection1/conf/lang/stopwords_ca.txt | 220 --- .../collection1/conf/lang/stopwords_cz.txt | 172 -- .../collection1/conf/lang/stopwords_da.txt | 108 -- .../collection1/conf/lang/stopwords_de.txt | 292 ---- .../collection1/conf/lang/stopwords_el.txt | 78 - .../collection1/conf/lang/stopwords_en.txt | 54 - .../collection1/conf/lang/stopwords_es.txt | 354 ---- .../collection1/conf/lang/stopwords_eu.txt | 99 -- .../collection1/conf/lang/stopwords_fa.txt | 313 ---- .../collection1/conf/lang/stopwords_fi.txt | 95 -- .../collection1/conf/lang/stopwords_fr.txt | 183 --- .../collection1/conf/lang/stopwords_ga.txt | 110 -- .../collection1/conf/lang/stopwords_gl.txt | 161 -- .../collection1/conf/lang/stopwords_hi.txt | 235 --- .../collection1/conf/lang/stopwords_hu.txt | 209 --- .../collection1/conf/lang/stopwords_hy.txt | 46 - .../collection1/conf/lang/stopwords_id.txt | 359 ---- .../collection1/conf/lang/stopwords_it.txt | 301 ---- .../collection1/conf/lang/stopwords_ja.txt | 127 -- .../collection1/conf/lang/stopwords_lv.txt | 172 -- .../collection1/conf/lang/stopwords_nl.txt | 117 -- .../collection1/conf/lang/stopwords_no.txt | 192 --- .../collection1/conf/lang/stopwords_pt.txt | 251 --- .../collection1/conf/lang/stopwords_ro.txt | 233 --- .../collection1/conf/lang/stopwords_ru.txt | 241 --- .../collection1/conf/lang/stopwords_sv.txt | 131 -- .../collection1/conf/lang/stopwords_th.txt | 119 -- .../collection1/conf/lang/stopwords_tr.txt | 212 --- .../collection1/conf/lang/userdict_ja.txt | 29 - .../solr/collection1/conf/protwords.txt | 21 - .../solr/collection1/conf/schema.xml | 927 ----------- .../solr/collection1/conf/solrconfig.xml | 1426 ---------------- .../solr/collection1/conf/stopwords.txt | 14 - .../solr/collection1/conf/synonyms.txt | 29 - .../test-files/solr/minimr/conf/currency.xml | 67 - .../test-files/solr/minimr/conf/elevate.xml | 42 - .../solr/minimr/conf/lang/contractions_ca.txt | 8 - .../solr/minimr/conf/lang/contractions_fr.txt | 9 - .../solr/minimr/conf/lang/contractions_ga.txt | 5 - .../solr/minimr/conf/lang/contractions_it.txt | 23 - .../solr/minimr/conf/lang/hyphenations_ga.txt | 5 - .../solr/minimr/conf/lang/stemdict_nl.txt | 6 - .../solr/minimr/conf/lang/stoptags_ja.txt | 420 ----- .../solr/minimr/conf/lang/stopwords_ar.txt | 125 -- .../solr/minimr/conf/lang/stopwords_bg.txt | 193 --- .../solr/minimr/conf/lang/stopwords_ca.txt | 220 --- .../solr/minimr/conf/lang/stopwords_cz.txt | 172 -- .../solr/minimr/conf/lang/stopwords_da.txt | 108 -- .../solr/minimr/conf/lang/stopwords_de.txt | 292 ---- .../solr/minimr/conf/lang/stopwords_el.txt | 78 - .../solr/minimr/conf/lang/stopwords_en.txt | 54 - .../solr/minimr/conf/lang/stopwords_es.txt | 354 ---- .../solr/minimr/conf/lang/stopwords_eu.txt | 99 -- .../solr/minimr/conf/lang/stopwords_fa.txt | 313 ---- .../solr/minimr/conf/lang/stopwords_fi.txt | 95 -- .../solr/minimr/conf/lang/stopwords_fr.txt | 183 --- .../solr/minimr/conf/lang/stopwords_ga.txt | 110 -- .../solr/minimr/conf/lang/stopwords_gl.txt | 161 -- .../solr/minimr/conf/lang/stopwords_hi.txt | 235 --- .../solr/minimr/conf/lang/stopwords_hu.txt | 209 --- .../solr/minimr/conf/lang/stopwords_hy.txt | 46 - .../solr/minimr/conf/lang/stopwords_id.txt | 359 ---- .../solr/minimr/conf/lang/stopwords_it.txt | 301 ---- .../solr/minimr/conf/lang/stopwords_ja.txt | 127 -- .../solr/minimr/conf/lang/stopwords_lv.txt | 172 -- .../solr/minimr/conf/lang/stopwords_nl.txt | 117 -- .../solr/minimr/conf/lang/stopwords_no.txt | 192 --- .../solr/minimr/conf/lang/stopwords_pt.txt | 251 --- .../solr/minimr/conf/lang/stopwords_ro.txt | 233 --- .../solr/minimr/conf/lang/stopwords_ru.txt | 241 --- .../solr/minimr/conf/lang/stopwords_sv.txt | 131 -- .../solr/minimr/conf/lang/stopwords_th.txt | 119 -- .../solr/minimr/conf/lang/stopwords_tr.txt | 212 --- .../solr/minimr/conf/lang/userdict_ja.txt | 29 - .../test-files/solr/minimr/conf/protwords.txt | 21 - .../test-files/solr/minimr/conf/schema.xml | 941 ----------- .../solr/minimr/conf/solrconfig.xml | 1446 ---------------- .../test-files/solr/minimr/conf/stopwords.txt | 14 - .../test-files/solr/minimr/conf/synonyms.txt | 29 - .../src/test-files/solr/minimr/solr.xml | 46 - .../test-files/solr/mrunit/conf/currency.xml | 67 - .../test-files/solr/mrunit/conf/elevate.xml | 42 - .../solr/mrunit/conf/lang/contractions_ca.txt | 8 - .../solr/mrunit/conf/lang/contractions_fr.txt | 9 - .../solr/mrunit/conf/lang/contractions_ga.txt | 5 - .../solr/mrunit/conf/lang/contractions_it.txt | 23 - .../solr/mrunit/conf/lang/hyphenations_ga.txt | 5 - .../solr/mrunit/conf/lang/stemdict_nl.txt | 6 - .../solr/mrunit/conf/lang/stoptags_ja.txt | 420 ----- .../solr/mrunit/conf/lang/stopwords_ar.txt | 125 -- .../solr/mrunit/conf/lang/stopwords_bg.txt | 193 --- .../solr/mrunit/conf/lang/stopwords_ca.txt | 220 --- .../solr/mrunit/conf/lang/stopwords_cz.txt | 172 -- .../solr/mrunit/conf/lang/stopwords_da.txt | 108 -- .../solr/mrunit/conf/lang/stopwords_de.txt | 292 ---- .../solr/mrunit/conf/lang/stopwords_el.txt | 78 - .../solr/mrunit/conf/lang/stopwords_en.txt | 54 - .../solr/mrunit/conf/lang/stopwords_es.txt | 354 ---- .../solr/mrunit/conf/lang/stopwords_eu.txt | 99 -- .../solr/mrunit/conf/lang/stopwords_fa.txt | 313 ---- .../solr/mrunit/conf/lang/stopwords_fi.txt | 95 -- .../solr/mrunit/conf/lang/stopwords_fr.txt | 183 --- .../solr/mrunit/conf/lang/stopwords_ga.txt | 110 -- .../solr/mrunit/conf/lang/stopwords_gl.txt | 161 -- .../solr/mrunit/conf/lang/stopwords_hi.txt | 235 --- .../solr/mrunit/conf/lang/stopwords_hu.txt | 209 --- .../solr/mrunit/conf/lang/stopwords_hy.txt | 46 - .../solr/mrunit/conf/lang/stopwords_id.txt | 359 ---- .../solr/mrunit/conf/lang/stopwords_it.txt | 301 ---- .../solr/mrunit/conf/lang/stopwords_ja.txt | 127 -- .../solr/mrunit/conf/lang/stopwords_lv.txt | 172 -- .../solr/mrunit/conf/lang/stopwords_nl.txt | 117 -- .../solr/mrunit/conf/lang/stopwords_no.txt | 192 --- .../solr/mrunit/conf/lang/stopwords_pt.txt | 251 --- .../solr/mrunit/conf/lang/stopwords_ro.txt | 233 --- .../solr/mrunit/conf/lang/stopwords_ru.txt | 241 --- .../solr/mrunit/conf/lang/stopwords_sv.txt | 131 -- .../solr/mrunit/conf/lang/stopwords_th.txt | 119 -- .../solr/mrunit/conf/lang/stopwords_tr.txt | 212 --- .../solr/mrunit/conf/lang/userdict_ja.txt | 29 - .../test-files/solr/mrunit/conf/protwords.txt | 21 - .../test-files/solr/mrunit/conf/schema.xml | 940 ----------- .../solr/mrunit/conf/solrconfig.xml | 1449 ----------------- .../test-files/solr/mrunit/conf/stopwords.txt | 14 - .../test-files/solr/mrunit/conf/synonyms.txt | 29 - .../src/test-files/solr/mrunit/solr.xml | 45 - .../src/test-files/solr/solr.xml | 34 - .../collection1/conf/currency.xml | 67 - .../solrcelltest/collection1/conf/elevate.xml | 42 - .../collection1/conf/lang/contractions_ca.txt | 8 - .../collection1/conf/lang/contractions_fr.txt | 9 - .../collection1/conf/lang/contractions_ga.txt | 5 - .../collection1/conf/lang/contractions_it.txt | 23 - .../collection1/conf/lang/hyphenations_ga.txt | 5 - .../collection1/conf/lang/stemdict_nl.txt | 6 - .../collection1/conf/lang/stoptags_ja.txt | 420 ----- .../collection1/conf/lang/stopwords_ar.txt | 125 -- .../collection1/conf/lang/stopwords_bg.txt | 193 --- .../collection1/conf/lang/stopwords_ca.txt | 220 --- .../collection1/conf/lang/stopwords_cz.txt | 172 -- .../collection1/conf/lang/stopwords_da.txt | 108 -- .../collection1/conf/lang/stopwords_de.txt | 292 ---- .../collection1/conf/lang/stopwords_el.txt | 78 - .../collection1/conf/lang/stopwords_en.txt | 54 - .../collection1/conf/lang/stopwords_es.txt | 354 ---- .../collection1/conf/lang/stopwords_eu.txt | 99 -- .../collection1/conf/lang/stopwords_fa.txt | 313 ---- .../collection1/conf/lang/stopwords_fi.txt | 95 -- .../collection1/conf/lang/stopwords_fr.txt | 183 --- .../collection1/conf/lang/stopwords_ga.txt | 110 -- .../collection1/conf/lang/stopwords_gl.txt | 161 -- .../collection1/conf/lang/stopwords_hi.txt | 235 --- .../collection1/conf/lang/stopwords_hu.txt | 209 --- .../collection1/conf/lang/stopwords_hy.txt | 46 - .../collection1/conf/lang/stopwords_id.txt | 359 ---- .../collection1/conf/lang/stopwords_it.txt | 301 ---- .../collection1/conf/lang/stopwords_ja.txt | 127 -- .../collection1/conf/lang/stopwords_lv.txt | 172 -- .../collection1/conf/lang/stopwords_nl.txt | 117 -- .../collection1/conf/lang/stopwords_no.txt | 192 --- .../collection1/conf/lang/stopwords_pt.txt | 251 --- .../collection1/conf/lang/stopwords_ro.txt | 233 --- .../collection1/conf/lang/stopwords_ru.txt | 241 --- .../collection1/conf/lang/stopwords_sv.txt | 131 -- .../collection1/conf/lang/stopwords_th.txt | 119 -- .../collection1/conf/lang/stopwords_tr.txt | 212 --- .../collection1/conf/lang/userdict_ja.txt | 29 - .../collection1/conf/protwords.txt | 21 - .../solrcelltest/collection1/conf/schema.xml | 893 ---------- .../collection1/conf/solrconfig.xml | 1426 ---------------- .../collection1/conf/stopwords.txt | 14 - .../collection1/conf/synonyms.txt | 29 - .../solr/solrcloud/conf/solrconfig.xml | 1437 ---------------- .../test-files/test-documents/NullHeader.docx | Bin 4355 -> 0 bytes .../test-documents/boilerplate.html | 58 - .../src/test-files/test-documents/cars.csv | 6 - .../src/test-files/test-documents/cars.csv.gz | Bin 167 -> 0 bytes .../src/test-files/test-documents/cars.tar.gz | Bin 298 -> 0 bytes .../test-files/test-documents/complex.mbox | 291 ---- .../src/test-files/test-documents/email.eml | 40 - .../src/test-files/test-documents/rsstest.rss | 36 - .../sample-statuses-20120521-100919.avro | Bin 3192 -> 0 bytes .../sample-statuses-20120906-141433 | 4 - ...ample-statuses-20120906-141433-medium.avro | Bin 249540 -> 0 bytes .../sample-statuses-20120906-141433.avro | Bin 1208 -> 0 bytes .../sample-statuses-20120906-141433.bz2 | Bin 1054 -> 0 bytes .../sample-statuses-20120906-141433.gz | Bin 907 -> 0 bytes .../test-documents/test-outlook.msg | Bin 19968 -> 0 bytes .../test-files/test-documents/testAIFF.aif | Bin 3894 -> 0 bytes .../src/test-files/test-documents/testBMP.bmp | Bin 22554 -> 0 bytes .../test-files/test-documents/testBMPfp.txt | 3 - .../test-files/test-documents/testEMLX.emlx | 72 - .../test-files/test-documents/testEXCEL.xls | Bin 13824 -> 0 bytes .../test-files/test-documents/testEXCEL.xlsx | Bin 9453 -> 0 bytes .../test-files/test-documents/testFLAC.flac | Bin 10604 -> 0 bytes .../src/test-files/test-documents/testFLV.flv | Bin 90580 -> 0 bytes .../test-documents/testJPEG_EXIF.jpg | Bin 16357 -> 0 bytes .../test-documents/testJPEG_EXIF.jpg.gz | Bin 8595 -> 0 bytes .../test-documents/testJPEG_EXIF.jpg.tar.gz | Bin 8722 -> 0 bytes .../test-files/test-documents/testMP3i18n.mp3 | Bin 40832 -> 0 bytes .../src/test-files/test-documents/testMP4.m4a | Bin 4770 -> 0 bytes .../src/test-files/test-documents/testPDF.pdf | Bin 34824 -> 0 bytes .../src/test-files/test-documents/testPNG.png | Bin 17041 -> 0 bytes .../test-documents/testPPT_various.ppt | Bin 164352 -> 0 bytes .../test-documents/testPPT_various.pptx | Bin 56659 -> 0 bytes .../src/test-files/test-documents/testPSD.psd | Bin 69410 -> 0 bytes .../test-files/test-documents/testPages.pages | Bin 134152 -> 0 bytes .../src/test-files/test-documents/testRFC822 | 41 - .../test-documents/testRTFVarious.rtf | 329 ---- .../src/test-files/test-documents/testSVG.svg | 23 - .../test-files/test-documents/testTIFF.tif | Bin 25584 -> 0 bytes .../test-files/test-documents/testVISIO.vsd | Bin 45568 -> 0 bytes .../src/test-files/test-documents/testWAV.wav | Bin 3884 -> 0 bytes .../test-documents/testWORD_various.doc | Bin 35328 -> 0 bytes .../src/test-files/test-documents/testXML.xml | 48 - .../test-files/test-documents/testXML2.xml | 22 - .../test-morphlines/loadSolrBasic.conf | 68 - .../solrCellDocumentTypes.conf | 304 ---- .../solrCellJPGCompressed.conf | 150 -- .../test-morphlines/solrCellXML.conf | 78 - .../test-morphlines/tokenizeText.conf | 38 - .../tutorialReadAvroContainer.conf | 145 -- .../solr/AbstractSolrMorphlineTestBase.java | 318 ---- .../solr/AbstractSolrMorphlineZkTestBase.java | 141 -- .../solr/CollectingDocumentLoader.java | 95 -- .../solr/EmbeddedTestSolrServer.java | 46 - .../morphlines/solr/SolrMorphlineTest.java | 69 - .../solr/SolrMorphlineZkAliasTest.java | 96 -- .../solr/SolrMorphlineZkAvroTest.java | 140 -- .../morphlines/solr/SolrMorphlineZkTest.java | 92 -- solr/licenses/Saxon-HE-9.6.0-2.jar.sha1 | 1 - solr/licenses/Saxon-HE-LICENSE-MPL.txt | 108 -- solr/licenses/aopalliance-1.0.jar.sha1 | 1 - solr/licenses/aopalliance-LICENSE-PD.txt | 1 - solr/licenses/argparse4j-0.4.3.jar.sha1 | 1 - solr/licenses/argparse4j-LICENSE-MIT.txt | 23 - solr/licenses/avro-1.7.5.jar.sha1 | 1 - solr/licenses/avro-LICENSE-ASL.txt | 308 ---- solr/licenses/avro-NOTICE.txt | 9 - solr/licenses/bcpkix-jdk15on-1.47.jar.sha1 | 1 - .../bcpkix-jdk15on-LICENSE-BSD_LIKE.txt | 15 - solr/licenses/bcpkix-jdk15on-NOTICE.txt | 2 - solr/licenses/config-1.0.2.jar.sha1 | 1 - solr/licenses/config-LICENSE-ASL.txt | 202 --- solr/licenses/config-NOTICE.txt | 0 solr/licenses/guice-3.0.jar.sha1 | 1 - solr/licenses/guice-LICENSE-ASL.txt | 202 --- solr/licenses/guice-NOTICE.txt | 0 solr/licenses/guice-servlet-3.0.jar.sha1 | 1 - solr/licenses/guice-servlet-LICENSE-ASL.txt | 202 --- solr/licenses/guice-servlet-NOTICE.txt | 0 ...hadoop-mapreduce-client-app-2.7.2.jar.sha1 | 1 - ...adoop-mapreduce-client-app-LICENSE-ASL.txt | 244 --- .../hadoop-mapreduce-client-app-NOTICE.txt | 2 - ...oop-mapreduce-client-common-2.7.2.jar.sha1 | 1 - ...op-mapreduce-client-common-LICENSE-ASL.txt | 244 --- .../hadoop-mapreduce-client-common-NOTICE.txt | 2 - ...adoop-mapreduce-client-core-2.7.2.jar.sha1 | 1 - ...doop-mapreduce-client-core-LICENSE-ASL.txt | 244 --- .../hadoop-mapreduce-client-core-NOTICE.txt | 2 - .../hadoop-mapreduce-client-hs-2.7.2.jar.sha1 | 1 - ...hadoop-mapreduce-client-hs-LICENSE-ASL.txt | 244 --- .../hadoop-mapreduce-client-hs-NOTICE.txt | 2 - ...duce-client-jobclient-2.7.2-tests.jar.sha1 | 1 - ...-mapreduce-client-jobclient-2.7.2.jar.sha1 | 1 - ...mapreduce-client-jobclient-LICENSE-ASL.txt | 244 --- ...doop-mapreduce-client-jobclient-NOTICE.txt | 2 - ...op-mapreduce-client-shuffle-2.7.2.jar.sha1 | 1 - ...p-mapreduce-client-shuffle-LICENSE-ASL.txt | 244 --- ...hadoop-mapreduce-client-shuffle-NOTICE.txt | 2 - solr/licenses/hadoop-yarn-api-2.7.2.jar.sha1 | 1 - solr/licenses/hadoop-yarn-api-LICENSE-ASL.txt | 244 --- solr/licenses/hadoop-yarn-api-NOTICE.txt | 2 - .../hadoop-yarn-client-2.7.2.jar.sha1 | 1 - .../hadoop-yarn-client-LICENSE-ASL.txt | 244 --- solr/licenses/hadoop-yarn-client-NOTICE.txt | 2 - .../hadoop-yarn-common-2.7.2.jar.sha1 | 1 - .../hadoop-yarn-common-LICENSE-ASL.txt | 244 --- solr/licenses/hadoop-yarn-common-NOTICE.txt | 2 - ...r-applicationhistoryservice-2.7.2.jar.sha1 | 1 - ...-applicationhistoryservice-LICENSE-ASL.txt | 244 --- ...erver-applicationhistoryservice-NOTICE.txt | 2 - .../hadoop-yarn-server-common-2.7.2.jar.sha1 | 1 - .../hadoop-yarn-server-common-LICENSE-ASL.txt | 244 --- .../hadoop-yarn-server-common-NOTICE.txt | 2 - ...oop-yarn-server-nodemanager-2.7.2.jar.sha1 | 1 - ...op-yarn-server-nodemanager-LICENSE-ASL.txt | 244 --- .../hadoop-yarn-server-nodemanager-NOTICE.txt | 2 - ...yarn-server-resourcemanager-2.7.2.jar.sha1 | 1 - ...arn-server-resourcemanager-LICENSE-ASL.txt | 244 --- ...oop-yarn-server-resourcemanager-NOTICE.txt | 2 - ...oop-yarn-server-tests-2.7.2-tests.jar.sha1 | 1 - .../hadoop-yarn-server-tests-LICENSE-ASL.txt | 244 --- .../hadoop-yarn-server-tests-NOTICE.txt | 2 - ...adoop-yarn-server-web-proxy-2.7.2.jar.sha1 | 1 - ...doop-yarn-server-web-proxy-LICENSE-ASL.txt | 244 --- .../hadoop-yarn-server-web-proxy-NOTICE.txt | 2 - solr/licenses/jackson-jaxrs-1.9.13.jar.sha1 | 1 - solr/licenses/jackson-jaxrs-LICENSE-ASL.txt | 13 - solr/licenses/jackson-jaxrs-NOTICE.txt | 7 - solr/licenses/javax.inject-1.jar.sha1 | 1 - solr/licenses/javax.inject-LICENSE-ASL.txt | 202 --- solr/licenses/javax.inject-NOTICE.txt | 0 solr/licenses/jaxb-impl-2.2.3-1.jar.sha1 | 1 - solr/licenses/jaxb-impl-LICENSE-CDDL.txt | 135 -- solr/licenses/jersey-bundle-1.9.jar.sha1 | 1 - solr/licenses/jersey-bundle-LICENSE-CDDL.txt | 85 - solr/licenses/jersey-guice-1.9.jar.sha1 | 1 - solr/licenses/jersey-guice-LICENSE-CDDL.txt | 85 - solr/licenses/jersey-json-1.9.jar.sha1 | 1 - solr/licenses/jersey-json-LICENSE-CDDL.txt | 85 - .../kite-morphlines-avro-1.1.0.jar.sha1 | 1 - .../kite-morphlines-avro-LICENSE-ASL.txt | 202 --- solr/licenses/kite-morphlines-avro-NOTICE.txt | 8 - .../kite-morphlines-core-1.1.0-tests.jar.sha1 | 1 - .../kite-morphlines-core-1.1.0.jar.sha1 | 1 - .../kite-morphlines-core-LICENSE-ASL.txt | 202 --- solr/licenses/kite-morphlines-core-NOTICE.txt | 8 - ...phlines-hadoop-sequencefile-1.1.0.jar.sha1 | 1 - ...hlines-hadoop-sequencefile-LICENSE-ASL.txt | 202 --- ...-morphlines-hadoop-sequencefile-NOTICE.txt | 8 - .../kite-morphlines-json-1.1.0.jar.sha1 | 1 - .../kite-morphlines-json-LICENSE-ASL.txt | 202 --- solr/licenses/kite-morphlines-json-NOTICE.txt | 8 - .../kite-morphlines-saxon-1.1.0.jar.sha1 | 1 - .../kite-morphlines-saxon-LICENSE-ASL.txt | 202 --- .../licenses/kite-morphlines-saxon-NOTICE.txt | 8 - .../kite-morphlines-tika-core-1.1.0.jar.sha1 | 1 - .../kite-morphlines-tika-core-LICENSE-ASL.txt | 202 --- .../kite-morphlines-tika-core-NOTICE.txt | 8 - ...-morphlines-tika-decompress-1.1.0.jar.sha1 | 1 - ...morphlines-tika-decompress-LICENSE-ASL.txt | 202 --- ...kite-morphlines-tika-decompress-NOTICE.txt | 8 - .../kite-morphlines-twitter-1.1.0.jar.sha1 | 1 - .../kite-morphlines-twitter-LICENSE-ASL.txt | 202 --- .../kite-morphlines-twitter-NOTICE.txt | 8 - solr/licenses/leveldb-0.7.jar.sha1 | 1 - solr/licenses/leveldb-LICENSE-BSD.txt | 27 - solr/licenses/leveldb-NOTICE.txt | 0 solr/licenses/leveldb-api-0.7.jar.sha1 | 1 - solr/licenses/leveldb-api-LICENSE-BSD.txt | 27 - solr/licenses/leveldb-api-NOTICE.txt | 0 solr/licenses/leveldbjni-1.8.jar.sha1 | 1 - solr/licenses/leveldbjni-LICENSE-BSD.txt | 27 - solr/licenses/leveldbjni-NOTICE.txt | 0 .../metrics-healthchecks-3.1.2.jar.sha1 | 1 - .../metrics-healthchecks-LICENSE-ASL.txt | 202 --- solr/licenses/metrics-healthchecks-NOTICE.txt | 11 - solr/licenses/mrunit-1.0.0-hadoop2.jar.sha1 | 1 - solr/licenses/mrunit-LICENSE-ASL.txt | 479 ------ solr/licenses/mrunit-NOTICE.txt | 5 - solr/licenses/netty-3.2.4.Final.jar.sha1 | 1 - solr/licenses/netty-LICENSE-ASL.txt | 202 --- solr/licenses/netty-NOTICE.txt | 98 -- solr/licenses/paranamer-2.3.jar.sha1 | 1 - solr/licenses/paranamer-LICENSE-BSD.txt | 28 - solr/licenses/paranamer-NOTICE.txt | 0 solr/licenses/rome-1.6.1.jar.sha1 | 1 - solr/licenses/snappy-java-1.0.5.jar.sha1 | 1 - solr/licenses/snappy-java-LICENSE-ASL.txt | 201 --- solr/licenses/snappy-java-NOTICE.txt | 0 .../map-reduce/set-map-reduce-classpath.sh | 52 - 480 files changed, 15 insertions(+), 59625 deletions(-) delete mode 100644 dev-tools/idea/.idea/libraries/Solr_morphlines_cell_library.xml delete mode 100644 dev-tools/idea/.idea/libraries/Solr_morphlines_core_library.xml delete mode 100644 dev-tools/idea/.idea/libraries/Solr_morphlines_core_test_library.xml delete mode 100644 dev-tools/idea/solr/contrib/map-reduce/map-reduce.iml delete mode 100644 dev-tools/idea/solr/contrib/morphlines-cell/morphlines-cell.iml delete mode 100644 dev-tools/idea/solr/contrib/morphlines-core/morphlines-core.iml delete mode 100644 dev-tools/maven/solr/contrib/map-reduce/pom.xml.template delete mode 100644 dev-tools/maven/solr/contrib/morphlines-cell/pom.xml.template delete mode 100644 dev-tools/maven/solr/contrib/morphlines-core/pom.xml.template delete mode 100644 solr/contrib/map-reduce/README.txt delete mode 100644 solr/contrib/map-reduce/build.xml delete mode 100644 solr/contrib/map-reduce/ivy.xml delete mode 100644 solr/contrib/map-reduce/src/java/assembly/hadoop-job.xml delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/AlphaNumericComparator.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/BatchWriter.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DataInputInputStream.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DataOutputOutputStream.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DryRunDocumentLoader.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/GoLive.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/HdfsFileFieldNames.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/HeartBeater.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/LineRandomizerMapper.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/LineRandomizerReducer.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/MapReduceIndexerTool.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/PathArgumentType.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/PathParts.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrCloudPartitioner.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrCounters.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrInputDocumentWritable.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrMapper.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrRecordWriter.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrReducer.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ToolRunnerHelpFormatter.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeMapper.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/UnbufferedDataInputInputStream.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/Utils.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ZooKeeperInspector.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/NoChangeUpdateConflictResolver.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/RejectingUpdateConflictResolver.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/RetainMostRecentUpdateConflictResolver.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/SolrInputDocumentComparator.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/SortingUpdateConflictResolver.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/UpdateConflictResolver.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/package-info.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineCounters.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineMapRunner.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineMapper.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/package-info.java delete mode 100644 solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/package-info.java delete mode 100644 solr/contrib/map-reduce/src/java/overview.html delete mode 100644 solr/contrib/map-reduce/src/test-files/README.txt delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/AlphaNumericComparatorTest.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/IdentityMapper.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/IdentityReducer.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/LineRandomizerMapperReducerTest.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineMapperTest.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineReducerTest.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/UtilsForTests.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRClientCluster.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRClientClusterFactory.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRCluster.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRYarnCluster.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRYarnClusterAdapter.java delete mode 100644 solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniYARNCluster.java delete mode 100644 solr/contrib/morphlines-cell/README.txt delete mode 100644 solr/contrib/morphlines-cell/build.xml delete mode 100644 solr/contrib/morphlines-cell/ivy.xml delete mode 100644 solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/SolrCellBuilder.java delete mode 100644 solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/StripNonCharSolrContentHandlerFactory.java delete mode 100644 solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/TrimSolrContentHandlerFactory.java delete mode 100644 solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/package-info.java delete mode 100644 solr/contrib/morphlines-cell/src/java/overview.html delete mode 100644 solr/contrib/morphlines-cell/src/test-files/README.txt delete mode 100644 solr/contrib/morphlines-cell/src/test/org/apache/solr/morphlines/cell/SolrCellMorphlineTest.java delete mode 100644 solr/contrib/morphlines-core/README.txt delete mode 100644 solr/contrib/morphlines-core/build.xml delete mode 100644 solr/contrib/morphlines-core/ivy.xml delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/DocumentLoader.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/FileUtils.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/GenerateSolrSequenceKeyBuilder.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/LoadSolrBuilder.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SafeConcurrentUpdateSolrClient.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SanitizeUnknownSolrFieldsBuilder.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrClientDocumentLoader.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrLocator.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrMorphlineContext.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/TokenizeTextBuilder.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/ZooKeeperDownloader.java delete mode 100644 solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/package-info.java delete mode 100644 solr/contrib/morphlines-core/src/java/overview.html delete mode 100644 solr/contrib/morphlines-core/src/test-files/custom-mimetypes.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/log4j.properties delete mode 100644 solr/contrib/morphlines-core/src/test-files/morphlines-core.marker delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/currency.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/elevate.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_ca.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_fr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_it.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/hyphenations_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stemdict_nl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stoptags_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ar.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_bg.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ca.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_cz.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_da.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_de.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_el.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_en.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_es.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_eu.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fa.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fi.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_gl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hi.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hu.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hy.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_id.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_it.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_lv.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_nl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_no.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_pt.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ro.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ru.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_sv.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_th.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_tr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/userdict_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/protwords.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/schema.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/solrconfig.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/stopwords.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/synonyms.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/currency.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/elevate.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_ca.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_fr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_it.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/hyphenations_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stemdict_nl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stoptags_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ar.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_bg.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ca.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_cz.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_da.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_de.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_el.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_en.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_es.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_eu.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fa.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fi.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_gl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hi.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hu.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hy.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_id.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_it.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_lv.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_nl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_no.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_pt.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ro.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ru.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_sv.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_th.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_tr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/userdict_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/protwords.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/schema.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/solrconfig.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/stopwords.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/synonyms.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/minimr/solr.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/currency.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/elevate.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_ca.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_fr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_it.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/hyphenations_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stemdict_nl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stoptags_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ar.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_bg.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ca.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_cz.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_da.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_de.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_el.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_en.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_es.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_eu.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fa.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fi.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_gl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hi.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hu.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hy.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_id.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_it.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_lv.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_nl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_no.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_pt.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ro.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ru.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_sv.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_th.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_tr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/userdict_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/protwords.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/schema.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/solrconfig.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/stopwords.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/synonyms.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/mrunit/solr.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solr.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/currency.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/elevate.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_ca.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_fr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_it.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/hyphenations_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stemdict_nl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stoptags_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ar.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_bg.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ca.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_cz.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_da.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_de.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_el.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_en.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_es.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_eu.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fa.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fi.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ga.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_gl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hi.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hu.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hy.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_id.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_it.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_lv.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_nl.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_no.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_pt.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ro.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ru.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_sv.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_th.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_tr.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/userdict_ja.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/protwords.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/schema.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/solrconfig.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/stopwords.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/synonyms.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/NullHeader.docx delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/boilerplate.html delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/cars.csv delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/cars.csv.gz delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/cars.tar.gz delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/complex.mbox delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/email.eml delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/rsstest.rss delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120521-100919.avro delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120906-141433 delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120906-141433-medium.avro delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120906-141433.avro delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120906-141433.bz2 delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120906-141433.gz delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/test-outlook.msg delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testAIFF.aif delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testBMP.bmp delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testBMPfp.txt delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testEMLX.emlx delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testEXCEL.xls delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testEXCEL.xlsx delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testFLAC.flac delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testFLV.flv delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testJPEG_EXIF.jpg delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testJPEG_EXIF.jpg.gz delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testJPEG_EXIF.jpg.tar.gz delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testMP3i18n.mp3 delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testMP4.m4a delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testPDF.pdf delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testPNG.png delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testPPT_various.ppt delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testPPT_various.pptx delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testPSD.psd delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testPages.pages delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testRFC822 delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testRTFVarious.rtf delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testSVG.svg delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testTIFF.tif delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testVISIO.vsd delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testWAV.wav delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testWORD_various.doc delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testXML.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-documents/testXML2.xml delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-morphlines/loadSolrBasic.conf delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellDocumentTypes.conf delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellJPGCompressed.conf delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellXML.conf delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-morphlines/tokenizeText.conf delete mode 100644 solr/contrib/morphlines-core/src/test-files/test-morphlines/tutorialReadAvroContainer.conf delete mode 100644 solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineTestBase.java delete mode 100644 solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineZkTestBase.java delete mode 100644 solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/CollectingDocumentLoader.java delete mode 100644 solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/EmbeddedTestSolrServer.java delete mode 100644 solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineTest.java delete mode 100644 solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAliasTest.java delete mode 100644 solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAvroTest.java delete mode 100644 solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkTest.java delete mode 100644 solr/licenses/Saxon-HE-9.6.0-2.jar.sha1 delete mode 100644 solr/licenses/Saxon-HE-LICENSE-MPL.txt delete mode 100644 solr/licenses/aopalliance-1.0.jar.sha1 delete mode 100644 solr/licenses/aopalliance-LICENSE-PD.txt delete mode 100644 solr/licenses/argparse4j-0.4.3.jar.sha1 delete mode 100644 solr/licenses/argparse4j-LICENSE-MIT.txt delete mode 100644 solr/licenses/avro-1.7.5.jar.sha1 delete mode 100644 solr/licenses/avro-LICENSE-ASL.txt delete mode 100644 solr/licenses/avro-NOTICE.txt delete mode 100644 solr/licenses/bcpkix-jdk15on-1.47.jar.sha1 delete mode 100644 solr/licenses/bcpkix-jdk15on-LICENSE-BSD_LIKE.txt delete mode 100644 solr/licenses/bcpkix-jdk15on-NOTICE.txt delete mode 100644 solr/licenses/config-1.0.2.jar.sha1 delete mode 100644 solr/licenses/config-LICENSE-ASL.txt delete mode 100644 solr/licenses/config-NOTICE.txt delete mode 100644 solr/licenses/guice-3.0.jar.sha1 delete mode 100644 solr/licenses/guice-LICENSE-ASL.txt delete mode 100644 solr/licenses/guice-NOTICE.txt delete mode 100644 solr/licenses/guice-servlet-3.0.jar.sha1 delete mode 100644 solr/licenses/guice-servlet-LICENSE-ASL.txt delete mode 100644 solr/licenses/guice-servlet-NOTICE.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-app-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-mapreduce-client-app-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-app-NOTICE.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-common-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-mapreduce-client-common-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-common-NOTICE.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-core-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-mapreduce-client-core-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-core-NOTICE.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-hs-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-mapreduce-client-hs-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-hs-NOTICE.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-jobclient-2.7.2-tests.jar.sha1 delete mode 100644 solr/licenses/hadoop-mapreduce-client-jobclient-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-mapreduce-client-jobclient-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-jobclient-NOTICE.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-shuffle-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-mapreduce-client-shuffle-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-mapreduce-client-shuffle-NOTICE.txt delete mode 100644 solr/licenses/hadoop-yarn-api-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-yarn-api-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-yarn-api-NOTICE.txt delete mode 100644 solr/licenses/hadoop-yarn-client-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-yarn-client-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-yarn-client-NOTICE.txt delete mode 100644 solr/licenses/hadoop-yarn-common-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-yarn-common-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-yarn-common-NOTICE.txt delete mode 100644 solr/licenses/hadoop-yarn-server-applicationhistoryservice-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-yarn-server-applicationhistoryservice-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-yarn-server-applicationhistoryservice-NOTICE.txt delete mode 100644 solr/licenses/hadoop-yarn-server-common-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-yarn-server-common-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-yarn-server-common-NOTICE.txt delete mode 100644 solr/licenses/hadoop-yarn-server-nodemanager-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-yarn-server-nodemanager-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-yarn-server-nodemanager-NOTICE.txt delete mode 100644 solr/licenses/hadoop-yarn-server-resourcemanager-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-yarn-server-resourcemanager-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-yarn-server-resourcemanager-NOTICE.txt delete mode 100644 solr/licenses/hadoop-yarn-server-tests-2.7.2-tests.jar.sha1 delete mode 100644 solr/licenses/hadoop-yarn-server-tests-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-yarn-server-tests-NOTICE.txt delete mode 100644 solr/licenses/hadoop-yarn-server-web-proxy-2.7.2.jar.sha1 delete mode 100644 solr/licenses/hadoop-yarn-server-web-proxy-LICENSE-ASL.txt delete mode 100644 solr/licenses/hadoop-yarn-server-web-proxy-NOTICE.txt delete mode 100644 solr/licenses/jackson-jaxrs-1.9.13.jar.sha1 delete mode 100644 solr/licenses/jackson-jaxrs-LICENSE-ASL.txt delete mode 100644 solr/licenses/jackson-jaxrs-NOTICE.txt delete mode 100644 solr/licenses/javax.inject-1.jar.sha1 delete mode 100644 solr/licenses/javax.inject-LICENSE-ASL.txt delete mode 100644 solr/licenses/javax.inject-NOTICE.txt delete mode 100644 solr/licenses/jaxb-impl-2.2.3-1.jar.sha1 delete mode 100644 solr/licenses/jaxb-impl-LICENSE-CDDL.txt delete mode 100644 solr/licenses/jersey-bundle-1.9.jar.sha1 delete mode 100644 solr/licenses/jersey-bundle-LICENSE-CDDL.txt delete mode 100644 solr/licenses/jersey-guice-1.9.jar.sha1 delete mode 100644 solr/licenses/jersey-guice-LICENSE-CDDL.txt delete mode 100644 solr/licenses/jersey-json-1.9.jar.sha1 delete mode 100644 solr/licenses/jersey-json-LICENSE-CDDL.txt delete mode 100644 solr/licenses/kite-morphlines-avro-1.1.0.jar.sha1 delete mode 100644 solr/licenses/kite-morphlines-avro-LICENSE-ASL.txt delete mode 100644 solr/licenses/kite-morphlines-avro-NOTICE.txt delete mode 100644 solr/licenses/kite-morphlines-core-1.1.0-tests.jar.sha1 delete mode 100644 solr/licenses/kite-morphlines-core-1.1.0.jar.sha1 delete mode 100644 solr/licenses/kite-morphlines-core-LICENSE-ASL.txt delete mode 100644 solr/licenses/kite-morphlines-core-NOTICE.txt delete mode 100644 solr/licenses/kite-morphlines-hadoop-sequencefile-1.1.0.jar.sha1 delete mode 100644 solr/licenses/kite-morphlines-hadoop-sequencefile-LICENSE-ASL.txt delete mode 100644 solr/licenses/kite-morphlines-hadoop-sequencefile-NOTICE.txt delete mode 100644 solr/licenses/kite-morphlines-json-1.1.0.jar.sha1 delete mode 100644 solr/licenses/kite-morphlines-json-LICENSE-ASL.txt delete mode 100644 solr/licenses/kite-morphlines-json-NOTICE.txt delete mode 100644 solr/licenses/kite-morphlines-saxon-1.1.0.jar.sha1 delete mode 100644 solr/licenses/kite-morphlines-saxon-LICENSE-ASL.txt delete mode 100644 solr/licenses/kite-morphlines-saxon-NOTICE.txt delete mode 100644 solr/licenses/kite-morphlines-tika-core-1.1.0.jar.sha1 delete mode 100644 solr/licenses/kite-morphlines-tika-core-LICENSE-ASL.txt delete mode 100644 solr/licenses/kite-morphlines-tika-core-NOTICE.txt delete mode 100644 solr/licenses/kite-morphlines-tika-decompress-1.1.0.jar.sha1 delete mode 100644 solr/licenses/kite-morphlines-tika-decompress-LICENSE-ASL.txt delete mode 100644 solr/licenses/kite-morphlines-tika-decompress-NOTICE.txt delete mode 100644 solr/licenses/kite-morphlines-twitter-1.1.0.jar.sha1 delete mode 100644 solr/licenses/kite-morphlines-twitter-LICENSE-ASL.txt delete mode 100644 solr/licenses/kite-morphlines-twitter-NOTICE.txt delete mode 100644 solr/licenses/leveldb-0.7.jar.sha1 delete mode 100644 solr/licenses/leveldb-LICENSE-BSD.txt delete mode 100644 solr/licenses/leveldb-NOTICE.txt delete mode 100644 solr/licenses/leveldb-api-0.7.jar.sha1 delete mode 100644 solr/licenses/leveldb-api-LICENSE-BSD.txt delete mode 100644 solr/licenses/leveldb-api-NOTICE.txt delete mode 100644 solr/licenses/leveldbjni-1.8.jar.sha1 delete mode 100644 solr/licenses/leveldbjni-LICENSE-BSD.txt delete mode 100644 solr/licenses/leveldbjni-NOTICE.txt delete mode 100644 solr/licenses/metrics-healthchecks-3.1.2.jar.sha1 delete mode 100644 solr/licenses/metrics-healthchecks-LICENSE-ASL.txt delete mode 100644 solr/licenses/metrics-healthchecks-NOTICE.txt delete mode 100644 solr/licenses/mrunit-1.0.0-hadoop2.jar.sha1 delete mode 100644 solr/licenses/mrunit-LICENSE-ASL.txt delete mode 100644 solr/licenses/mrunit-NOTICE.txt delete mode 100644 solr/licenses/netty-3.2.4.Final.jar.sha1 delete mode 100644 solr/licenses/netty-LICENSE-ASL.txt delete mode 100644 solr/licenses/netty-NOTICE.txt delete mode 100644 solr/licenses/paranamer-2.3.jar.sha1 delete mode 100644 solr/licenses/paranamer-LICENSE-BSD.txt delete mode 100644 solr/licenses/paranamer-NOTICE.txt delete mode 100644 solr/licenses/rome-1.6.1.jar.sha1 delete mode 100644 solr/licenses/snappy-java-1.0.5.jar.sha1 delete mode 100644 solr/licenses/snappy-java-LICENSE-ASL.txt delete mode 100644 solr/licenses/snappy-java-NOTICE.txt delete mode 100755 solr/server/scripts/map-reduce/set-map-reduce-classpath.sh diff --git a/.gitignore b/.gitignore index 625cfa97c80..4962ab95c9f 100644 --- a/.gitignore +++ b/.gitignore @@ -45,7 +45,6 @@ parent.iml /solr/example/example-DIH/solr/mail/lib/*.jar solr/contrib/dataimporthandler/test-lib/ -solr/contrib/morphlines-core/test-lib/ solr/core/test-lib/ diff --git a/dev-tools/idea/.idea/ant.xml b/dev-tools/idea/.idea/ant.xml index 8d454ad8bcb..8723e63de9e 100644 --- a/dev-tools/idea/.idea/ant.xml +++ b/dev-tools/idea/.idea/ant.xml @@ -46,9 +46,6 @@ - - - diff --git a/dev-tools/idea/.idea/libraries/Solr_morphlines_cell_library.xml b/dev-tools/idea/.idea/libraries/Solr_morphlines_cell_library.xml deleted file mode 100644 index cbc99073448..00000000000 --- a/dev-tools/idea/.idea/libraries/Solr_morphlines_cell_library.xml +++ /dev/null @@ -1,10 +0,0 @@ - - - - - - - - - - \ No newline at end of file diff --git a/dev-tools/idea/.idea/libraries/Solr_morphlines_core_library.xml b/dev-tools/idea/.idea/libraries/Solr_morphlines_core_library.xml deleted file mode 100644 index 4260f5816d2..00000000000 --- a/dev-tools/idea/.idea/libraries/Solr_morphlines_core_library.xml +++ /dev/null @@ -1,10 +0,0 @@ - - - - - - - - - - \ No newline at end of file diff --git a/dev-tools/idea/.idea/libraries/Solr_morphlines_core_test_library.xml b/dev-tools/idea/.idea/libraries/Solr_morphlines_core_test_library.xml deleted file mode 100644 index 74699f467dd..00000000000 --- a/dev-tools/idea/.idea/libraries/Solr_morphlines_core_test_library.xml +++ /dev/null @@ -1,10 +0,0 @@ - - - - - - - - - - \ No newline at end of file diff --git a/dev-tools/idea/.idea/modules.xml b/dev-tools/idea/.idea/modules.xml index d6b44f48500..7ad2a781442 100644 --- a/dev-tools/idea/.idea/modules.xml +++ b/dev-tools/idea/.idea/modules.xml @@ -56,9 +56,6 @@ - - - diff --git a/dev-tools/idea/.idea/workspace.xml b/dev-tools/idea/.idea/workspace.xml index 65be571e7fb..0ca7f0c2b70 100644 --- a/dev-tools/idea/.idea/workspace.xml +++ b/dev-tools/idea/.idea/workspace.xml @@ -316,30 +316,6 @@ - - - - - - - - - - - - - - - - + @@ -395,13 +371,10 @@ - - - - - - - + + + + diff --git a/dev-tools/idea/solr/contrib/map-reduce/map-reduce.iml b/dev-tools/idea/solr/contrib/map-reduce/map-reduce.iml deleted file mode 100644 index af266b01ffd..00000000000 --- a/dev-tools/idea/solr/contrib/map-reduce/map-reduce.iml +++ /dev/null @@ -1,43 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/dev-tools/idea/solr/contrib/morphlines-cell/morphlines-cell.iml b/dev-tools/idea/solr/contrib/morphlines-cell/morphlines-cell.iml deleted file mode 100644 index f6f3900eba1..00000000000 --- a/dev-tools/idea/solr/contrib/morphlines-cell/morphlines-cell.iml +++ /dev/null @@ -1,29 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/dev-tools/idea/solr/contrib/morphlines-core/morphlines-core.iml b/dev-tools/idea/solr/contrib/morphlines-core/morphlines-core.iml deleted file mode 100644 index 15f26f25dff..00000000000 --- a/dev-tools/idea/solr/contrib/morphlines-core/morphlines-core.iml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/dev-tools/maven/solr/contrib/map-reduce/pom.xml.template b/dev-tools/maven/solr/contrib/map-reduce/pom.xml.template deleted file mode 100644 index 623c8cb78c0..00000000000 --- a/dev-tools/maven/solr/contrib/map-reduce/pom.xml.template +++ /dev/null @@ -1,90 +0,0 @@ - - - 4.0.0 - - org.apache.solr - solr-parent - @version@ - ../../pom.xml - - org.apache.solr - solr-map-reduce - jar - Apache Solr map-reduce index construction - Apache Solr - map-reduce index construction - - solr/contrib/map-reduce - ../../../.. - ${relative-top-level}/${module-directory} - - - scm:git:${vc-anonymous-base-url} - scm:git:${vc-dev-base-url} - ${vc-browse-base-url};f=${module-directory} - - - - - - - org.apache.lucene - lucene-test-framework - test - - - org.apache.solr - solr-test-framework - test - - - org.apache.solr - solr-morphlines-core - ${project.version} - test-jar - test - -@solr-map-reduce.internal.dependencies@ -@solr-map-reduce.external.dependencies@ -@solr-map-reduce.internal.test.dependencies@ -@solr-map-reduce.external.test.dependencies@ - - - ${module-path}/src/java - ${module-path}/src/test - - - ${module-path}/src/test-files - - - - ${module-path}/../morphlines-core/src/test-files - - - ${top-level}/dev-tools/maven/solr - - maven.testlogging.properties - - - - - diff --git a/dev-tools/maven/solr/contrib/morphlines-cell/pom.xml.template b/dev-tools/maven/solr/contrib/morphlines-cell/pom.xml.template deleted file mode 100644 index 2849e90c17b..00000000000 --- a/dev-tools/maven/solr/contrib/morphlines-cell/pom.xml.template +++ /dev/null @@ -1,90 +0,0 @@ - - - 4.0.0 - - org.apache.solr - solr-parent - @version@ - ../../pom.xml - - org.apache.solr - solr-morphlines-cell - jar - Apache Solr Cell Morphlines - Apache Solr - Cell Morphlines - - solr/contrib/morphlines-cell - ../../../.. - ${relative-top-level}/${module-directory} - - - scm:git:${vc-anonymous-base-url} - scm:git:${vc-dev-base-url} - ${vc-browse-base-url};f=${module-directory} - - - - - - - org.apache.lucene - lucene-test-framework - test - - - org.apache.solr - solr-test-framework - test - - - org.apache.solr - solr-morphlines-core - ${project.version} - test-jar - test - -@solr-morphlines-cell.internal.dependencies@ -@solr-morphlines-cell.external.dependencies@ -@solr-morphlines-cell.internal.test.dependencies@ -@solr-morphlines-cell.external.test.dependencies@ - - - ${module-path}/src/java - ${module-path}/src/test - - - ${module-path}/src/test-files - - - - ${module-path}/../morphlines-core/src/test-files - - - ${top-level}/dev-tools/maven/solr - - maven.testlogging.properties - - - - - diff --git a/dev-tools/maven/solr/contrib/morphlines-core/pom.xml.template b/dev-tools/maven/solr/contrib/morphlines-core/pom.xml.template deleted file mode 100644 index a14707d1bbe..00000000000 --- a/dev-tools/maven/solr/contrib/morphlines-core/pom.xml.template +++ /dev/null @@ -1,91 +0,0 @@ - - - 4.0.0 - - org.apache.solr - solr-parent - @version@ - ../../pom.xml - - org.apache.solr - solr-morphlines-core - jar - Apache Solr Morphlines Core - Apache Solr - Morphlines Core - - solr/contrib/morphlines-core - ../../../.. - ${relative-top-level}/${module-directory} - - - scm:git:${vc-anonymous-base-url} - scm:git:${vc-dev-base-url} - ${vc-browse-base-url};f=${module-directory} - - - - - - - org.apache.lucene - lucene-test-framework - test - - - org.apache.solr - solr-test-framework - test - -@solr-morphlines-core.internal.dependencies@ -@solr-morphlines-core.external.dependencies@ -@solr-morphlines-core.internal.test.dependencies@ -@solr-morphlines-core.external.test.dependencies@ - - - ${module-path}/src/java - ${module-path}/src/test - - - ${module-path}/src/test-files - - - ${top-level}/dev-tools/maven/solr - - maven.testlogging.properties - - - - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - - diff --git a/dev-tools/maven/solr/contrib/pom.xml.template b/dev-tools/maven/solr/contrib/pom.xml.template index 20cbac5ec49..8159ecc934e 100644 --- a/dev-tools/maven/solr/contrib/pom.xml.template +++ b/dev-tools/maven/solr/contrib/pom.xml.template @@ -39,9 +39,6 @@ extraction langid ltr - morphlines-cell - morphlines-core - map-reduce uima velocity diff --git a/lucene/ivy-ignore-conflicts.properties b/lucene/ivy-ignore-conflicts.properties index 34b6bcab229..3e80311b443 100644 --- a/lucene/ivy-ignore-conflicts.properties +++ b/lucene/ivy-ignore-conflicts.properties @@ -10,6 +10,5 @@ # trigger a conflict) when the ant check-lib-versions target is run. /com.google.guava/guava = 16.0.1 -/com.google.inject/guice=4.0-beta5 /javax.servlet/servlet-api = 2.5, 3.0-alpha-1 /org.ow2.asm/asm = 5.0_BETA \ No newline at end of file diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties index 7f57fedddf8..6bd3f8d7b8b 100644 --- a/lucene/ivy-versions.properties +++ b/lucene/ivy-versions.properties @@ -3,7 +3,6 @@ # when the lexical sort check is performed by the ant check-lib-versions target. /antlr/antlr = 2.7.7 -/aopalliance/aopalliance = 1.0 /com.adobe.xmp/xmpcore = 5.1.2 com.carrotsearch.randomizedtesting.version = 2.5.0 @@ -26,10 +25,6 @@ com.fasterxml.jackson.core.version = 2.5.4 /com.google.guava/guava = 14.0.1 -com.google.inject.guice.version = 3.0 -/com.google.inject.extensions/guice-servlet = ${com.google.inject.guice.version} -/com.google.inject/guice = ${com.google.inject.guice.version} - /com.google.protobuf/protobuf-java = 3.1.0 /com.googlecode.juniversalchardet/juniversalchardet = 1.0.3 /com.googlecode.mp4parser/isoparser = 1.1.18 @@ -37,24 +32,14 @@ com.google.inject.guice.version = 3.0 /com.ibm.icu/icu4j = 56.1 /com.pff/java-libpst = 0.8.1 -com.rometools.version = 1.6.1 -/com.rometools/rome = ${com.rometools.version} - com.sun.jersey.version = 1.9 -/com.sun.jersey.contribs/jersey-guice = ${com.sun.jersey.version} -/com.sun.jersey/jersey-bundle = ${com.sun.jersey.version} /com.sun.jersey/jersey-core = ${com.sun.jersey.version} -/com.sun.jersey/jersey-json = ${com.sun.jersey.version} /com.sun.jersey/jersey-server = ${com.sun.jersey.version} /com.sun.mail/gimap = 1.5.1 /com.sun.mail/javax.mail = 1.5.1 -/com.sun.xml.bind/jaxb-impl = 2.2.3-1 - /com.tdunning/t-digest = 3.1 -/com.thoughtworks.paranamer/paranamer = 2.3 -/com.typesafe/config = 1.0.2 /commons-beanutils/commons-beanutils = 1.8.3 /commons-cli/commons-cli = 1.2 /commons-codec/commons-codec = 1.10 @@ -74,7 +59,6 @@ io.dropwizard.metrics.version = 3.1.2 /io.dropwizard.metrics/metrics-core = ${io.dropwizard.metrics.version} /io.dropwizard.metrics/metrics-ganglia = ${io.dropwizard.metrics.version} /io.dropwizard.metrics/metrics-graphite = ${io.dropwizard.metrics.version} -/io.dropwizard.metrics/metrics-healthchecks = ${io.dropwizard.metrics.version} /io.dropwizard.metrics/metrics-jetty9 = ${io.dropwizard.metrics.version} /io.dropwizard.metrics/metrics-jvm = ${io.dropwizard.metrics.version} @@ -82,7 +66,6 @@ io.netty.netty-all.version = 4.0.36.Final /io.netty/netty-all = ${io.netty.netty-all.version} /javax.activation/activation = 1.1.1 -/javax.inject/javax.inject= 1 /javax.servlet/javax.servlet-api = 3.1.0 /javax.servlet/servlet-api = 2.4 /jdom/jdom = 1.0 @@ -95,14 +78,11 @@ io.netty.netty-all.version = 4.0.36.Final /net.bytebuddy/byte-buddy = 1.6.2 /net.hydromatic/eigenbase-properties = 1.1.5 /net.sf.ehcache/ehcache-core = 2.4.4 -/net.sf.saxon/Saxon-HE = 9.6.0-2 -/net.sourceforge.argparse4j/argparse4j = 0.4.3 /net.sourceforge.jmatio/jmatio = 1.0 /net.sourceforge.nekohtml/nekohtml = 1.9.17 /org.antlr/antlr4-runtime = 4.5.1-1 /org.apache.ant/ant = 1.8.2 -/org.apache.avro/avro = 1.7.5 org.apache.calcite.avatica.version = 1.9.0 /org.apache.calcite.avatica/avatica-core = ${org.apache.calcite.avatica.version} @@ -160,23 +140,7 @@ org.apache.hadoop.version = 2.7.2 /org.apache.hadoop/hadoop-auth = ${org.apache.hadoop.version} /org.apache.hadoop/hadoop-common = ${org.apache.hadoop.version} /org.apache.hadoop/hadoop-hdfs = ${org.apache.hadoop.version} - -/org.apache.hadoop/hadoop-mapreduce-client-app = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-mapreduce-client-common = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-mapreduce-client-core = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-mapreduce-client-hs = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-mapreduce-client-jobclient = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-mapreduce-client-shuffle = ${org.apache.hadoop.version} /org.apache.hadoop/hadoop-minikdc = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-yarn-api = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-yarn-client = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-yarn-common = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-yarn-server-applicationhistoryservice = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-yarn-server-common = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-yarn-server-nodemanager = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-yarn-server-resourcemanager = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-yarn-server-tests = ${org.apache.hadoop.version} -/org.apache.hadoop/hadoop-yarn-server-web-proxy = ${org.apache.hadoop.version} /org.apache.htrace/htrace-core = 3.2.0-incubating @@ -193,7 +157,6 @@ org.apache.james.apache.mime4j.version = 0.7.2 /org.apache.james/apache-mime4j-dom = ${org.apache.james.apache.mime4j.version} /org.apache.mina/mina-core = 2.0.0-M5 -/org.apache.mrunit/mrunit = 1.0.0 org.apache.pdfbox.version = 2.0.1 /org.apache.pdfbox/fontbox = ${org.apache.pdfbox.version} @@ -228,7 +191,6 @@ org.apache.uima.version = 2.3.1 org.bouncycastle.version = 1.45 /org.bouncycastle/bcmail-jdk15 = ${org.bouncycastle.version} -/org.bouncycastle/bcpkix-jdk15on = 1.47 /org.bouncycastle/bcprov-jdk15 = ${org.bouncycastle.version} /org.carrot2.attributes/attributes-binder = 1.3.1 @@ -245,7 +207,6 @@ org.carrot2.morfologik.version = 2.1.1 org.codehaus.jackson.version = 1.9.13 /org.codehaus.jackson/jackson-core-asl = ${org.codehaus.jackson.version} -/org.codehaus.jackson/jackson-jaxrs = ${org.codehaus.jackson.version} /org.codehaus.jackson/jackson-mapper-asl = ${org.codehaus.jackson.version} org.codehaus.janino.version = 2.7.6 @@ -271,29 +232,10 @@ org.eclipse.jetty.version = 9.3.14.v20161028 /org.eclipse.jetty/jetty-webapp = ${org.eclipse.jetty.version} /org.eclipse.jetty/jetty-xml = ${org.eclipse.jetty.version} -/org.fusesource.leveldbjni/leveldbjni = 1.8 - org.gagravarr.vorbis.java.version = 0.8 /org.gagravarr/vorbis-java-core = ${org.gagravarr.vorbis.java.version} /org.gagravarr/vorbis-java-tika = ${org.gagravarr.vorbis.java.version} -org.iq80.leveldb.version = 0.7 -/org.iq80.leveldb/leveldb = ${org.iq80.leveldb.version} -/org.iq80.leveldb/leveldb-api = ${org.iq80.leveldb.version} - -org.jboss.netty.netty.version = 3.2.4.Final -/org.jboss.netty/netty = ${org.jboss.netty.netty.version} - -org.kitesdk.kite-morphlines.version = 1.1.0 -/org.kitesdk/kite-morphlines-avro = ${org.kitesdk.kite-morphlines.version} -/org.kitesdk/kite-morphlines-core = ${org.kitesdk.kite-morphlines.version} -/org.kitesdk/kite-morphlines-hadoop-sequencefile = ${org.kitesdk.kite-morphlines.version} -/org.kitesdk/kite-morphlines-json = ${org.kitesdk.kite-morphlines.version} -/org.kitesdk/kite-morphlines-saxon = ${org.kitesdk.kite-morphlines.version} -/org.kitesdk/kite-morphlines-tika-core = ${org.kitesdk.kite-morphlines.version} -/org.kitesdk/kite-morphlines-tika-decompress = ${org.kitesdk.kite-morphlines.version} -/org.kitesdk/kite-morphlines-twitter = ${org.kitesdk.kite-morphlines.version} - /org.locationtech.spatial4j/spatial4j = 0.6 /org.mockito/mockito-core = 2.6.2 @@ -322,7 +264,6 @@ org.slf4j.version = 1.7.7 /org.slf4j/slf4j-log4j12 = ${org.slf4j.version} /org.tukaani/xz = 1.5 -/org.xerial.snappy/snappy-java = 1.0.5 /rome/rome = 1.0 /xerces/xercesImpl = 2.9.1 diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index e74e4ae3be0..428bde46afa 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -104,6 +104,11 @@ Apache UIMA 2.3.1 Apache ZooKeeper 3.4.6 Jetty 9.3.14.v20161028 +Upgrade Notes +---------------------- + +* Solr contribs map-reduce, morphlines-core and morphlines-cell have been removed. + Detailed Change List ---------------------- @@ -128,6 +133,11 @@ Bug Fixes * SOLR-10281: ADMIN_PATHS is duplicated in two places and inconsistent. This can cause automatic retries to /admin/metrics handler by the CloudSolrClient. (shalin) +Other Changes +---------------------- + +* SOLR-9221: Remove Solr contribs: map-reduce, morphlines-core and morphlines-cell. (Steve Rowe) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/contrib/map-reduce/README.txt b/solr/contrib/map-reduce/README.txt deleted file mode 100644 index a655aca7663..00000000000 --- a/solr/contrib/map-reduce/README.txt +++ /dev/null @@ -1,20 +0,0 @@ -Apache Solr MapReduce - -*Experimental* - This contrib is currently subject to change in ways that may -break back compatibility. - -The Solr MapReduce contrib provides an a mapreduce job that allows you to build -Solr indexes and optionally merge them into a live Solr cluster. - -Example: - -# Build an index with map-reduce and deploy it to SolrCloud - -source $solr_distrib/example/scripts/map-reduce/set-map-reduce-classpath.sh - -$hadoop_distrib/bin/hadoop --config $hadoop_conf_dir jar \ -$solr_distrib/dist/solr-map-reduce-*.jar -D 'mapred.child.java.opts=-Xmx500m' \ --libjars "$HADOOP_LIBJAR" --morphline-file readAvroContainer.conf \ ---zk-host 127.0.0.1:9983 --output-dir hdfs://127.0.0.1:8020/outdir \ ---collection $collection --log4j log4j.properties --go-live \ ---verbose "hdfs://127.0.0.1:8020/indir" \ No newline at end of file diff --git a/solr/contrib/map-reduce/build.xml b/solr/contrib/map-reduce/build.xml deleted file mode 100644 index d6aefc79253..00000000000 --- a/solr/contrib/map-reduce/build.xml +++ /dev/null @@ -1,157 +0,0 @@ - - - - - - - - Solr map-reduce index construction. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/map-reduce/ivy.xml b/solr/contrib/map-reduce/ivy.xml deleted file mode 100644 index d2e158ec3b5..00000000000 --- a/solr/contrib/map-reduce/ivy.xml +++ /dev/null @@ -1,37 +0,0 @@ - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/map-reduce/src/java/assembly/hadoop-job.xml b/solr/contrib/map-reduce/src/java/assembly/hadoop-job.xml deleted file mode 100644 index 1640b6ff72e..00000000000 --- a/solr/contrib/map-reduce/src/java/assembly/hadoop-job.xml +++ /dev/null @@ -1,39 +0,0 @@ - - - - - - job - - jar - - false - - - false - runtime - lib - - ${groupId}:${artifactId} - - - - true - - ${groupId}:${artifactId} - - - - diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/AlphaNumericComparator.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/AlphaNumericComparator.java deleted file mode 100644 index f596a880067..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/AlphaNumericComparator.java +++ /dev/null @@ -1,75 +0,0 @@ -//The MIT License -// -// Copyright (c) 2003 Ron Alford, Mike Grove, Bijan Parsia, Evren Sirin -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to -// deal in the Software without restriction, including without limitation the -// rights to use, copy, modify, merge, publish, distribute, sublicense, and/or -// sell copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING -// FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS -// IN THE SOFTWARE. -package org.apache.solr.hadoop; - -import java.util.Comparator; - -/** - * This is a comparator to perform a mix of alphabetical+numeric comparison. For - * example, if there is a list {"test10", "test2", "test150", "test25", "test1"} - * then what we generally expect from the ordering is the result {"test1", - * "test2", "test10", "test25", "test150"}. However, standard lexigraphic - * ordering does not do that and "test10" comes before "test2". This class is - * provided to overcome that problem. This functionality is useful to sort the - * benchmark files (like the ones in in DL-benchmark-suite) from smallest to the - * largest. Comparisons are done on the String values retuned by toString() so - * care should be taken when this comparator is used to sort arbitrary Java - * objects. - * - */ -final class AlphaNumericComparator implements Comparator { - - public AlphaNumericComparator() { - } - - public int compare(Object o1, Object o2) { - String s1 = o1.toString(); - String s2 = o2.toString(); - int n1 = s1.length(), n2 = s2.length(); - int i1 = 0, i2 = 0; - while (i1 < n1 && i2 < n2) { - int p1 = i1; - int p2 = i2; - char c1 = s1.charAt(i1++); - char c2 = s2.charAt(i2++); - if(c1 != c2) { - if (Character.isDigit(c1) && Character.isDigit(c2)) { - int value1 = 0, value2 = 0; - while (i1 < n1 && Character.isDigit(c1 = s1.charAt(i1))) { - i1++; - } - value1 = Integer.parseInt(s1.substring(p1, i1)); - while (i2 < n2 && Character.isDigit(c2 = s2.charAt(i2))) { - i2++; - } - value2 = Integer.parseInt(s2.substring(p2, i2)); - if (value1 != value2) { - return value1 - value2; - } - } - return c1 - c2; - } - } - - return n1 - n2; - } -} \ No newline at end of file diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/BatchWriter.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/BatchWriter.java deleted file mode 100644 index 24458d5622f..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/BatchWriter.java +++ /dev/null @@ -1,243 +0,0 @@ -/* - * 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.hadoop; - -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskID; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer; -import org.apache.solr.client.solrj.response.UpdateResponse; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.common.util.ExecutorUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Locale; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Enables adding batches of documents to an EmbeddedSolrServer. - */ -class BatchWriter { - - private final EmbeddedSolrServer solr; - private volatile Exception batchWriteException = null; - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public Exception getBatchWriteException() { - return batchWriteException; - } - - public void setBatchWriteException(Exception batchWriteException) { - this.batchWriteException = batchWriteException; - } - - /** The number of writing threads. */ - final int writerThreads; - - /** Queue Size */ - final int queueSize; - - private final ThreadPoolExecutor batchPool; - - private TaskID taskId = null; - - /** - * The number of in progress batches, must be zero before the close can - * actually start closing - */ - AtomicInteger executingBatches = new AtomicInteger(0); - - /** - * Create the batch writer object, set the thread to daemon mode, and start - * it. - * - */ - - final class Batch implements Runnable { - - private List documents; - private UpdateResponse result; - - public Batch(Collection batch) { - documents = new ArrayList<>(batch); - } - - public void run() { - try { - executingBatches.getAndIncrement(); - result = runUpdate(documents); - } finally { - executingBatches.getAndDecrement(); - } - } - - protected List getDocuments() { - return documents; - } - - protected void setDocuments(List documents) { - this.documents = documents; - } - - protected UpdateResponse getResult() { - return result; - } - - protected void setResult(UpdateResponse result) { - this.result = result; - } - - protected void reset(List documents) { - if (this.documents == null) { - this.documents = new ArrayList<>(documents); - } else { - this.documents.clear(); - this.documents.addAll(documents); - } - result = null; - } - - protected void reset(SolrInputDocument document) { - if (this.documents == null) { - this.documents = new ArrayList<>(); - } else { - this.documents.clear(); - } - this.documents.add(document); - result = null; - } - } - - protected UpdateResponse runUpdate(List batchToWrite) { - try { - UpdateResponse result = solr.add(batchToWrite); - SolrRecordWriter.incrementCounter(taskId, SolrCounters.class.getName(), SolrCounters.BATCHES_WRITTEN.toString(), 1); - SolrRecordWriter.incrementCounter(taskId, SolrCounters.class.getName(), SolrCounters.DOCUMENTS_WRITTEN.toString(), batchToWrite.size()); - if (LOG.isDebugEnabled()) { - SolrRecordWriter.incrementCounter(taskId, SolrCounters.class.getName(), SolrCounters.BATCH_WRITE_TIME.toString(), result.getElapsedTime()); - } - return result; - } catch (Throwable e) { - if (e instanceof Exception) { - setBatchWriteException((Exception) e); - } else { - setBatchWriteException(new Exception(e)); - } - SolrRecordWriter.incrementCounter(taskId, getClass().getName() + ".errors", e.getClass().getName(), 1); - LOG.error("Unable to process batch", e); - return null; - } - } - - - public BatchWriter(EmbeddedSolrServer solr, int batchSize, TaskID tid, - int writerThreads, int queueSize) { - this.solr = solr; - this.writerThreads = writerThreads; - this.queueSize = queueSize; - taskId = tid; - - // we need to obtain the settings before the constructor - if (writerThreads != 0) { - batchPool = new ExecutorUtil.MDCAwareThreadPoolExecutor(writerThreads, writerThreads, 5, - TimeUnit.SECONDS, new LinkedBlockingQueue(queueSize), - new ThreadPoolExecutor.CallerRunsPolicy()); - } else { // single threaded case - batchPool = null; - } - } - - public void queueBatch(Collection batch) - throws IOException, SolrServerException { - - throwIf(); - Batch b = new Batch(batch); - if (batchPool != null) { - batchPool.execute(b); - } else { // single threaded case - b.run(); - throwIf(); - } - } - - public synchronized void close(TaskAttemptContext context) - throws InterruptedException, SolrServerException, IOException { - - if (batchPool != null) { - context.setStatus("Waiting for batches to complete"); - batchPool.shutdown(); - - while (!batchPool.isTerminated()) { - LOG.info(String.format(Locale.ENGLISH, - "Waiting for %d items and %d threads to finish executing", batchPool - .getQueue().size(), batchPool.getActiveCount())); - batchPool.awaitTermination(5, TimeUnit.SECONDS); - } - } - context.setStatus("Committing Solr Phase 1"); - solr.commit(true, false); - context.setStatus("Optimizing Solr"); - int maxSegments = context.getConfiguration().getInt(SolrOutputFormat.SOLR_RECORD_WRITER_MAX_SEGMENTS, 1); - LOG.info("Optimizing Solr: forcing merge down to {} segments", maxSegments); - long start = System.nanoTime(); - solr.optimize(true, false, maxSegments); - context.getCounter(SolrCounters.class.getName(), SolrCounters.PHYSICAL_REDUCER_MERGE_TIME.toString()).increment(System.nanoTime() - start); - float secs = (System.nanoTime() - start) / (float)(10^9); - LOG.info("Optimizing Solr: done forcing merge down to {} segments in {} secs", maxSegments, secs); - context.setStatus("Committing Solr Phase 2"); - solr.commit(true, false); - context.setStatus("Shutting down Solr"); - solr.close(); - } - - /** - * Throw a legal exception if a previous batch write had an exception. The - * previous state is cleared. Uses {@link #batchWriteException} for the state - * from the last exception. - * - * This will loose individual exceptions if the exceptions happen rapidly. - * - * @throws IOException On low level IO error - * @throws SolrServerException On Solr Exception - */ - private void throwIf() throws IOException, SolrServerException { - - final Exception last = batchWriteException; - batchWriteException = null; - - if (last == null) { - return; - } - if (last instanceof SolrServerException) { - throw (SolrServerException) last; - } - if (last instanceof IOException) { - throw (IOException) last; - } - throw new IOException("Batch Write Failure", last); - } -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DataInputInputStream.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DataInputInputStream.java deleted file mode 100644 index d1d46bce416..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DataInputInputStream.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.hadoop; - -import java.io.DataInput; -import java.io.IOException; -import java.io.InputStream; - -import org.apache.hadoop.classification.InterfaceAudience; - -/** - * An InputStream that wraps a DataInput. - * @see DataOutputOutputStream - */ -@InterfaceAudience.Private -public class DataInputInputStream extends InputStream { - - private DataInput in; - - /** - * Construct an InputStream from the given DataInput. If 'in' - * is already an InputStream, simply returns it. Otherwise, wraps - * it in an InputStream. - * @param in the DataInput to wrap - * @return an InputStream instance that reads from 'in' - */ - public static InputStream constructInputStream(DataInput in) { - if (in instanceof InputStream) { - return (InputStream)in; - } else { - return new DataInputInputStream(in); - } - } - - - public DataInputInputStream(DataInput in) { - this.in = in; - } - - @Override - public int read() throws IOException { - return in.readUnsignedByte(); - } -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DataOutputOutputStream.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DataOutputOutputStream.java deleted file mode 100644 index 389c52a577d..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DataOutputOutputStream.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.hadoop; - -import java.io.DataOutput; -import java.io.IOException; -import java.io.OutputStream; - -import org.apache.hadoop.classification.InterfaceAudience; - -/** - * OutputStream implementation that wraps a DataOutput. - */ -@InterfaceAudience.Private -public class DataOutputOutputStream extends OutputStream { - - private final DataOutput out; - - /** - * Construct an OutputStream from the given DataOutput. If 'out' - * is already an OutputStream, simply returns it. Otherwise, wraps - * it in an OutputStream. - * @param out the DataOutput to wrap - * @return an OutputStream instance that outputs to 'out' - */ - public static OutputStream constructOutputStream(DataOutput out) { - if (out instanceof OutputStream) { - return (OutputStream)out; - } else { - return new DataOutputOutputStream(out); - } - } - - private DataOutputOutputStream(DataOutput out) { - this.out = out; - } - - @Override - public void write(int b) throws IOException { - out.writeByte(b); - } - - @Override - public void write(byte[] b, int off, int len) throws IOException { - out.write(b, off, len); - } - - @Override - public void write(byte[] b) throws IOException { - out.write(b); - } -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DryRunDocumentLoader.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DryRunDocumentLoader.java deleted file mode 100644 index bacf1d0e1fc..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/DryRunDocumentLoader.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.hadoop; - -import org.apache.solr.client.solrj.response.SolrPingResponse; -import org.apache.solr.client.solrj.response.UpdateResponse; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.morphlines.solr.DocumentLoader; - -/** - * Prints documents to stdout instead of loading them into Solr for quicker turnaround during early - * trial & debug sessions. - */ -final class DryRunDocumentLoader implements DocumentLoader { - - @Override - public void beginTransaction() { - } - - @Override - public void load(SolrInputDocument doc) { - System.out.println("dryrun: " + doc); - } - - @Override - public void commitTransaction() { - } - - @Override - public UpdateResponse rollbackTransaction() { - return new UpdateResponse(); - } - - @Override - public void shutdown() { - } - - @Override - public SolrPingResponse ping() { - return new SolrPingResponse(); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/GoLive.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/GoLive.java deleted file mode 100644 index 5b1c343ed68..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/GoLive.java +++ /dev/null @@ -1,182 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletionService; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.impl.CloudSolrClient; -import org.apache.solr.client.solrj.impl.HttpSolrClient; -import org.apache.solr.client.solrj.request.CoreAdminRequest; -import org.apache.solr.common.util.ExecutorUtil; -import org.apache.solr.hadoop.MapReduceIndexerTool.Options; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * The optional (parallel) GoLive phase merges the output shards of the previous - * phase into a set of live customer facing Solr servers, typically a SolrCloud. - */ -class GoLive { - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - // TODO: handle clusters with replicas - public boolean goLive(Options options, FileStatus[] outDirs) { - LOG.info("Live merging of output shards into Solr cluster..."); - boolean success = false; - long start = System.nanoTime(); - int concurrentMerges = options.goLiveThreads; - ThreadPoolExecutor executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(concurrentMerges, - concurrentMerges, 1, TimeUnit.SECONDS, - new LinkedBlockingQueue()); - - try { - CompletionService completionService = new ExecutorCompletionService<>(executor); - Set> pending = new HashSet<>(); - int cnt = -1; - for (final FileStatus dir : outDirs) { - - LOG.debug("processing: " + dir.getPath()); - - cnt++; - List urls = options.shardUrls.get(cnt); - - for (String url : urls) { - - String baseUrl = url; - if (baseUrl.endsWith("/")) { - baseUrl = baseUrl.substring(0, baseUrl.length() - 1); - } - - int lastPathIndex = baseUrl.lastIndexOf("/"); - if (lastPathIndex == -1) { - LOG.error("Found unexpected shardurl, live merge failed: " + baseUrl); - return false; - } - - final String name = baseUrl.substring(lastPathIndex + 1); - baseUrl = baseUrl.substring(0, lastPathIndex); - final String mergeUrl = baseUrl; - - Callable task = () -> { - Request req = new Request(); - LOG.info("Live merge " + dir.getPath() + " into " + mergeUrl); - try (final HttpSolrClient client = new HttpSolrClient.Builder(mergeUrl).build()) { - CoreAdminRequest.MergeIndexes mergeRequest = new CoreAdminRequest.MergeIndexes(); - mergeRequest.setCoreName(name); - mergeRequest.setIndexDirs(Arrays.asList(dir.getPath().toString() + "/data/index")); - mergeRequest.process(client); - req.success = true; - } catch (SolrServerException | IOException e) { - req.e = e; - } - return req; - }; - pending.add(completionService.submit(task)); - } - } - - while (pending != null && pending.size() > 0) { - try { - Future future = completionService.take(); - if (future == null) break; - pending.remove(future); - - try { - Request req = future.get(); - - if (!req.success) { - // failed - LOG.error("A live merge command failed", req.e); - return false; - } - - } catch (ExecutionException e) { - LOG.error("Error sending live merge command", e); - return false; - } - - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.error("Live merge process interrupted", e); - return false; - } - } - - cnt = -1; - - - try { - LOG.info("Committing live merge..."); - if (options.zkHost != null) { - try (CloudSolrClient server = new CloudSolrClient.Builder().withZkHost(options.zkHost).build()) { - server.setDefaultCollection(options.collection); - server.commit(); - } - } else { - for (List urls : options.shardUrls) { - for (String url : urls) { - // TODO: we should do these concurrently - try (HttpSolrClient server = new HttpSolrClient.Builder(url).build()) { - server.commit(); - } - } - } - } - LOG.info("Done committing live merge"); - } catch (Exception e) { - LOG.error("Error sending commits to live Solr cluster", e); - return false; - } - - success = true; - return true; - } finally { - ExecutorUtil.shutdownAndAwaitTermination(executor); - float secs = (System.nanoTime() - start) / (float)(10^9); - LOG.info("Live merging of index shards into Solr cluster took " + secs + " secs"); - if (success) { - LOG.info("Live merging completed successfully"); - } else { - LOG.info("Live merging failed"); - } - } - - // if an output dir does not exist, we should fail and do no merge? - } - - private static final class Request { - Exception e; - boolean success = false; - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/HdfsFileFieldNames.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/HdfsFileFieldNames.java deleted file mode 100644 index c9eaef6c9e9..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/HdfsFileFieldNames.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.hadoop; - - -/** - * Solr field names for metadata of an HDFS file. - */ -public interface HdfsFileFieldNames { - - public static final String FILE_UPLOAD_URL = "file_upload_url"; - public static final String FILE_DOWNLOAD_URL = "file_download_url"; - public static final String FILE_SCHEME = "file_scheme"; - public static final String FILE_HOST = "file_host"; - public static final String FILE_PORT = "file_port"; - public static final String FILE_PATH = "file_path"; - public static final String FILE_NAME = "file_name"; - public static final String FILE_LENGTH = "file_length"; - public static final String FILE_LAST_MODIFIED = "file_last_modified"; - public static final String FILE_OWNER = "file_owner"; - public static final String FILE_GROUP = "file_group"; - public static final String FILE_PERMISSIONS_USER = "file_permissions_user"; - public static final String FILE_PERMISSIONS_GROUP = "file_permissions_group"; - public static final String FILE_PERMISSIONS_OTHER = "file_permissions_other"; - public static final String FILE_PERMISSIONS_STICKYBIT = "file_permissions_stickybit"; - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/HeartBeater.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/HeartBeater.java deleted file mode 100644 index 66cd3dc4434..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/HeartBeater.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.hadoop; - -import java.lang.invoke.MethodHandles; -import java.util.Locale; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.mapreduce.TaskInputOutputContext; -import org.apache.hadoop.util.Progressable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This class runs a background thread that once every 60 seconds checks to see if - * a progress report is needed. If a report is needed it is issued. - * - * A simple counter {@link #threadsNeedingHeartBeat} handles the number of - * threads requesting a heart beat. - * - * The expected usage pattern is - * - *
- *  try {
- *       heartBeater.needHeartBeat();
- *       do something that may take a while
- *    } finally {
- *       heartBeater.cancelHeartBeat();
- *    }
- * 
- * - * - */ -public class HeartBeater extends Thread { - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - /** - * count of threads asking for heart beat, at 0 no heart beat done. This could - * be an atomic long but then missmatches in need/cancel could result in - * negative counts. - */ - private volatile int threadsNeedingHeartBeat = 0; - - private Progressable progress; - - /** - * The amount of time to wait between checks for the need to issue a heart - * beat. In milliseconds. - */ - private final long waitTimeMs = TimeUnit.MILLISECONDS.convert(60, TimeUnit.SECONDS); - - private final CountDownLatch isClosing = new CountDownLatch(1); - - /** - * Create the heart beat object thread set it to daemon priority and start the - * thread. When the count in {@link #threadsNeedingHeartBeat} is positive, the - * heart beat will be issued on the progress object every 60 seconds. - */ - public HeartBeater(Progressable progress) { - setDaemon(true); - this.progress = progress; - LOG.info("Heart beat reporting class is " + progress.getClass().getName()); - start(); - } - - public Progressable getProgress() { - return progress; - } - - public void setProgress(Progressable progress) { - this.progress = progress; - } - - @Override - public void run() { - LOG.info("HeartBeat thread running"); - while (true) { - try { - synchronized (this) { - if (threadsNeedingHeartBeat > 0) { - progress.progress(); - if (LOG.isInfoEnabled()) { - LOG.info(String.format(Locale.ENGLISH, "Issuing heart beat for %d threads", - threadsNeedingHeartBeat)); - } - } else { - if (LOG.isInfoEnabled()) { - LOG.info(String.format(Locale.ENGLISH, "heartbeat skipped count %d", - threadsNeedingHeartBeat)); - } - } - } - if (isClosing.await(waitTimeMs, TimeUnit.MILLISECONDS)) { - return; - } - } catch (Throwable e) { - LOG.error("HeartBeat throwable", e); - } - } - } - - /** - * inform the background thread that heartbeats are to be issued. Issue a - * heart beat also - */ - public synchronized void needHeartBeat() { - threadsNeedingHeartBeat++; - // Issue a progress report right away, - // just in case the the cancel comes before the background thread issues a - // report. - // If enough cases like this happen the 600 second timeout can occur - progress.progress(); - if (threadsNeedingHeartBeat == 1) { - // this.notify(); // wake up the heartbeater - } - } - - /** - * inform the background thread that this heartbeat request is not needed. - * This must be called at some point after each {@link #needHeartBeat()} - * request. - */ - public synchronized void cancelHeartBeat() { - if (threadsNeedingHeartBeat > 0) { - threadsNeedingHeartBeat--; - } else { - Exception e = new Exception("Dummy"); - e.fillInStackTrace(); - LOG.warn("extra call to cancelHeartBeat", e); - } - } - - public void setStatus(String status) { - if (progress instanceof TaskInputOutputContext) { - ((TaskInputOutputContext) progress).setStatus(status); - } - } - - /** Releases any resources */ - public void close() { - isClosing.countDown(); - } -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/LineRandomizerMapper.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/LineRandomizerMapper.java deleted file mode 100644 index 6f2638c0019..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/LineRandomizerMapper.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.Random; - -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Mapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * MR Mapper that randomizing a list of URLs. - * - * Mapper input is (offset, URL) pairs. Each such pair indicates a file to - * index. - * - * Mapper output is (randomPosition, URL) pairs. The reducer receives these - * pairs sorted by randomPosition. - */ -public class LineRandomizerMapper extends Mapper { - - private Random random; - - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - @Override - protected void setup(Context context) throws IOException, InterruptedException { - super.setup(context); - random = createRandom(context); - } - - @Override - protected void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException { - LOGGER.debug("map key: {}, value: {}", key, value); - context.write(new LongWritable(random.nextLong()), value); - } - - private Random createRandom(Context context) { - long taskId = 0; - if (context.getTaskAttemptID() != null) { // MRUnit returns null - LOGGER.debug("context.getTaskAttemptID().getId(): {}", context.getTaskAttemptID().getId()); - LOGGER.debug("context.getTaskAttemptID().getTaskID().getId(): {}", context.getTaskAttemptID().getTaskID().getId()); - taskId = context.getTaskAttemptID().getTaskID().getId(); // taskId = 0, 1, ..., N - } - // create a good random seed, yet ensure deterministic PRNG sequence for easy reproducability - return new Random(421439783L * (taskId + 1)); - } - -} \ No newline at end of file diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/LineRandomizerReducer.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/LineRandomizerReducer.java deleted file mode 100644 index cbe194a526f..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/LineRandomizerReducer.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; - -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Reducer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * MR Reducer that randomizing a list of URLs. - * - * Reducer input is (randomPosition, URL) pairs. Each such pair indicates a file - * to index. - * - * Reducer output is a list of URLs, each URL in a random position. - */ -public class LineRandomizerReducer extends Reducer { - - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - @Override - protected void reduce(LongWritable key, Iterable values, Context context) throws IOException, InterruptedException { - for (Text value : values) { - LOGGER.debug("reduce key: {}, value: {}", key, value); - context.write(value, NullWritable.get()); - } - } -} \ No newline at end of file diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/MapReduceIndexerTool.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/MapReduceIndexerTool.java deleted file mode 100644 index 1c1785576f1..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/MapReduceIndexerTool.java +++ /dev/null @@ -1,1388 +0,0 @@ -/* - * 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.hadoop; - - -import java.io.BufferedInputStream; -import java.io.BufferedReader; -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.OutputStreamWriter; -import java.io.Writer; -import java.lang.invoke.MethodHandles; -import java.net.URISyntaxException; -import java.net.URL; -import java.net.URLClassLoader; -import java.nio.charset.StandardCharsets; -import java.text.NumberFormat; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Random; -import net.sourceforge.argparse4j.ArgumentParsers; -import net.sourceforge.argparse4j.impl.Arguments; -import net.sourceforge.argparse4j.impl.action.HelpArgumentAction; -import net.sourceforge.argparse4j.impl.choice.RangeArgumentChoice; -import net.sourceforge.argparse4j.impl.type.FileArgumentType; -import net.sourceforge.argparse4j.inf.Argument; -import net.sourceforge.argparse4j.inf.ArgumentGroup; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.ArgumentParserException; -import net.sourceforge.argparse4j.inf.FeatureControl; -import net.sourceforge.argparse4j.inf.Namespace; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.JobClient; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.lib.input.NLineInputFormat; -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; -import org.apache.hadoop.util.GenericOptionsParser; -import org.apache.hadoop.util.Tool; -import org.apache.hadoop.util.ToolRunner; -import org.apache.solr.common.cloud.SolrZkClient; -import org.apache.solr.hadoop.dedup.RetainMostRecentUpdateConflictResolver; -import org.apache.solr.hadoop.morphline.MorphlineMapRunner; -import org.apache.solr.hadoop.morphline.MorphlineMapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.kitesdk.morphline.base.Fields; - -import com.google.common.base.Preconditions; -import com.google.common.io.ByteStreams; - - -/** - * Public API for a MapReduce batch job driver that creates a set of Solr index shards from a set of - * input files and writes the indexes into HDFS, in a flexible, scalable and fault-tolerant manner. - * Also supports merging the output shards into a set of live customer facing Solr servers, - * typically a SolrCloud. - */ -public class MapReduceIndexerTool extends Configured implements Tool { - - Job job; // visible for testing only - - public static final String RESULTS_DIR = "results"; - - static final String MAIN_MEMORY_RANDOMIZATION_THRESHOLD = - MapReduceIndexerTool.class.getName() + ".mainMemoryRandomizationThreshold"; - - private static final String FULL_INPUT_LIST = "full-input-list.txt"; - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - - /** - * See http://argparse4j.sourceforge.net and for details see http://argparse4j.sourceforge.net/usage.html - */ - static final class MyArgumentParser { - - private static final String SHOW_NON_SOLR_CLOUD = "--show-non-solr-cloud"; - - private boolean showNonSolrCloud = false; - - /** - * Parses the given command line arguments. - * - * @return exitCode null indicates the caller shall proceed with processing, - * non-null indicates the caller shall exit the program with the - * given exit status code. - */ - public Integer parseArgs(String[] args, Configuration conf, Options opts) { - assert args != null; - assert conf != null; - assert opts != null; - - if (args.length == 0) { - args = new String[] { "--help" }; - } - - showNonSolrCloud = Arrays.asList(args).contains(SHOW_NON_SOLR_CLOUD); // intercept it first - - ArgumentParser parser = ArgumentParsers - .newArgumentParser("hadoop [GenericOptions]... jar solr-map-reduce-*.jar ", false) - .defaultHelp(true) - .description( - "MapReduce batch job driver that takes a morphline and creates a set of Solr index shards from a set of input files " + - "and writes the indexes into HDFS, in a flexible, scalable and fault-tolerant manner. " + - "It also supports merging the output shards into a set of live customer facing Solr servers, " + - "typically a SolrCloud. The program proceeds in several consecutive MapReduce based phases, as follows:" + - "\n\n" + - "1) Randomization phase: This (parallel) phase randomizes the list of input files in order to spread " + - "indexing load more evenly among the mappers of the subsequent phase." + - "\n\n" + - "2) Mapper phase: This (parallel) phase takes the input files, extracts the relevant content, transforms it " + - "and hands SolrInputDocuments to a set of reducers. " + - "The ETL functionality is flexible and " + - "customizable using chains of arbitrary morphline commands that pipe records from one transformation command to another. " + - "Commands to parse and transform a set of standard data formats such as Avro, CSV, Text, HTML, XML, " + - "PDF, Word, Excel, etc. are provided out of the box, and additional custom commands and parsers for additional " + - "file or data formats can be added as morphline plugins. " + - "This is done by implementing a simple Java interface that consumes a record (e.g. a file in the form of an InputStream " + - "plus some headers plus contextual metadata) and generates as output zero or more records. " + - "Any kind of data format can be indexed and any Solr documents for any kind of Solr schema can be generated, " + - "and any custom ETL logic can be registered and executed.\n" + - "Record fields, including MIME types, can also explicitly be passed by force from the CLI to the morphline, for example: " + - "hadoop ... -D " + MorphlineMapRunner.MORPHLINE_FIELD_PREFIX + Fields.ATTACHMENT_MIME_TYPE + "=text/csv" + - "\n\n" + - "3) Reducer phase: This (parallel) phase loads the mapper's SolrInputDocuments into one EmbeddedSolrServer per reducer. " + - "Each such reducer and Solr server can be seen as a (micro) shard. The Solr servers store their " + - "data in HDFS." + - "\n\n" + - "4) Mapper-only merge phase: This (parallel) phase merges the set of reducer shards into the number of solr " + - "shards expected by the user, using a mapper-only job. This phase is omitted if the number " + - "of shards is already equal to the number of shards expected by the user. " + - "\n\n" + - "5) Go-live phase: This optional (parallel) phase merges the output shards of the previous phase into a set of " + - "live customer facing Solr servers, typically a SolrCloud. " + - "If this phase is omitted you can explicitly point each Solr server to one of the HDFS output shard directories." + - "\n\n" + - "Fault Tolerance: Mapper and reducer task attempts are retried on failure per the standard MapReduce semantics. " + - "On program startup all data in the --output-dir is deleted if that output directory already exists. " + - "If the whole job fails you can retry simply by rerunning the program again using the same arguments." - ); - - parser.addArgument("--help", "-help", "-h") - .help("Show this help message and exit") - .action(new HelpArgumentAction() { - @Override - public void run(ArgumentParser parser, Argument arg, Map attrs, String flag, Object value) throws ArgumentParserException { - parser.printHelp(); - System.out.println(); - System.out.print(ToolRunnerHelpFormatter.getGenericCommandUsage()); - //ToolRunner.printGenericCommandUsage(System.out); - System.out.println( - "Examples: \n\n" + - - "# (Re)index an Avro based Twitter tweet file:\n" + - "sudo -u hdfs hadoop \\\n" + - " --config /etc/hadoop/conf.cloudera.mapreduce1 \\\n" + - " jar target/solr-map-reduce-*.jar \\\n" + - " -D 'mapred.child.java.opts=-Xmx500m' \\\n" + -// " -D 'mapreduce.child.java.opts=-Xmx500m' \\\n" + - " --log4j src/test/resources/log4j.properties \\\n" + - " --morphline-file ../search-core/src/test/resources/test-morphlines/tutorialReadAvroContainer.conf \\\n" + - " --solr-home-dir src/test/resources/solr/minimr \\\n" + - " --output-dir hdfs://c2202.mycompany.com/user/$USER/test \\\n" + - " --shards 1 \\\n" + - " hdfs:///user/$USER/test-documents/sample-statuses-20120906-141433.avro\n" + - "\n" + - "# Go live by merging resulting index shards into a live Solr cluster\n" + - "# (explicitly specify Solr URLs - for a SolrCloud cluster see next example):\n" + - "sudo -u hdfs hadoop \\\n" + - " --config /etc/hadoop/conf.cloudera.mapreduce1 \\\n" + - " jar target/solr-map-reduce-*.jar \\\n" + - " -D 'mapred.child.java.opts=-Xmx500m' \\\n" + -// " -D 'mapreduce.child.java.opts=-Xmx500m' \\\n" + - " --log4j src/test/resources/log4j.properties \\\n" + - " --morphline-file ../search-core/src/test/resources/test-morphlines/tutorialReadAvroContainer.conf \\\n" + - " --solr-home-dir src/test/resources/solr/minimr \\\n" + - " --output-dir hdfs://c2202.mycompany.com/user/$USER/test \\\n" + - " --shard-url http://solr001.mycompany.com:8983/solr/collection1 \\\n" + - " --shard-url http://solr002.mycompany.com:8983/solr/collection1 \\\n" + - " --go-live \\\n" + - " hdfs:///user/foo/indir\n" + - "\n" + - "# Go live by merging resulting index shards into a live SolrCloud cluster\n" + - "# (discover shards and Solr URLs through ZooKeeper):\n" + - "sudo -u hdfs hadoop \\\n" + - " --config /etc/hadoop/conf.cloudera.mapreduce1 \\\n" + - " jar target/solr-map-reduce-*.jar \\\n" + - " -D 'mapred.child.java.opts=-Xmx500m' \\\n" + -// " -D 'mapreduce.child.java.opts=-Xmx500m' \\\n" + - " --log4j src/test/resources/log4j.properties \\\n" + - " --morphline-file ../search-core/src/test/resources/test-morphlines/tutorialReadAvroContainer.conf \\\n" + - " --output-dir hdfs://c2202.mycompany.com/user/$USER/test \\\n" + - " --zk-host zk01.mycompany.com:2181/solr \\\n" + - " --collection collection1 \\\n" + - " --go-live \\\n" + - " hdfs:///user/foo/indir\n" - ); - throw new FoundHelpArgument(); // Trick to prevent processing of any remaining arguments - } - }); - - ArgumentGroup requiredGroup = parser.addArgumentGroup("Required arguments"); - - Argument outputDirArg = requiredGroup.addArgument("--output-dir") - .metavar("HDFS_URI") - .type(new PathArgumentType(conf) { - @Override - public Path convert(ArgumentParser parser, Argument arg, String value) throws ArgumentParserException { - Path path = super.convert(parser, arg, value); - if ("hdfs".equals(path.toUri().getScheme()) && path.toUri().getAuthority() == null) { - // TODO: consider defaulting to hadoop's fs.default.name here or in SolrRecordWriter.createEmbeddedSolrServer() - throw new ArgumentParserException("Missing authority in path URI: " + path, parser); - } - return path; - } - }.verifyHasScheme().verifyIsAbsolute().verifyCanWriteParent()) - .required(true) - .help("HDFS directory to write Solr indexes to. Inside there one output directory per shard will be generated. " + - "Example: hdfs://c2202.mycompany.com/user/$USER/test"); - - Argument inputListArg = parser.addArgument("--input-list") - .action(Arguments.append()) - .metavar("URI") - // .type(new PathArgumentType(fs).verifyExists().verifyCanRead()) - .type(Path.class) - .help("Local URI or HDFS URI of a UTF-8 encoded file containing a list of HDFS URIs to index, " + - "one URI per line in the file. If '-' is specified, URIs are read from the standard input. " + - "Multiple --input-list arguments can be specified."); - - Argument morphlineFileArg = requiredGroup.addArgument("--morphline-file") - .metavar("FILE") - .type(new FileArgumentType().verifyExists().verifyIsFile().verifyCanRead()) - .required(true) - .help("Relative or absolute path to a local config file that contains one or more morphlines. " + - "The file must be UTF-8 encoded. Example: /path/to/morphline.conf"); - - Argument morphlineIdArg = parser.addArgument("--morphline-id") - .metavar("STRING") - .type(String.class) - .help("The identifier of the morphline that shall be executed within the morphline config file " + - "specified by --morphline-file. If the --morphline-id option is ommitted the first (i.e. " + - "top-most) morphline within the config file is used. Example: morphline1"); - - Argument solrHomeDirArg = nonSolrCloud(parser.addArgument("--solr-home-dir") - .metavar("DIR") - .type(new FileArgumentType() { - @Override - public File convert(ArgumentParser parser, Argument arg, String value) throws ArgumentParserException { - File solrHomeDir = super.convert(parser, arg, value); - File solrConfigFile = new File(new File(solrHomeDir, "conf"), "solrconfig.xml"); - new FileArgumentType().verifyExists().verifyIsFile().verifyCanRead().convert( - parser, arg, solrConfigFile.getPath()); - return solrHomeDir; - } - }.verifyIsDirectory().verifyCanRead()) - .required(false) - .help("Relative or absolute path to a local dir containing Solr conf/ dir and in particular " + - "conf/solrconfig.xml and optionally also lib/ dir. This directory will be uploaded to each MR task. " + - "Example: src/test/resources/solr/minimr")); - - Argument updateConflictResolverArg = parser.addArgument("--update-conflict-resolver") - .metavar("FQCN") - .type(String.class) - .setDefault(RetainMostRecentUpdateConflictResolver.class.getName()) - .help("Fully qualified class name of a Java class that implements the UpdateConflictResolver interface. " + - "This enables deduplication and ordering of a series of document updates for the same unique document " + - "key. For example, a MapReduce batch job might index multiple files in the same job where some of the " + - "files contain old and new versions of the very same document, using the same unique document key.\n" + - "Typically, implementations of this interface forbid collisions by throwing an exception, or ignore all but " + - "the most recent document version, or, in the general case, order colliding updates ascending from least " + - "recent to most recent (partial) update. The caller of this interface (i.e. the Hadoop Reducer) will then " + - "apply the updates to Solr in the order returned by the orderUpdates() method.\n" + - "The default RetainMostRecentUpdateConflictResolver implementation ignores all but the most recent document " + - "version, based on a configurable numeric Solr field, which defaults to the file_last_modified timestamp"); - - Argument mappersArg = parser.addArgument("--mappers") - .metavar("INTEGER") - .type(Integer.class) - .choices(new RangeArgumentChoice(-1, Integer.MAX_VALUE)) // TODO: also support X% syntax where X is an integer - .setDefault(-1) - .help("Tuning knob that indicates the maximum number of MR mapper tasks to use. -1 indicates use all map slots " + - "available on the cluster."); - - Argument reducersArg = parser.addArgument("--reducers") - .metavar("INTEGER") - .type(Integer.class) - .choices(new RangeArgumentChoice(-2, Integer.MAX_VALUE)) // TODO: also support X% syntax where X is an integer - .setDefault(-1) - .help("Tuning knob that indicates the number of reducers to index into. " + - "0 is reserved for a mapper-only feature that may ship in a future release. " + - "-1 indicates use all reduce slots available on the cluster. " + - "-2 indicates use one reducer per output shard, which disables the mtree merge MR algorithm. " + - "The mtree merge MR algorithm improves scalability by spreading load " + - "(in particular CPU load) among a number of parallel reducers that can be much larger than the number " + - "of solr shards expected by the user. It can be seen as an extension of concurrent lucene merges " + - "and tiered lucene merges to the clustered case. The subsequent mapper-only phase " + - "merges the output of said large number of reducers to the number of shards expected by the user, " + - "again by utilizing more available parallelism on the cluster."); - - Argument fanoutArg = parser.addArgument("--fanout") - .metavar("INTEGER") - .type(Integer.class) - .choices(new RangeArgumentChoice(2, Integer.MAX_VALUE)) - .setDefault(Integer.MAX_VALUE) - .help(FeatureControl.SUPPRESS); - - Argument maxSegmentsArg = parser.addArgument("--max-segments") - .metavar("INTEGER") - .type(Integer.class) - .choices(new RangeArgumentChoice(1, Integer.MAX_VALUE)) - .setDefault(1) - .help("Tuning knob that indicates the maximum number of segments to be contained on output in the index of " + - "each reducer shard. After a reducer has built its output index it applies a merge policy to merge segments " + - "until there are <= maxSegments lucene segments left in this index. " + - "Merging segments involves reading and rewriting all data in all these segment files, " + - "potentially multiple times, which is very I/O intensive and time consuming. " + - "However, an index with fewer segments can later be merged faster, " + - "and it can later be queried faster once deployed to a live Solr serving shard. " + - "Set maxSegments to 1 to optimize the index for low query latency. " + - "In a nutshell, a small maxSegments value trades indexing latency for subsequently improved query latency. " + - "This can be a reasonable trade-off for batch indexing systems."); - - Argument fairSchedulerPoolArg = parser.addArgument("--fair-scheduler-pool") - .metavar("STRING") - .help("Optional tuning knob that indicates the name of the fair scheduler pool to submit jobs to. " + - "The Fair Scheduler is a pluggable MapReduce scheduler that provides a way to share large clusters. " + - "Fair scheduling is a method of assigning resources to jobs such that all jobs get, on average, an " + - "equal share of resources over time. When there is a single job running, that job uses the entire " + - "cluster. When other jobs are submitted, tasks slots that free up are assigned to the new jobs, so " + - "that each job gets roughly the same amount of CPU time. Unlike the default Hadoop scheduler, which " + - "forms a queue of jobs, this lets short jobs finish in reasonable time while not starving long jobs. " + - "It is also an easy way to share a cluster between multiple of users. Fair sharing can also work with " + - "job priorities - the priorities are used as weights to determine the fraction of total compute time " + - "that each job gets."); - - Argument dryRunArg = parser.addArgument("--dry-run") - .action(Arguments.storeTrue()) - .help("Run in local mode and print documents to stdout instead of loading them into Solr. This executes " + - "the morphline in the client process (without submitting a job to MR) for quicker turnaround during " + - "early trial & debug sessions."); - - Argument log4jConfigFileArg = parser.addArgument("--log4j") - .metavar("FILE") - .type(new FileArgumentType().verifyExists().verifyIsFile().verifyCanRead()) - .help("Relative or absolute path to a log4j.properties config file on the local file system. This file " + - "will be uploaded to each MR task. Example: /path/to/log4j.properties"); - - Argument verboseArg = parser.addArgument("--verbose", "-v") - .action(Arguments.storeTrue()) - .help("Turn on verbose output."); - - parser.addArgument(SHOW_NON_SOLR_CLOUD) - .action(Arguments.storeTrue()) - .help("Also show options for Non-SolrCloud mode as part of --help."); - - ArgumentGroup clusterInfoGroup = parser - .addArgumentGroup("Cluster arguments") - .description( - "Arguments that provide information about your Solr cluster. " - + nonSolrCloud("If you are building shards for a SolrCloud cluster, pass the --zk-host argument. " - + "If you are building shards for " - + "a Non-SolrCloud cluster, pass the --shard-url argument one or more times. To build indexes for " - + "a replicated Non-SolrCloud cluster with --shard-url, pass replica urls consecutively and also pass --shards. " - + "Using --go-live requires either --zk-host or --shard-url.")); - - Argument zkHostArg = clusterInfoGroup.addArgument("--zk-host") - .metavar("STRING") - .type(String.class) - .help("The address of a ZooKeeper ensemble being used by a SolrCloud cluster. " - + "This ZooKeeper ensemble will be examined to determine the number of output " - + "shards to create as well as the Solr URLs to merge the output shards into when using the --go-live option. " - + "Requires that you also pass the --collection to merge the shards into.\n" - + "\n" - + "The --zk-host option implements the same partitioning semantics as the standard SolrCloud " - + "Near-Real-Time (NRT) API. This enables to mix batch updates from MapReduce ingestion with " - + "updates from standard Solr NRT ingestion on the same SolrCloud cluster, " - + "using identical unique document keys.\n" - + "\n" - + "Format is: a list of comma separated host:port pairs, each corresponding to a zk " - + "server. Example: '127.0.0.1:2181,127.0.0.1:2182,127.0.0.1:2183' If " - + "the optional chroot suffix is used the example would look " - + "like: '127.0.0.1:2181/solr,127.0.0.1:2182/solr,127.0.0.1:2183/solr' " - + "where the client would be rooted at '/solr' and all paths " - + "would be relative to this root - i.e. getting/setting/etc... " - + "'/foo/bar' would result in operations being run on " - + "'/solr/foo/bar' (from the server perspective).\n" - + nonSolrCloud("\n" - + "If --solr-home-dir is not specified, the Solr home directory for the collection " - + "will be downloaded from this ZooKeeper ensemble.")); - - Argument shardUrlsArg = nonSolrCloud(clusterInfoGroup.addArgument("--shard-url") - .metavar("URL") - .type(String.class) - .action(Arguments.append()) - .help("Solr URL to merge resulting shard into if using --go-live. " + - "Example: http://solr001.mycompany.com:8983/solr/collection1. " + - "Multiple --shard-url arguments can be specified, one for each desired shard. " + - "If you are merging shards into a SolrCloud cluster, use --zk-host instead.")); - - Argument shardsArg = nonSolrCloud(clusterInfoGroup.addArgument("--shards") - .metavar("INTEGER") - .type(Integer.class) - .choices(new RangeArgumentChoice(1, Integer.MAX_VALUE)) - .help("Number of output shards to generate.")); - - ArgumentGroup goLiveGroup = parser.addArgumentGroup("Go live arguments") - .description("Arguments for merging the shards that are built into a live Solr cluster. " + - "Also see the Cluster arguments."); - - Argument goLiveArg = goLiveGroup.addArgument("--go-live") - .action(Arguments.storeTrue()) - .help("Allows you to optionally merge the final index shards into a live Solr cluster after they are built. " + - "You can pass the ZooKeeper address with --zk-host and the relevant cluster information will be auto detected. " + - nonSolrCloud("If you are not using a SolrCloud cluster, --shard-url arguments can be used to specify each SolrCore to merge " + - "each shard into.")); - - Argument collectionArg = goLiveGroup.addArgument("--collection") - .metavar("STRING") - .help("The SolrCloud collection to merge shards into when using --go-live and --zk-host. Example: collection1"); - - Argument goLiveThreadsArg = goLiveGroup.addArgument("--go-live-threads") - .metavar("INTEGER") - .type(Integer.class) - .choices(new RangeArgumentChoice(1, Integer.MAX_VALUE)) - .setDefault(1000) - .help("Tuning knob that indicates the maximum number of live merges to run in parallel at one time."); - - // trailing positional arguments - Argument inputFilesArg = parser.addArgument("input-files") - .metavar("HDFS_URI") - .type(new PathArgumentType(conf).verifyHasScheme().verifyExists().verifyCanRead()) - .nargs("*") - .setDefault() - .help("HDFS URI of file or directory tree to index."); - - Namespace ns; - try { - ns = parser.parseArgs(args); - } catch (FoundHelpArgument e) { - return 0; - } catch (ArgumentParserException e) { - parser.handleError(e); - return 1; - } - - opts.log4jConfigFile = (File) ns.get(log4jConfigFileArg.getDest()); - if (opts.log4jConfigFile != null) { - Utils.configureLog4jProperties(opts.log4jConfigFile.getPath()); - } - LOG.debug("Parsed command line args: {}", ns); - - opts.inputLists = ns.getList(inputListArg.getDest()); - if (opts.inputLists == null) { - opts.inputLists = Collections.EMPTY_LIST; - } - opts.inputFiles = ns.getList(inputFilesArg.getDest()); - opts.outputDir = (Path) ns.get(outputDirArg.getDest()); - opts.mappers = ns.getInt(mappersArg.getDest()); - opts.reducers = ns.getInt(reducersArg.getDest()); - opts.updateConflictResolver = ns.getString(updateConflictResolverArg.getDest()); - opts.fanout = ns.getInt(fanoutArg.getDest()); - opts.maxSegments = ns.getInt(maxSegmentsArg.getDest()); - opts.morphlineFile = (File) ns.get(morphlineFileArg.getDest()); - opts.morphlineId = ns.getString(morphlineIdArg.getDest()); - opts.solrHomeDir = (File) ns.get(solrHomeDirArg.getDest()); - opts.fairSchedulerPool = ns.getString(fairSchedulerPoolArg.getDest()); - opts.isDryRun = ns.getBoolean(dryRunArg.getDest()); - opts.isVerbose = ns.getBoolean(verboseArg.getDest()); - opts.zkHost = ns.getString(zkHostArg.getDest()); - opts.shards = ns.getInt(shardsArg.getDest()); - opts.shardUrls = buildShardUrls(ns.getList(shardUrlsArg.getDest()), opts.shards); - opts.goLive = ns.getBoolean(goLiveArg.getDest()); - opts.goLiveThreads = ns.getInt(goLiveThreadsArg.getDest()); - opts.collection = ns.getString(collectionArg.getDest()); - - try { - if (opts.reducers == 0) { - throw new ArgumentParserException("--reducers must not be zero", parser); - } - verifyGoLiveArgs(opts, parser); - } catch (ArgumentParserException e) { - parser.handleError(e); - return 1; - } - - if (opts.inputLists.isEmpty() && opts.inputFiles.isEmpty()) { - LOG.info("No input files specified - nothing to process"); - return 0; // nothing to process - } - return null; - } - - // make it a "hidden" option, i.e. the option is functional and enabled but not shown in --help output - private Argument nonSolrCloud(Argument arg) { - return showNonSolrCloud ? arg : arg.help(FeatureControl.SUPPRESS); - } - - private String nonSolrCloud(String msg) { - return showNonSolrCloud ? msg : ""; - } - - /** Marker trick to prevent processing of any remaining arguments once --help option has been parsed */ - private static final class FoundHelpArgument extends RuntimeException { - } - } - // END OF INNER CLASS - - static List> buildShardUrls(List urls, Integer numShards) { - if (urls == null) return null; - List> shardUrls = new ArrayList<>(urls.size()); - List list = null; - - int sz; - if (numShards == null) { - numShards = urls.size(); - } - sz = (int) Math.ceil(urls.size() / (float)numShards); - for (int i = 0; i < urls.size(); i++) { - if (i % sz == 0) { - list = new ArrayList<>(); - shardUrls.add(list); - } - list.add((String) urls.get(i)); - } - - return shardUrls; - } - - static final class Options { - boolean goLive; - String collection; - String zkHost; - Integer goLiveThreads; - List> shardUrls; - List inputLists; - List inputFiles; - Path outputDir; - int mappers; - int reducers; - String updateConflictResolver; - int fanout; - Integer shards; - int maxSegments; - File morphlineFile; - String morphlineId; - File solrHomeDir; - String fairSchedulerPool; - boolean isDryRun; - File log4jConfigFile; - boolean isVerbose; - } - // END OF INNER CLASS - - - /** API for command line clients */ - public static void main(String[] args) throws Exception { - int res = ToolRunner.run(new Configuration(), new MapReduceIndexerTool(), args); - System.exit(res); - } - - public MapReduceIndexerTool() {} - - @Override - public int run(String[] args) throws Exception { - Options opts = new Options(); - Integer exitCode = new MyArgumentParser().parseArgs(args, getConf(), opts); - if (exitCode != null) { - return exitCode; - } - return run(opts); - } - - /** API for Java clients; visible for testing; may become a public API eventually */ - int run(Options options) throws Exception { - if (getConf().getBoolean("isMR1", false) && "local".equals(getConf().get("mapred.job.tracker"))) { - throw new IllegalStateException( - "Running with LocalJobRunner (i.e. all of Hadoop inside a single JVM) is not supported " + - "because LocalJobRunner does not (yet) implement the Hadoop Distributed Cache feature, " + - "which is required for passing files via --files and --libjars"); - } - - long programStartTime = System.nanoTime(); - if (options.fairSchedulerPool != null) { - getConf().set("mapred.fairscheduler.pool", options.fairSchedulerPool); - } - getConf().setInt(SolrOutputFormat.SOLR_RECORD_WRITER_MAX_SEGMENTS, options.maxSegments); - - // switch off a false warning about allegedly not implementing Tool - // also see http://hadoop.6.n7.nabble.com/GenericOptionsParser-warning-td8103.html - // also see https://issues.apache.org/jira/browse/HADOOP-8183 - getConf().setBoolean("mapred.used.genericoptionsparser", true); - - if (options.log4jConfigFile != null) { - Utils.setLogConfigFile(options.log4jConfigFile, getConf()); - addDistributedCacheFile(options.log4jConfigFile, getConf()); - } - - job = Job.getInstance(getConf()); - job.setJarByClass(getClass()); - - if (options.morphlineFile == null) { - throw new ArgumentParserException("Argument --morphline-file is required", null); - } - verifyGoLiveArgs(options, null); - verifyZKStructure(options, null); - - int mappers = new JobClient(job.getConfiguration()).getClusterStatus().getMaxMapTasks(); // MR1 - //int mappers = job.getCluster().getClusterStatus().getMapSlotCapacity(); // Yarn only - LOG.info("Cluster reports {} mapper slots", mappers); - - if (options.mappers == -1) { - mappers = 8 * mappers; // better accomodate stragglers - } else { - mappers = options.mappers; - } - if (mappers <= 0) { - throw new IllegalStateException("Illegal number of mappers: " + mappers); - } - options.mappers = mappers; - - FileSystem fs = options.outputDir.getFileSystem(job.getConfiguration()); - if (fs.exists(options.outputDir) && !delete(options.outputDir, true, fs)) { - return -1; - } - Path outputResultsDir = new Path(options.outputDir, RESULTS_DIR); - Path outputReduceDir = new Path(options.outputDir, "reducers"); - Path outputStep1Dir = new Path(options.outputDir, "tmp1"); - Path outputStep2Dir = new Path(options.outputDir, "tmp2"); - Path outputTreeMergeStep = new Path(options.outputDir, "mtree-merge-output"); - Path fullInputList = new Path(outputStep1Dir, FULL_INPUT_LIST); - - LOG.debug("Creating list of input files for mappers: {}", fullInputList); - long numFiles = addInputFiles(options.inputFiles, options.inputLists, fullInputList, job.getConfiguration()); - if (numFiles == 0) { - LOG.info("No input files found - nothing to process"); - return 0; - } - int numLinesPerSplit = (int) ceilDivide(numFiles, mappers); - if (numLinesPerSplit < 0) { // numeric overflow from downcasting long to int? - numLinesPerSplit = Integer.MAX_VALUE; - } - numLinesPerSplit = Math.max(1, numLinesPerSplit); - - int realMappers = Math.min(mappers, (int) ceilDivide(numFiles, numLinesPerSplit)); - calculateNumReducers(options, realMappers); - int reducers = options.reducers; - LOG.info("Using these parameters: " + - "numFiles: {}, mappers: {}, realMappers: {}, reducers: {}, shards: {}, fanout: {}, maxSegments: {}", - new Object[] {numFiles, mappers, realMappers, reducers, options.shards, options.fanout, options.maxSegments}); - - - LOG.info("Randomizing list of {} input files to spread indexing load more evenly among mappers", numFiles); - long startTime = System.nanoTime(); - if (numFiles < job.getConfiguration().getInt(MAIN_MEMORY_RANDOMIZATION_THRESHOLD, 100001)) { - // If there are few input files reduce latency by directly running main memory randomization - // instead of launching a high latency MapReduce job - randomizeFewInputFiles(fs, outputStep2Dir, fullInputList); - } else { - // Randomize using a MapReduce job. Use sequential algorithm below a certain threshold because there's no - // benefit in using many parallel mapper tasks just to randomize the order of a few lines each - int numLinesPerRandomizerSplit = Math.max(10 * 1000 * 1000, numLinesPerSplit); - Job randomizerJob = randomizeManyInputFiles(getConf(), fullInputList, outputStep2Dir, numLinesPerRandomizerSplit); - if (!waitForCompletion(randomizerJob, options.isVerbose)) { - return -1; // job failed - } - } - float secs = (System.nanoTime() - startTime) / (float)(10^9); - LOG.info("Done. Randomizing list of {} input files took {} secs", numFiles, secs); - - - job.setInputFormatClass(NLineInputFormat.class); - NLineInputFormat.addInputPath(job, outputStep2Dir); - NLineInputFormat.setNumLinesPerSplit(job, numLinesPerSplit); - FileOutputFormat.setOutputPath(job, outputReduceDir); - - String mapperClass = job.getConfiguration().get(JobContext.MAP_CLASS_ATTR); - if (mapperClass == null) { // enable customization - Class clazz = MorphlineMapper.class; - mapperClass = clazz.getName(); - job.setMapperClass(clazz); - } - job.setJobName(getClass().getName() + "/" + Utils.getShortClassName(mapperClass)); - - if (job.getConfiguration().get(JobContext.REDUCE_CLASS_ATTR) == null) { // enable customization - job.setReducerClass(SolrReducer.class); - } - if (options.updateConflictResolver == null) { - throw new IllegalArgumentException("updateConflictResolver must not be null"); - } - job.getConfiguration().set(SolrReducer.UPDATE_CONFLICT_RESOLVER, options.updateConflictResolver); - - if (options.zkHost != null) { - assert options.collection != null; - /* - * MapReduce partitioner that partitions the Mapper output such that each - * SolrInputDocument gets sent to the SolrCloud shard that it would have - * been sent to if the document were ingested via the standard SolrCloud - * Near Real Time (NRT) API. - * - * In other words, this class implements the same partitioning semantics - * as the standard SolrCloud NRT API. This enables to mix batch updates - * from MapReduce ingestion with updates from standard NRT ingestion on - * the same SolrCloud cluster, using identical unique document keys. - */ - if (job.getConfiguration().get(JobContext.PARTITIONER_CLASS_ATTR) == null) { // enable customization - job.setPartitionerClass(SolrCloudPartitioner.class); - } - job.getConfiguration().set(SolrCloudPartitioner.ZKHOST, options.zkHost); - job.getConfiguration().set(SolrCloudPartitioner.COLLECTION, options.collection); - } - job.getConfiguration().setInt(SolrCloudPartitioner.SHARDS, options.shards); - - job.setOutputFormatClass(SolrOutputFormat.class); - if (options.solrHomeDir != null) { - SolrOutputFormat.setupSolrHomeCache(options.solrHomeDir, job); - } else { - assert options.zkHost != null; - // use the config that this collection uses for the SolrHomeCache. - ZooKeeperInspector zki = new ZooKeeperInspector(); - SolrZkClient zkClient = zki.getZkClient(options.zkHost); - try { - String configName = zki.readConfigName(zkClient, options.collection); - File tmpSolrHomeDir = zki.downloadConfigDir(zkClient, configName); - SolrOutputFormat.setupSolrHomeCache(tmpSolrHomeDir, job); - options.solrHomeDir = tmpSolrHomeDir; - } finally { - zkClient.close(); - } - } - - MorphlineMapRunner runner = setupMorphline(options); - if (options.isDryRun && runner != null) { - LOG.info("Indexing {} files in dryrun mode", numFiles); - startTime = System.nanoTime(); - dryRun(runner, fs, fullInputList); - secs = (System.nanoTime() - startTime) / (float)(10^9); - LOG.info("Done. Indexing {} files in dryrun mode took {} secs", numFiles, secs); - goodbye(null, programStartTime); - return 0; - } - job.getConfiguration().set(MorphlineMapRunner.MORPHLINE_FILE_PARAM, options.morphlineFile.getName()); - - job.setNumReduceTasks(reducers); - job.setOutputKeyClass(Text.class); - job.setOutputValueClass(SolrInputDocumentWritable.class); - LOG.info("Indexing {} files using {} real mappers into {} reducers", new Object[] {numFiles, realMappers, reducers}); - startTime = System.nanoTime(); - if (!waitForCompletion(job, options.isVerbose)) { - return -1; // job failed - } - - secs = (System.nanoTime() - startTime) / (float)(10^9); - LOG.info("Done. Indexing {} files using {} real mappers into {} reducers took {} secs", new Object[] {numFiles, realMappers, reducers, secs}); - - int mtreeMergeIterations = 0; - if (reducers > options.shards) { - mtreeMergeIterations = (int) Math.round(log(options.fanout, reducers / options.shards)); - } - LOG.debug("MTree merge iterations to do: {}", mtreeMergeIterations); - int mtreeMergeIteration = 1; - while (reducers > options.shards) { // run a mtree merge iteration - job = Job.getInstance(getConf()); - job.setJarByClass(getClass()); - job.setJobName(getClass().getName() + "/" + Utils.getShortClassName(TreeMergeMapper.class)); - job.setMapperClass(TreeMergeMapper.class); - job.setOutputFormatClass(TreeMergeOutputFormat.class); - job.setNumReduceTasks(0); - job.setOutputKeyClass(Text.class); - job.setOutputValueClass(NullWritable.class); - job.setInputFormatClass(NLineInputFormat.class); - - Path inputStepDir = new Path(options.outputDir, "mtree-merge-input-iteration" + mtreeMergeIteration); - fullInputList = new Path(inputStepDir, FULL_INPUT_LIST); - LOG.debug("MTree merge iteration {}/{}: Creating input list file for mappers {}", new Object[] {mtreeMergeIteration, mtreeMergeIterations, fullInputList}); - numFiles = createTreeMergeInputDirList(outputReduceDir, fs, fullInputList); - if (numFiles != reducers) { - throw new IllegalStateException("Not same reducers: " + reducers + ", numFiles: " + numFiles); - } - NLineInputFormat.addInputPath(job, fullInputList); - NLineInputFormat.setNumLinesPerSplit(job, options.fanout); - FileOutputFormat.setOutputPath(job, outputTreeMergeStep); - - LOG.info("MTree merge iteration {}/{}: Merging {} shards into {} shards using fanout {}", new Object[] { - mtreeMergeIteration, mtreeMergeIterations, reducers, (reducers / options.fanout), options.fanout}); - startTime = System.nanoTime(); - if (!waitForCompletion(job, options.isVerbose)) { - return -1; // job failed - } - if (!renameTreeMergeShardDirs(outputTreeMergeStep, job, fs)) { - return -1; - } - secs = (System.nanoTime() - startTime) / (float)(10^9); - LOG.info("MTree merge iteration {}/{}: Done. Merging {} shards into {} shards using fanout {} took {} secs", - new Object[] {mtreeMergeIteration, mtreeMergeIterations, reducers, (reducers / options.fanout), options.fanout, secs}); - - if (!delete(outputReduceDir, true, fs)) { - return -1; - } - if (!rename(outputTreeMergeStep, outputReduceDir, fs)) { - return -1; - } - assert reducers % options.fanout == 0; - reducers = reducers / options.fanout; - mtreeMergeIteration++; - } - assert reducers == options.shards; - - // normalize output shard dir prefix, i.e. - // rename part-r-00000 to part-00000 (stems from zero tree merge iterations) - // rename part-m-00000 to part-00000 (stems from > 0 tree merge iterations) - for (FileStatus stats : fs.listStatus(outputReduceDir)) { - String dirPrefix = SolrOutputFormat.getOutputName(job); - Path srcPath = stats.getPath(); - if (stats.isDirectory() && srcPath.getName().startsWith(dirPrefix)) { - String dstName = dirPrefix + srcPath.getName().substring(dirPrefix.length() + "-m".length()); - Path dstPath = new Path(srcPath.getParent(), dstName); - if (!rename(srcPath, dstPath, fs)) { - return -1; - } - } - }; - - // publish results dir - if (!rename(outputReduceDir, outputResultsDir, fs)) { - return -1; - } - - if (options.goLive && !new GoLive().goLive(options, listSortedOutputShardDirs(outputResultsDir, fs))) { - return -1; - } - - goodbye(job, programStartTime); - return 0; - } - - private void calculateNumReducers(Options options, int realMappers) throws IOException { - if (options.shards <= 0) { - throw new IllegalStateException("Illegal number of shards: " + options.shards); - } - if (options.fanout <= 1) { - throw new IllegalStateException("Illegal fanout: " + options.fanout); - } - if (realMappers <= 0) { - throw new IllegalStateException("Illegal realMappers: " + realMappers); - } - - - int reducers = new JobClient(job.getConfiguration()).getClusterStatus().getMaxReduceTasks(); // MR1 - //reducers = job.getCluster().getClusterStatus().getReduceSlotCapacity(); // Yarn only - LOG.info("Cluster reports {} reduce slots", reducers); - - if (options.reducers == -2) { - reducers = options.shards; - } else if (options.reducers == -1) { - reducers = Math.min(reducers, realMappers); // no need to use many reducers when using few mappers - } else { - if (options.reducers == 0) { - throw new IllegalStateException("Illegal zero reducers"); - } - reducers = options.reducers; - } - reducers = Math.max(reducers, options.shards); - - if (reducers != options.shards) { - // Ensure fanout isn't misconfigured. fanout can't meaningfully be larger than what would be - // required to merge all leaf shards in one single tree merge iteration into root shards - options.fanout = Math.min(options.fanout, (int) ceilDivide(reducers, options.shards)); - - // Ensure invariant reducers == options.shards * (fanout ^ N) where N is an integer >= 1. - // N is the number of mtree merge iterations. - // This helps to evenly spread docs among root shards and simplifies the impl of the mtree merge algorithm. - int s = options.shards; - while (s < reducers) { - s = s * options.fanout; - } - reducers = s; - assert reducers % options.fanout == 0; - } - options.reducers = reducers; - } - - private long addInputFiles(List inputFiles, List inputLists, Path fullInputList, Configuration conf) - throws IOException { - - long numFiles = 0; - FileSystem fs = fullInputList.getFileSystem(conf); - FSDataOutputStream out = fs.create(fullInputList); - try { - Writer writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)); - - for (Path inputFile : inputFiles) { - FileSystem inputFileFs = inputFile.getFileSystem(conf); - if (inputFileFs.exists(inputFile)) { - PathFilter pathFilter = new PathFilter() { - @Override - public boolean accept(Path path) { // ignore "hidden" files and dirs - return !(path.getName().startsWith(".") || path.getName().startsWith("_")); - } - }; - numFiles += addInputFilesRecursively(inputFile, writer, inputFileFs, pathFilter); - } - } - - for (Path inputList : inputLists) { - InputStream in; - if (inputList.toString().equals("-")) { - in = System.in; - } else if (inputList.isAbsoluteAndSchemeAuthorityNull()) { - in = new BufferedInputStream(new FileInputStream(inputList.toString())); - } else { - in = inputList.getFileSystem(conf).open(inputList); - } - try { - BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8)); - String line; - while ((line = reader.readLine()) != null) { - writer.write(line + "\n"); - numFiles++; - } - reader.close(); - } finally { - in.close(); - } - } - - writer.close(); - } finally { - out.close(); - } - return numFiles; - } - - /** - * Add the specified file to the input set, if path is a directory then - * add the files contained therein. - */ - private long addInputFilesRecursively(Path path, Writer writer, FileSystem fs, PathFilter pathFilter) throws IOException { - long numFiles = 0; - for (FileStatus stat : fs.listStatus(path, pathFilter)) { - LOG.debug("Adding path {}", stat.getPath()); - if (stat.isDirectory()) { - numFiles += addInputFilesRecursively(stat.getPath(), writer, fs, pathFilter); - } else { - writer.write(stat.getPath().toString() + "\n"); - numFiles++; - } - } - return numFiles; - } - - private void randomizeFewInputFiles(FileSystem fs, Path outputStep2Dir, Path fullInputList) throws IOException { - List lines = new ArrayList(); - BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(fullInputList), StandardCharsets.UTF_8)); - try { - String line; - while ((line = reader.readLine()) != null) { - lines.add(line); - } - } finally { - reader.close(); - } - - Collections.shuffle(lines, new Random(421439783L)); // constant seed for reproducability - - FSDataOutputStream out = fs.create(new Path(outputStep2Dir, FULL_INPUT_LIST)); - Writer writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)); - try { - for (String line : lines) { - writer.write(line + "\n"); - } - } finally { - writer.close(); - } - } - - /** - * To uniformly spread load across all mappers we randomize fullInputList - * with a separate small Mapper & Reducer preprocessing step. This way - * each input line ends up on a random position in the output file list. - * Each mapper indexes a disjoint consecutive set of files such that each - * set has roughly the same size, at least from a probabilistic - * perspective. - * - * For example an input file with the following input list of URLs: - * - * A - * B - * C - * D - * - * might be randomized into the following output list of URLs: - * - * C - * A - * D - * B - * - * The implementation sorts the list of lines by randomly generated numbers. - */ - private Job randomizeManyInputFiles(Configuration baseConfig, Path fullInputList, Path outputStep2Dir, int numLinesPerSplit) - throws IOException { - - Job job2 = Job.getInstance(baseConfig); - job2.setJarByClass(getClass()); - job2.setJobName(getClass().getName() + "/" + Utils.getShortClassName(LineRandomizerMapper.class)); - job2.setInputFormatClass(NLineInputFormat.class); - NLineInputFormat.addInputPath(job2, fullInputList); - NLineInputFormat.setNumLinesPerSplit(job2, numLinesPerSplit); - job2.setMapperClass(LineRandomizerMapper.class); - job2.setReducerClass(LineRandomizerReducer.class); - job2.setOutputFormatClass(TextOutputFormat.class); - FileOutputFormat.setOutputPath(job2, outputStep2Dir); - job2.setNumReduceTasks(1); - job2.setOutputKeyClass(LongWritable.class); - job2.setOutputValueClass(Text.class); - return job2; - } - - // do the same as if the user had typed 'hadoop ... --files ' - private void addDistributedCacheFile(File file, Configuration conf) throws IOException { - String HADOOP_TMP_FILES = "tmpfiles"; // see Hadoop's GenericOptionsParser - String tmpFiles = conf.get(HADOOP_TMP_FILES, ""); - if (tmpFiles.length() > 0) { // already present? - tmpFiles = tmpFiles + ","; - } - GenericOptionsParser parser = new GenericOptionsParser( - new Configuration(conf), - new String[] { "--files", file.getCanonicalPath() }); - String additionalTmpFiles = parser.getConfiguration().get(HADOOP_TMP_FILES); - assert additionalTmpFiles != null; - assert additionalTmpFiles.length() > 0; - tmpFiles += additionalTmpFiles; - conf.set(HADOOP_TMP_FILES, tmpFiles); - } - - private MorphlineMapRunner setupMorphline(Options options) throws IOException, URISyntaxException { - if (options.morphlineId != null) { - job.getConfiguration().set(MorphlineMapRunner.MORPHLINE_ID_PARAM, options.morphlineId); - } - addDistributedCacheFile(options.morphlineFile, job.getConfiguration()); - if (!options.isDryRun) { - return null; - } - - /* - * Ensure scripting support for Java via morphline "java" command works even in dryRun mode, - * i.e. when executed in the client side driver JVM. To do so, collect all classpath URLs from - * the class loaders chain that org.apache.hadoop.util.RunJar (hadoop jar xyz-job.jar) and - * org.apache.hadoop.util.GenericOptionsParser (--libjars) have installed, then tell - * FastJavaScriptEngine.parse() where to find classes that JavaBuilder scripts might depend on. - * This ensures that scripts that reference external java classes compile without exceptions - * like this: - * - * ... caused by compilation failed: mfm:///MyJavaClass1.java:2: package - * org.kitesdk.morphline.api does not exist - */ - LOG.trace("dryRun: java.class.path: {}", System.getProperty("java.class.path")); - String fullClassPath = ""; - ClassLoader loader = Thread.currentThread().getContextClassLoader(); // see org.apache.hadoop.util.RunJar - while (loader != null) { // walk class loaders, collect all classpath URLs - if (loader instanceof URLClassLoader) { - URL[] classPathPartURLs = ((URLClassLoader) loader).getURLs(); // see org.apache.hadoop.util.RunJar - LOG.trace("dryRun: classPathPartURLs: {}", Arrays.asList(classPathPartURLs)); - StringBuilder classPathParts = new StringBuilder(); - for (URL url : classPathPartURLs) { - File file = new File(url.toURI()); - if (classPathPartURLs.length > 0) { - classPathParts.append(File.pathSeparator); - } - classPathParts.append(file.getPath()); - } - LOG.trace("dryRun: classPathParts: {}", classPathParts); - String separator = File.pathSeparator; - if (fullClassPath.length() == 0 || classPathParts.length() == 0) { - separator = ""; - } - fullClassPath = classPathParts + separator + fullClassPath; - } - loader = loader.getParent(); - } - - // tell FastJavaScriptEngine.parse() where to find the classes that the script might depend on - if (fullClassPath.length() > 0) { - assert System.getProperty("java.class.path") != null; - fullClassPath = System.getProperty("java.class.path") + File.pathSeparator + fullClassPath; - LOG.trace("dryRun: fullClassPath: {}", fullClassPath); - System.setProperty("java.class.path", fullClassPath); // see FastJavaScriptEngine.parse() - } - - job.getConfiguration().set(MorphlineMapRunner.MORPHLINE_FILE_PARAM, options.morphlineFile.getPath()); - return new MorphlineMapRunner( - job.getConfiguration(), new DryRunDocumentLoader(), options.solrHomeDir.getPath()); - } - - /* - * Executes the morphline in the current process (without submitting a job to MR) for quicker - * turnaround during trial & debug sessions - */ - private void dryRun(MorphlineMapRunner runner, FileSystem fs, Path fullInputList) throws IOException { - BufferedReader reader = new BufferedReader(new InputStreamReader(fs.open(fullInputList), StandardCharsets.UTF_8)); - try { - String line; - while ((line = reader.readLine()) != null) { - runner.map(line, job.getConfiguration(), null); - } - runner.cleanup(); - } finally { - reader.close(); - } - } - - private int createTreeMergeInputDirList(Path outputReduceDir, FileSystem fs, Path fullInputList) - throws FileNotFoundException, IOException { - - FileStatus[] dirs = listSortedOutputShardDirs(outputReduceDir, fs); - int numFiles = 0; - FSDataOutputStream out = fs.create(fullInputList); - try { - Writer writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8)); - for (FileStatus stat : dirs) { - LOG.debug("Adding path {}", stat.getPath()); - Path dir = new Path(stat.getPath(), "data/index"); - if (!fs.isDirectory(dir)) { - throw new IllegalStateException("Not a directory: " + dir); - } - writer.write(dir.toString() + "\n"); - numFiles++; - } - writer.close(); - } finally { - out.close(); - } - return numFiles; - } - - private FileStatus[] listSortedOutputShardDirs(Path outputReduceDir, FileSystem fs) throws FileNotFoundException, - IOException { - - final String dirPrefix = SolrOutputFormat.getOutputName(job); - FileStatus[] dirs = fs.listStatus(outputReduceDir, new PathFilter() { - @Override - public boolean accept(Path path) { - return path.getName().startsWith(dirPrefix); - } - }); - for (FileStatus dir : dirs) { - if (!dir.isDirectory()) { - throw new IllegalStateException("Not a directory: " + dir.getPath()); - } - } - - // use alphanumeric sort (rather than lexicographical sort) to properly handle more than 99999 shards - Arrays.sort(dirs, (f1, f2) -> new AlphaNumericComparator().compare(f1.getPath().getName(), f2.getPath().getName())); - - return dirs; - } - - /* - * You can run MapReduceIndexerTool in Solrcloud mode, and once the MR job completes, you can use - * the standard solrj Solrcloud API to send doc updates and deletes to SolrCloud, and those updates - * and deletes will go to the right Solr shards, and it will work just fine. - * - * The MapReduce framework doesn't guarantee that input split N goes to the map task with the - * taskId = N. The job tracker and Yarn schedule and assign tasks, considering data locality - * aspects, but without regard of the input split# withing the overall list of input splits. In - * other words, split# != taskId can be true. - * - * To deal with this issue, our mapper tasks write a little auxiliary metadata file (per task) - * that tells the job driver which taskId processed which split#. Once the mapper-only job is - * completed, the job driver renames the output dirs such that the dir name contains the true solr - * shard id, based on these auxiliary files. - * - * This way each doc gets assigned to the right Solr shard even with #reducers > #solrshards - * - * Example for a merge with two shards: - * - * part-m-00000 and part-m-00001 goes to outputShardNum = 0 and will end up in merged part-m-00000 - * part-m-00002 and part-m-00003 goes to outputShardNum = 1 and will end up in merged part-m-00001 - * part-m-00004 and part-m-00005 goes to outputShardNum = 2 and will end up in merged part-m-00002 - * ... and so on - * - * Also see run() method above where it uses NLineInputFormat.setNumLinesPerSplit(job, - * options.fanout) - * - * Also see TreeMergeOutputFormat.TreeMergeRecordWriter.writeShardNumberFile() - */ - private boolean renameTreeMergeShardDirs(Path outputTreeMergeStep, Job job, FileSystem fs) throws IOException { - final String dirPrefix = SolrOutputFormat.getOutputName(job); - FileStatus[] dirs = fs.listStatus(outputTreeMergeStep, new PathFilter() { - @Override - public boolean accept(Path path) { - return path.getName().startsWith(dirPrefix); - } - }); - - for (FileStatus dir : dirs) { - if (!dir.isDirectory()) { - throw new IllegalStateException("Not a directory: " + dir.getPath()); - } - } - - // Example: rename part-m-00004 to _part-m-00004 - for (FileStatus dir : dirs) { - Path path = dir.getPath(); - Path renamedPath = new Path(path.getParent(), "_" + path.getName()); - if (!rename(path, renamedPath, fs)) { - return false; - } - } - - // Example: rename _part-m-00004 to part-m-00002 - for (FileStatus dir : dirs) { - Path path = dir.getPath(); - Path renamedPath = new Path(path.getParent(), "_" + path.getName()); - - // read auxiliary metadata file (per task) that tells which taskId - // processed which split# aka solrShard - Path solrShardNumberFile = new Path(renamedPath, TreeMergeMapper.SOLR_SHARD_NUMBER); - InputStream in = fs.open(solrShardNumberFile); - byte[] bytes = ByteStreams.toByteArray(in); - in.close(); - Preconditions.checkArgument(bytes.length > 0); - int solrShard = Integer.parseInt(new String(bytes, StandardCharsets.UTF_8)); - if (!delete(solrShardNumberFile, false, fs)) { - return false; - } - - // same as FileOutputFormat.NUMBER_FORMAT - NumberFormat numberFormat = NumberFormat.getInstance(Locale.ENGLISH); - numberFormat.setMinimumIntegerDigits(5); - numberFormat.setGroupingUsed(false); - Path finalPath = new Path(renamedPath.getParent(), dirPrefix + "-m-" + numberFormat.format(solrShard)); - - LOG.info("MTree merge renaming solr shard: " + solrShard + " from dir: " + dir.getPath() + " to dir: " + finalPath); - if (!rename(renamedPath, finalPath, fs)) { - return false; - } - } - return true; - } - - private static void verifyGoLiveArgs(Options opts, ArgumentParser parser) throws ArgumentParserException { - if (opts.zkHost == null && opts.solrHomeDir == null) { - throw new ArgumentParserException("At least one of --zk-host or --solr-home-dir is required", parser); - } - if (opts.goLive && opts.zkHost == null && opts.shardUrls == null) { - throw new ArgumentParserException("--go-live requires that you also pass --shard-url or --zk-host", parser); - } - - if (opts.zkHost != null && opts.collection == null) { - throw new ArgumentParserException("--zk-host requires that you also pass --collection", parser); - } - - if (opts.zkHost != null) { - return; - // verify structure of ZK directory later, to avoid checking run-time errors during parsing. - } else if (opts.shardUrls != null) { - if (opts.shardUrls.size() == 0) { - throw new ArgumentParserException("--shard-url requires at least one URL", parser); - } - } else if (opts.shards != null) { - if (opts.shards <= 0) { - throw new ArgumentParserException("--shards must be a positive number: " + opts.shards, parser); - } - } else { - throw new ArgumentParserException("You must specify one of the following (mutually exclusive) arguments: " - + "--zk-host or --shard-url or --shards", parser); - } - - if (opts.shardUrls != null) { - opts.shards = opts.shardUrls.size(); - } - - assert opts.shards != null; - assert opts.shards > 0; - } - - private static void verifyZKStructure(Options opts, ArgumentParser parser) throws ArgumentParserException { - if (opts.zkHost != null) { - assert opts.collection != null; - ZooKeeperInspector zki = new ZooKeeperInspector(); - try { - opts.shardUrls = zki.extractShardUrls(opts.zkHost, opts.collection); - } catch (Exception e) { - LOG.debug("Cannot extract SolrCloud shard URLs from ZooKeeper", e); - throw new ArgumentParserException(e, parser); - } - assert opts.shardUrls != null; - if (opts.shardUrls.size() == 0) { - throw new ArgumentParserException("--zk-host requires ZooKeeper " + opts.zkHost - + " to contain at least one SolrCore for collection: " + opts.collection, parser); - } - opts.shards = opts.shardUrls.size(); - LOG.debug("Using SolrCloud shard URLs: {}", opts.shardUrls); - } - } - - private boolean waitForCompletion(Job job, boolean isVerbose) - throws IOException, InterruptedException, ClassNotFoundException { - - LOG.debug("Running job: " + getJobInfo(job)); - boolean success = job.waitForCompletion(isVerbose); - if (!success) { - LOG.error("Job failed! " + getJobInfo(job)); - } - return success; - } - - private void goodbye(Job job, long startTime) { - float secs = (System.nanoTime() - startTime) / (float)(10^9); - if (job != null) { - LOG.info("Succeeded with job: " + getJobInfo(job)); - } - LOG.info("Success. Done. Program took {} secs. Goodbye.", secs); - } - - private String getJobInfo(Job job) { - return "jobName: " + job.getJobName() + ", jobId: " + job.getJobID(); - } - - private boolean rename(Path src, Path dst, FileSystem fs) throws IOException { - boolean success = fs.rename(src, dst); - if (!success) { - LOG.error("Cannot rename " + src + " to " + dst); - } - return success; - } - - private boolean delete(Path path, boolean recursive, FileSystem fs) throws IOException { - boolean success = fs.delete(path, recursive); - if (!success) { - LOG.error("Cannot delete " + path); - } - return success; - } - - // same as IntMath.divide(p, q, RoundingMode.CEILING) - private long ceilDivide(long p, long q) { - long result = p / q; - if (p % q != 0) { - result++; - } - return result; - } - - /** - * Returns logbasevalue. - */ - private double log(double base, double value) { - return Math.log(value) / Math.log(base); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/PathArgumentType.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/PathArgumentType.java deleted file mode 100644 index 770a2f9f90b..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/PathArgumentType.java +++ /dev/null @@ -1,233 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; - -import net.sourceforge.argparse4j.inf.Argument; -import net.sourceforge.argparse4j.inf.ArgumentParser; -import net.sourceforge.argparse4j.inf.ArgumentParserException; -import net.sourceforge.argparse4j.inf.ArgumentType; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsAction; - -/** - * ArgumentType subclass for HDFS Path type, using fluent style API. - */ -public class PathArgumentType implements ArgumentType { - - private final Configuration conf; - private FileSystem fs; - private boolean acceptSystemIn = false; - private boolean verifyExists = false; - private boolean verifyNotExists = false; - private boolean verifyIsFile = false; - private boolean verifyIsDirectory = false; - private boolean verifyCanRead = false; - private boolean verifyCanWrite = false; - private boolean verifyCanWriteParent = false; - private boolean verifyCanExecute = false; - private boolean verifyIsAbsolute = false; - private boolean verifyHasScheme = false; - private String verifyScheme = null; - - public PathArgumentType(Configuration conf) { - this.conf = conf; - } - - public PathArgumentType acceptSystemIn() { - acceptSystemIn = true; - return this; - } - - public PathArgumentType verifyExists() { - verifyExists = true; - return this; - } - - public PathArgumentType verifyNotExists() { - verifyNotExists = true; - return this; - } - - public PathArgumentType verifyIsFile() { - verifyIsFile = true; - return this; - } - - public PathArgumentType verifyIsDirectory() { - verifyIsDirectory = true; - return this; - } - - public PathArgumentType verifyCanRead() { - verifyCanRead = true; - return this; - } - - public PathArgumentType verifyCanWrite() { - verifyCanWrite = true; - return this; - } - - public PathArgumentType verifyCanWriteParent() { - verifyCanWriteParent = true; - return this; - } - - public PathArgumentType verifyCanExecute() { - verifyCanExecute = true; - return this; - } - - public PathArgumentType verifyIsAbsolute() { - verifyIsAbsolute = true; - return this; - } - - public PathArgumentType verifyHasScheme() { - verifyHasScheme = true; - return this; - } - - public PathArgumentType verifyScheme(String scheme) { - verifyScheme = scheme; - return this; - } - - @Override - public Path convert(ArgumentParser parser, Argument arg, String value) throws ArgumentParserException { - Path file = new Path(value); - try { - fs = file.getFileSystem(conf); - if (verifyHasScheme && !isSystemIn(file)) { - verifyHasScheme(parser, file); - } - if (verifyScheme != null && !isSystemIn(file)) { - verifyScheme(parser, file); - } - if (verifyIsAbsolute && !isSystemIn(file)) { - verifyIsAbsolute(parser, file); - } - if (verifyExists && !isSystemIn(file)) { - verifyExists(parser, file); - } - if (verifyNotExists && !isSystemIn(file)) { - verifyNotExists(parser, file); - } - if (verifyIsFile && !isSystemIn(file)) { - verifyIsFile(parser, file); - } - if (verifyIsDirectory && !isSystemIn(file)) { - verifyIsDirectory(parser, file); - } - if (verifyCanRead && !isSystemIn(file)) { - verifyCanRead(parser, file); - } - if (verifyCanWrite && !isSystemIn(file)) { - verifyCanWrite(parser, file); - } - if (verifyCanWriteParent && !isSystemIn(file)) { - verifyCanWriteParent(parser, file); - } - if (verifyCanExecute && !isSystemIn(file)) { - verifyCanExecute(parser, file); - } - } catch (IOException e) { - throw new ArgumentParserException(e, parser); - } - return file; - } - - private void verifyExists(ArgumentParser parser, Path file) throws ArgumentParserException, IOException { - if (!fs.exists(file)) { - throw new ArgumentParserException("File not found: " + file, parser); - } - } - - private void verifyNotExists(ArgumentParser parser, Path file) throws ArgumentParserException, IOException { - if (fs.exists(file)) { - throw new ArgumentParserException("File found: " + file, parser); - } - } - - private void verifyIsFile(ArgumentParser parser, Path file) throws ArgumentParserException, IOException { - if (!fs.isFile(file)) { - throw new ArgumentParserException("Not a file: " + file, parser); - } - } - - private void verifyIsDirectory(ArgumentParser parser, Path file) throws ArgumentParserException, IOException { - if (!fs.isDirectory(file)) { - throw new ArgumentParserException("Not a directory: " + file, parser); - } - } - - private void verifyCanRead(ArgumentParser parser, Path file) throws ArgumentParserException, IOException { - verifyExists(parser, file); - if (!fs.getFileStatus(file).getPermission().getUserAction().implies(FsAction.READ)) { - throw new ArgumentParserException("Insufficient permissions to read file: " + file, parser); - } - } - - private void verifyCanWrite(ArgumentParser parser, Path file) throws ArgumentParserException, IOException { - verifyExists(parser, file); - if (!fs.getFileStatus(file).getPermission().getUserAction().implies(FsAction.WRITE)) { - throw new ArgumentParserException("Insufficient permissions to write file: " + file, parser); - } - } - - private void verifyCanWriteParent(ArgumentParser parser, Path file) throws ArgumentParserException, IOException { - Path parent = file.getParent(); - if (parent == null || !fs.exists(parent) || !fs.getFileStatus(parent).getPermission().getUserAction().implies(FsAction.WRITE)) { - throw new ArgumentParserException("Cannot write parent of file: " + file, parser); - } - } - - private void verifyCanExecute(ArgumentParser parser, Path file) throws ArgumentParserException, IOException { - verifyExists(parser, file); - if (!fs.getFileStatus(file).getPermission().getUserAction().implies(FsAction.EXECUTE)) { - throw new ArgumentParserException("Insufficient permissions to execute file: " + file, parser); - } - } - - private void verifyIsAbsolute(ArgumentParser parser, Path file) throws ArgumentParserException { - if (!file.isAbsolute()) { - throw new ArgumentParserException("Not an absolute file: " + file, parser); - } - } - - private void verifyHasScheme(ArgumentParser parser, Path file) throws ArgumentParserException { - if (file.toUri().getScheme() == null) { - throw new ArgumentParserException("URI scheme is missing in path: " + file, parser); - } - } - - private void verifyScheme(ArgumentParser parser, Path file) throws ArgumentParserException { - if (!verifyScheme.equals(file.toUri().getScheme())) { - throw new ArgumentParserException("Scheme of path: " + file + " must be: " + verifyScheme, parser); - } - } - - private boolean isSystemIn(Path file) { - return acceptSystemIn && file.toString().equals("-"); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/PathParts.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/PathParts.java deleted file mode 100644 index 690901b4c76..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/PathParts.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.net.URI; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.server.namenode.NameNode; - -/** - * Extracts various components of an HDFS Path - */ -public final class PathParts { - - private final String uploadURL; - private final Configuration conf; - private final FileSystem fs; - private final Path normalizedPath; - private FileStatus stats; - - public PathParts(String uploadURL, Configuration conf) throws IOException { - if (uploadURL == null) { - throw new IllegalArgumentException("Path must not be null: " + uploadURL); - } - this.uploadURL = uploadURL; - if (conf == null) { - throw new IllegalArgumentException("Configuration must not be null: " + uploadURL); - } - this.conf = conf; - URI uri = stringToUri(uploadURL); - this.fs = FileSystem.get(uri, conf); - if (fs == null) { - throw new IllegalArgumentException("File system must not be null: " + uploadURL); - } - this.normalizedPath = fs.makeQualified(new Path(uri)); - if (!normalizedPath.isAbsolute()) { - throw new IllegalArgumentException("Path must be absolute: " + uploadURL); - } - if (getScheme() == null) { - throw new IllegalArgumentException("Scheme must not be null: " + uploadURL); - } - if (getHost() == null) { - throw new IllegalArgumentException("Host must not be null: " + uploadURL); - } - if (getPort() < 0) { - throw new IllegalArgumentException("Port must not be negative: " + uploadURL); - } - } - - public String getUploadURL() { - return uploadURL; - } - - public Path getUploadPath() { - return new Path(getUploadURL()); - } - - public String getURIPath() { - return normalizedPath.toUri().getPath(); - } - - public String getName() { - return normalizedPath.getName(); - } - - public String getScheme() { - return normalizedPath.toUri().getScheme(); - } - - public String getHost() { - return normalizedPath.toUri().getHost(); - } - - public int getPort() { - int port = normalizedPath.toUri().getPort(); - if (port == -1) { - port = fs.getWorkingDirectory().toUri().getPort(); - if (port == -1) { - port = NameNode.DEFAULT_PORT; - } - } - return port; - } - - public String getId() { - return getScheme() + "://" + getHost() + ":" + getPort() + getURIPath(); - } - - public String getDownloadURL() { - return getId(); - } - - public Configuration getConfiguration() { - return conf; - } - - public FileSystem getFileSystem() { - return fs; - } - - public FileStatus getFileStatus() throws IOException { - if (stats == null) { - stats = getFileSystem().getFileStatus(getUploadPath()); - } - return stats; - } - - private URI stringToUri(String pathString) { - //return new Path(pathString).toUri().normalize(); - return URI.create(pathString).normalize(); - } -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrCloudPartitioner.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrCloudPartitioner.java deleted file mode 100644 index c8ad1a79a73..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrCloudPartitioner.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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.hadoop; - -import java.lang.invoke.MethodHandles; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Partitioner; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.common.cloud.DocCollection; -import org.apache.solr.common.cloud.DocRouter; -import org.apache.solr.common.cloud.Slice; -import org.apache.solr.common.params.MapSolrParams; -import org.apache.solr.common.params.SolrParams; -import org.apache.solr.common.util.Hash; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * MapReduce partitioner that partitions the Mapper output such that each - * SolrInputDocument gets sent to the SolrCloud shard that it would have been - * sent to if the document were ingested via the standard SolrCloud Near Real - * Time (NRT) API. - * - * In other words, this class implements the same partitioning semantics as the - * standard SolrCloud NRT API. This enables to mix batch updates from MapReduce - * ingestion with updates from standard NRT ingestion on the same SolrCloud - * cluster, using identical unique document keys. - */ -public class SolrCloudPartitioner extends Partitioner implements Configurable { - - private Configuration conf; - private DocCollection docCollection; - private Map shardNumbers; - private int shards = 0; - private final SolrParams emptySolrParams = new MapSolrParams(Collections.EMPTY_MAP); - - public static final String SHARDS = SolrCloudPartitioner.class.getName() + ".shards"; - public static final String ZKHOST = SolrCloudPartitioner.class.getName() + ".zkHost"; - public static final String COLLECTION = SolrCloudPartitioner.class.getName() + ".collection"; - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public SolrCloudPartitioner() {} - - @Override - public void setConf(Configuration conf) { - this.conf = conf; - this.shards = conf.getInt(SHARDS, -1); - if (shards <= 0) { - throw new IllegalArgumentException("Illegal shards: " + shards); - } - String zkHost = conf.get(ZKHOST); - if (zkHost == null) { - throw new IllegalArgumentException("zkHost must not be null"); - } - String collection = conf.get(COLLECTION); - if (collection == null) { - throw new IllegalArgumentException("collection must not be null"); - } - LOG.info("Using SolrCloud zkHost: {}, collection: {}", zkHost, collection); - docCollection = new ZooKeeperInspector().extractDocCollection(zkHost, collection); - if (docCollection == null) { - throw new IllegalArgumentException("docCollection must not be null"); - } - if (docCollection.getSlicesMap().size() != shards) { - throw new IllegalArgumentException("Incompatible shards: + " + shards + " for docCollection: " + docCollection); - } - List slices = new ZooKeeperInspector().getSortedSlices(docCollection.getSlices()); - if (slices.size() != shards) { - throw new IllegalStateException("Incompatible sorted shards: + " + shards + " for docCollection: " + docCollection); - } - shardNumbers = new HashMap(10 * slices.size()); // sparse for performance - for (int i = 0; i < slices.size(); i++) { - shardNumbers.put(slices.get(i).getName(), i); - } - LOG.debug("Using SolrCloud docCollection: {}", docCollection); - DocRouter docRouter = docCollection.getRouter(); - if (docRouter == null) { - throw new IllegalArgumentException("docRouter must not be null"); - } - LOG.info("Using SolrCloud docRouterClass: {}", docRouter.getClass()); - } - - @Override - public Configuration getConf() { - return conf; - } - - @Override - public int getPartition(Text key, SolrInputDocumentWritable value, int numPartitions) { - DocRouter docRouter = docCollection.getRouter(); - SolrInputDocument doc = value.getSolrInputDocument(); - String keyStr = key.toString(); - - // TODO: scalability: replace linear search in HashBasedRouter.hashToSlice() with binary search on sorted hash ranges - Slice slice = docRouter.getTargetSlice(keyStr, doc, null, emptySolrParams, docCollection); - -// LOG.info("slice: {}", slice); - if (slice == null) { - throw new IllegalStateException("No matching slice found! The slice seems unavailable. docRouterClass: " - + docRouter.getClass().getName()); - } - int rootShard = shardNumbers.get(slice.getName()); - if (rootShard < 0 || rootShard >= shards) { - throw new IllegalStateException("Illegal shard number " + rootShard + " for slice: " + slice + ", docCollection: " - + docCollection); - } - - // map doc to micro shard aka leaf shard, akin to HashBasedRouter.sliceHash() - // taking into account mtree merge algorithm - assert numPartitions % shards == 0; // Also note that numPartitions is equal to the number of reducers - int hashCode = Hash.murmurhash3_x86_32(keyStr, 0, keyStr.length(), 0); - int offset = (hashCode & Integer.MAX_VALUE) % (numPartitions / shards); - int microShard = (rootShard * (numPartitions / shards)) + offset; -// LOG.info("Subpartitions rootShard: {}, offset: {}", rootShard, offset); -// LOG.info("Partitioned to p: {} for numPartitions: {}, shards: {}, key: {}, value: {}", microShard, numPartitions, shards, key, value); - - assert microShard >= 0 && microShard < numPartitions; - return microShard; - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrCounters.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrCounters.java deleted file mode 100644 index 88e9acb57cc..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrCounters.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.hadoop; - -public enum SolrCounters { - - DOCUMENTS_WRITTEN (getClassName(SolrReducer.class) - + ": Number of documents processed"), - - BATCHES_WRITTEN (getClassName(SolrReducer.class) - + ": Number of document batches processed"), - - BATCH_WRITE_TIME (getClassName(SolrReducer.class) - + ": Time spent by reducers writing batches [ms]"), - - PHYSICAL_REDUCER_MERGE_TIME (getClassName(SolrReducer.class) - + ": Time spent by reducers on physical merges [ms]"), - - LOGICAL_TREE_MERGE_TIME (getClassName(TreeMergeMapper.class) - + ": Time spent on logical tree merges [ms]"), - - PHYSICAL_TREE_MERGE_TIME (getClassName(TreeMergeMapper.class) - + ": Time spent on physical tree merges [ms]"); - - private final String label; - - private SolrCounters(String label) { - this.label = label; - } - - public String toString() { - return label; - } - - private static String getClassName(Class clazz) { - return Utils.getShortClassName(clazz); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrInputDocumentWritable.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrInputDocumentWritable.java deleted file mode 100644 index e043f7a0ed2..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrInputDocumentWritable.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.hadoop; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; - -import org.apache.hadoop.io.Writable; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.common.util.FastOutputStream; -import org.apache.solr.common.util.JavaBinCodec; - -public class SolrInputDocumentWritable implements Writable { - private SolrInputDocument sid; - - public SolrInputDocumentWritable() { - } - - public SolrInputDocumentWritable(SolrInputDocument sid) { - this.sid = sid; - } - - public SolrInputDocument getSolrInputDocument() { - return sid; - } - - @Override - public String toString() { - return sid.toString(); - } - - @Override - public void write(DataOutput out) throws IOException { - JavaBinCodec codec = new JavaBinCodec(); - FastOutputStream daos = FastOutputStream.wrap(DataOutputOutputStream.constructOutputStream(out)); - codec.init(daos); - try { - codec.writeVal(sid); - } finally { - daos.flushBuffer(); - } - } - - @Override - public void readFields(DataInput in) throws IOException { - JavaBinCodec codec = new JavaBinCodec(); - UnbufferedDataInputInputStream dis = new UnbufferedDataInputInputStream(in); - sid = (SolrInputDocument)codec.readVal(dis); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrMapper.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrMapper.java deleted file mode 100644 index 6c73f1bfe9f..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrMapper.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Mapper; - -public class SolrMapper extends Mapper { - - private Path solrHomeDir; - - @Override - protected void setup(Context context) throws IOException, InterruptedException { - Utils.getLogConfigFile(context.getConfiguration()); - super.setup(context); - solrHomeDir = SolrRecordWriter.findSolrConfig(context.getConfiguration()); - } - - protected Path getSolrHomeDir() { - return solrHomeDir; - } -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java deleted file mode 100644 index b52939e3f9d..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java +++ /dev/null @@ -1,280 +0,0 @@ -/* - * 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.hadoop; - -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.lang.invoke.MethodHandles; -import java.net.URI; -import java.nio.file.Files; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Locale; -import java.util.Set; -import java.util.UUID; -import java.util.zip.ZipEntry; -import java.util.zip.ZipOutputStream; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.filecache.DistributedCache; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.RecordWriter; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SolrOutputFormat extends FileOutputFormat { - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - /** - * The parameter used to pass the solr config zip file information. This will - * be the hdfs path to the configuration zip file - */ - public static final String SETUP_OK = "solr.output.format.setup"; - - /** The key used to pass the zip file name through the configuration. */ - public static final String ZIP_NAME = "solr.zip.name"; - - /** - * The base name of the zip file containing the configuration information. - * This file is passed via the distributed cache using a unique name, obtained - * via {@link #getZipName(Configuration jobConf)}. - */ - public static final String ZIP_FILE_BASE_NAME = "solr.zip"; - - /** - * The key used to pass the boolean configuration parameter that instructs for - * regular or zip file output - */ - public static final String OUTPUT_ZIP_FILE = "solr.output.zip.format"; - - static int defaultSolrWriterThreadCount = 0; - - public static final String SOLR_WRITER_THREAD_COUNT = "solr.record.writer.num.threads"; - - static int defaultSolrWriterQueueSize = 1; - - public static final String SOLR_WRITER_QUEUE_SIZE = "solr.record.writer.max.queues.size"; - - static int defaultSolrBatchSize = 20; - - public static final String SOLR_RECORD_WRITER_BATCH_SIZE = "solr.record.writer.batch.size"; - - public static final String SOLR_RECORD_WRITER_MAX_SEGMENTS = "solr.record.writer.maxSegments"; - - public static String getSetupOk() { - return SETUP_OK; - } - - /** Get the number of threads used for index writing */ - public static void setSolrWriterThreadCount(int count, Configuration conf) { - conf.setInt(SOLR_WRITER_THREAD_COUNT, count); - } - - /** Set the number of threads used for index writing */ - public static int getSolrWriterThreadCount(Configuration conf) { - return conf.getInt(SOLR_WRITER_THREAD_COUNT, defaultSolrWriterThreadCount); - } - - /** - * Set the maximum size of the the queue for documents to be written to the - * index. - */ - public static void setSolrWriterQueueSize(int count, Configuration conf) { - conf.setInt(SOLR_WRITER_QUEUE_SIZE, count); - } - - /** Return the maximum size for the number of documents pending index writing. */ - public static int getSolrWriterQueueSize(Configuration conf) { - return conf.getInt(SOLR_WRITER_QUEUE_SIZE, defaultSolrWriterQueueSize); - } - - /** - * Return the file name portion of the configuration zip file, from the - * configuration. - */ - public static String getZipName(Configuration conf) { - return conf.get(ZIP_NAME, ZIP_FILE_BASE_NAME); - } - - /** - * configure the job to output zip files of the output index, or full - * directory trees. Zip files are about 1/5th the size of the raw index, and - * much faster to write, but take more cpu to create. - * - * @param output true if should output zip files - * @param conf to use - */ - public static void setOutputZipFormat(boolean output, Configuration conf) { - conf.setBoolean(OUTPUT_ZIP_FILE, output); - } - - /** - * return true if the output should be a zip file of the index, rather than - * the raw index - * - * @param conf to use - * @return true if output zip files is on - */ - public static boolean isOutputZipFormat(Configuration conf) { - return conf.getBoolean(OUTPUT_ZIP_FILE, false); - } - - public static String getOutputName(JobContext job) { - return FileOutputFormat.getOutputName(job); - } - - @Override - public void checkOutputSpecs(JobContext job) throws IOException { - super.checkOutputSpecs(job); - if (job.getConfiguration().get(SETUP_OK) == null) { - throw new IOException("Solr home cache not set up!"); - } - } - - - @Override - public RecordWriter getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException { - Utils.getLogConfigFile(context.getConfiguration()); - Path workDir = getDefaultWorkFile(context, ""); - int batchSize = getBatchSize(context.getConfiguration()); - return new SolrRecordWriter<>(context, workDir, batchSize); - } - - public static void setupSolrHomeCache(File solrHomeDir, Job job) throws IOException{ - File solrHomeZip = createSolrHomeZip(solrHomeDir); - addSolrConfToDistributedCache(job, solrHomeZip); - } - - public static File createSolrHomeZip(File solrHomeDir) throws IOException { - return createSolrHomeZip(solrHomeDir, false); - } - - private static File createSolrHomeZip(File solrHomeDir, boolean safeToModify) throws IOException { - if (solrHomeDir == null || !(solrHomeDir.exists() && solrHomeDir.isDirectory())) { - throw new IOException("Invalid solr home: " + solrHomeDir); - } - File solrHomeZip = File.createTempFile("solr", ".zip"); - createZip(solrHomeDir, solrHomeZip); - return solrHomeZip; - } - - public static void addSolrConfToDistributedCache(Job job, File solrHomeZip) - throws IOException { - // Make a reasonably unique name for the zip file in the distributed cache - // to avoid collisions if multiple jobs are running. - String hdfsZipName = UUID.randomUUID().toString() + '.' - + ZIP_FILE_BASE_NAME; - Configuration jobConf = job.getConfiguration(); - jobConf.set(ZIP_NAME, hdfsZipName); - - Path zipPath = new Path("/tmp", getZipName(jobConf)); - FileSystem fs = FileSystem.get(jobConf); - fs.copyFromLocalFile(new Path(solrHomeZip.toString()), zipPath); - final URI baseZipUrl = fs.getUri().resolve( - zipPath.toString() + '#' + getZipName(jobConf)); - - DistributedCache.addCacheArchive(baseZipUrl, jobConf); - LOG.debug("Set Solr distributed cache: {}", Arrays.asList(job.getCacheArchives())); - LOG.debug("Set zipPath: {}", zipPath); - // Actually send the path for the configuration zip file - jobConf.set(SETUP_OK, zipPath.toString()); - } - - private static void createZip(File dir, File out) throws IOException { - HashSet files = new HashSet<>(); - // take only conf/ and lib/ - for (String allowedDirectory : SolrRecordWriter - .getAllowedConfigDirectories()) { - File configDir = new File(dir, allowedDirectory); - boolean configDirExists; - /** If the directory does not exist, and is required, bail out */ - if (!(configDirExists = configDir.exists()) - && SolrRecordWriter.isRequiredConfigDirectory(allowedDirectory)) { - throw new IOException(String.format(Locale.ENGLISH, - "required configuration directory %s is not present in %s", - allowedDirectory, dir)); - } - if (!configDirExists) { - continue; - } - listFiles(configDir, files); // Store the files in the existing, allowed - // directory configDir, in the list of files - // to store in the zip file - } - - Files.deleteIfExists(out.toPath()); - int subst = dir.toString().length(); - ZipOutputStream zos = new ZipOutputStream(new FileOutputStream(out)); - byte[] buf = new byte[1024]; - for (File f : files) { - ZipEntry ze = new ZipEntry(f.toString().substring(subst)); - zos.putNextEntry(ze); - InputStream is = new FileInputStream(f); - int cnt; - while ((cnt = is.read(buf)) >= 0) { - zos.write(buf, 0, cnt); - } - is.close(); - zos.flush(); - zos.closeEntry(); - } - - ZipEntry ze = new ZipEntry("solr.xml"); - zos.putNextEntry(ze); - zos.write("".getBytes("UTF-8")); - zos.flush(); - zos.closeEntry(); - zos.close(); - } - - private static void listFiles(File dir, Set files) throws IOException { - File[] list = dir.listFiles(); - - if (list == null && dir.isFile()) { - files.add(dir); - return; - } - - for (File f : list) { - if (f.isFile()) { - files.add(f); - } else { - listFiles(f, files); - } - } - } - - public static int getBatchSize(Configuration jobConf) { - // TODO Auto-generated method stub - return jobConf.getInt(SolrOutputFormat.SOLR_RECORD_WRITER_BATCH_SIZE, - defaultSolrBatchSize); - } - - public static void setBatchSize(int count, Configuration jobConf) { - jobConf.setInt(SOLR_RECORD_WRITER_BATCH_SIZE, count); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrRecordWriter.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrRecordWriter.java deleted file mode 100644 index f528c5425ba..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrRecordWriter.java +++ /dev/null @@ -1,479 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Locale; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import com.google.common.collect.ImmutableMap; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.filecache.DistributedCache; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapreduce.RecordWriter; -import org.apache.hadoop.mapreduce.Reducer; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskID; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.core.CoreContainer; -import org.apache.solr.core.CoreDescriptor; -import org.apache.solr.core.DirectoryFactory; -import org.apache.solr.core.HdfsDirectoryFactory; -import org.apache.solr.core.SolrCore; -import org.apache.solr.core.SolrResourceLoader; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class SolrRecordWriter extends RecordWriter { - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public final static List allowedConfigDirectories = new ArrayList<>( - Arrays.asList(new String[] { "conf", "lib", "solr.xml", "core1" })); - - public final static Set requiredConfigDirectories = new HashSet<>(); - - static { - requiredConfigDirectories.add("conf"); - } - - /** - * Return the list of directories names that may be included in the - * configuration data passed to the tasks. - * - * @return an UnmodifiableList of directory names - */ - public static List getAllowedConfigDirectories() { - return Collections.unmodifiableList(allowedConfigDirectories); - } - - /** - * check if the passed in directory is required to be present in the - * configuration data set. - * - * @param directory The directory to check - * @return true if the directory is required. - */ - public static boolean isRequiredConfigDirectory(final String directory) { - return requiredConfigDirectories.contains(directory); - } - - /** The path that the final index will be written to */ - - /** The location in a local temporary directory that the index is built in. */ - -// /** -// * If true, create a zip file of the completed index in the final storage -// * location A .zip will be appended to the final output name if it is not -// * already present. -// */ -// private boolean outputZipFile = false; - - private final HeartBeater heartBeater; - private final BatchWriter batchWriter; - private final List batch; - private final int batchSize; - private long numDocsWritten = 0; - private long nextLogTime = System.nanoTime(); - - private static HashMap.Context> contextMap = new HashMap<>(); - - public SolrRecordWriter(TaskAttemptContext context, Path outputShardDir, int batchSize) { - this.batchSize = batchSize; - this.batch = new ArrayList<>(batchSize); - Configuration conf = context.getConfiguration(); - - // setLogLevel("org.apache.solr.core", "WARN"); - // setLogLevel("org.apache.solr.update", "WARN"); - - heartBeater = new HeartBeater(context); - try { - heartBeater.needHeartBeat(); - - Path solrHomeDir = SolrRecordWriter.findSolrConfig(conf); - FileSystem fs = outputShardDir.getFileSystem(conf); - EmbeddedSolrServer solr = createEmbeddedSolrServer(solrHomeDir, fs, outputShardDir); - batchWriter = new BatchWriter(solr, batchSize, - context.getTaskAttemptID().getTaskID(), - SolrOutputFormat.getSolrWriterThreadCount(conf), - SolrOutputFormat.getSolrWriterQueueSize(conf)); - - } catch (Exception e) { - throw new IllegalStateException(String.format(Locale.ENGLISH, - "Failed to initialize record writer for %s, %s", context.getJobName(), conf - .get("mapred.task.id")), e); - } finally { - heartBeater.cancelHeartBeat(); - } - } - - public static EmbeddedSolrServer createEmbeddedSolrServer(Path solrHomeDir, FileSystem fs, Path outputShardDir) - throws IOException { - - LOG.info("Creating embedded Solr server with solrHomeDir: " + solrHomeDir + ", fs: " + fs + ", outputShardDir: " + outputShardDir); - - Path solrDataDir = new Path(outputShardDir, "data"); - - String dataDirStr = solrDataDir.toUri().toString(); - - SolrResourceLoader loader = new SolrResourceLoader(Paths.get(solrHomeDir.toString()), null, null); - - LOG.info(String - .format(Locale.ENGLISH, - "Constructed instance information solr.home %s (%s), instance dir %s, conf dir %s, writing index to solr.data.dir %s, with permdir %s", - solrHomeDir, solrHomeDir.toUri(), loader.getInstancePath(), - loader.getConfigDir(), dataDirStr, outputShardDir)); - - // TODO: This is fragile and should be well documented - System.setProperty("solr.directoryFactory", HdfsDirectoryFactory.class.getName()); - System.setProperty("solr.lock.type", DirectoryFactory.LOCK_TYPE_HDFS); - System.setProperty("solr.hdfs.nrtcachingdirectory", "false"); - System.setProperty("solr.hdfs.blockcache.enabled", "false"); - System.setProperty("solr.autoCommit.maxTime", "600000"); - System.setProperty("solr.autoSoftCommit.maxTime", "-1"); - - CoreContainer container = new CoreContainer(loader); - container.load(); - SolrCore core = container.create("", ImmutableMap.of(CoreDescriptor.CORE_DATADIR, dataDirStr)); - - if (!(core.getDirectoryFactory() instanceof HdfsDirectoryFactory)) { - throw new UnsupportedOperationException( - "Invalid configuration. Currently, the only DirectoryFactory supported is " - + HdfsDirectoryFactory.class.getSimpleName()); - } - - EmbeddedSolrServer solr = new EmbeddedSolrServer(container, ""); - return solr; - } - - public static void incrementCounter(TaskID taskId, String groupName, String counterName, long incr) { - Reducer.Context context = contextMap.get(taskId); - if (context != null) { - context.getCounter(groupName, counterName).increment(incr); - } - } - - public static void incrementCounter(TaskID taskId, Enum counterName, long incr) { - Reducer.Context context = contextMap.get(taskId); - if (context != null) { - context.getCounter(counterName).increment(incr); - } - } - - public static void addReducerContext(Reducer.Context context) { - TaskID taskID = context.getTaskAttemptID().getTaskID(); - contextMap.put(taskID, context); - } - - public static Path findSolrConfig(Configuration conf) throws IOException { - // FIXME when mrunit supports the new cache apis - //URI[] localArchives = context.getCacheArchives(); - Path[] localArchives = DistributedCache.getLocalCacheArchives(conf); - for (Path unpackedDir : localArchives) { - if (unpackedDir.getName().equals(SolrOutputFormat.getZipName(conf))) { - LOG.info("Using this unpacked directory as solr home: {}", unpackedDir); - return unpackedDir; - } - } - throw new IOException(String.format(Locale.ENGLISH, - "No local cache archives, where is %s:%s", SolrOutputFormat - .getSetupOk(), SolrOutputFormat.getZipName(conf))); - } - - /** - * Write a record. This method accumulates records in to a batch, and when - * {@link #batchSize} items are present flushes it to the indexer. The writes - * can take a substantial amount of time, depending on {@link #batchSize}. If - * there is heavy disk contention the writes may take more than the 600 second - * default timeout. - */ - @Override - public void write(K key, V value) throws IOException { - heartBeater.needHeartBeat(); - try { - try { - SolrInputDocumentWritable sidw = (SolrInputDocumentWritable) value; - batch.add(sidw.getSolrInputDocument()); - if (batch.size() >= batchSize) { - batchWriter.queueBatch(batch); - numDocsWritten += batch.size(); - if (System.nanoTime() >= nextLogTime) { - LOG.info("docsWritten: {}", numDocsWritten); - nextLogTime += TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS); - } - batch.clear(); - } - } catch (SolrServerException e) { - throw new IOException(e); - } - } finally { - heartBeater.cancelHeartBeat(); - } - - } - - @Override - public void close(TaskAttemptContext context) throws IOException, InterruptedException { - if (context != null) { - heartBeater.setProgress(context); - } - try { - heartBeater.needHeartBeat(); - if (batch.size() > 0) { - batchWriter.queueBatch(batch); - numDocsWritten += batch.size(); - batch.clear(); - } - LOG.info("docsWritten: {}", numDocsWritten); - batchWriter.close(context); -// if (outputZipFile) { -// context.setStatus("Writing Zip"); -// packZipFile(); // Written to the perm location -// } else { -// context.setStatus("Copying Index"); -// fs.completeLocalOutput(perm, temp); // copy to dfs -// } - } catch (Exception e) { - if (e instanceof IOException) { - throw (IOException) e; - } - throw new IOException(e); - } finally { - heartBeater.cancelHeartBeat(); - heartBeater.close(); -// File tempFile = new File(temp.toString()); -// if (tempFile.exists()) { -// FileUtils.forceDelete(new File(temp.toString())); -// } - } - - context.setStatus("Done"); - } - -// private void packZipFile() throws IOException { -// FSDataOutputStream out = null; -// ZipOutputStream zos = null; -// int zipCount = 0; -// LOG.info("Packing zip file for " + perm); -// try { -// out = fs.create(perm, false); -// zos = new ZipOutputStream(out); -// -// String name = perm.getName().replaceAll(".zip$", ""); -// LOG.info("adding index directory" + temp); -// zipCount = zipDirectory(conf, zos, name, temp.toString(), temp); -// /** -// for (String configDir : allowedConfigDirectories) { -// if (!isRequiredConfigDirectory(configDir)) { -// continue; -// } -// final Path confPath = new Path(solrHome, configDir); -// LOG.info("adding configdirectory" + confPath); -// -// zipCount += zipDirectory(conf, zos, name, solrHome.toString(), confPath); -// } -// **/ -// } catch (Throwable ohFoo) { -// LOG.error("packZipFile exception", ohFoo); -// if (ohFoo instanceof RuntimeException) { -// throw (RuntimeException) ohFoo; -// } -// if (ohFoo instanceof IOException) { -// throw (IOException) ohFoo; -// } -// throw new IOException(ohFoo); -// -// } finally { -// if (zos != null) { -// if (zipCount == 0) { // If no entries were written, only close out, as -// // the zip will throw an error -// LOG.error("No entries written to zip file " + perm); -// fs.delete(perm, false); -// // out.close(); -// } else { -// LOG.info(String.format("Wrote %d items to %s for %s", zipCount, perm, -// temp)); -// zos.close(); -// } -// } -// } -// } -// -// /** -// * Write a file to a zip output stream, removing leading path name components -// * from the actual file name when creating the zip file entry. -// * -// * The entry placed in the zip file is baseName/ -// * relativePath, where relativePath is constructed -// * by removing a leading root from the path for -// * itemToZip. -// * -// * If itemToZip is an empty directory, it is ignored. If -// * itemToZip is a directory, the contents of the directory are -// * added recursively. -// * -// * @param zos The zip output stream -// * @param baseName The base name to use for the file name entry in the zip -// * file -// * @param root The path to remove from itemToZip to make a -// * relative path name -// * @param itemToZip The path to the file to be added to the zip file -// * @return the number of entries added -// * @throws IOException -// */ -// static public int zipDirectory(final Configuration conf, -// final ZipOutputStream zos, final String baseName, final String root, -// final Path itemToZip) throws IOException { -// LOG -// .info(String -// .format("zipDirectory: %s %s %s", baseName, root, itemToZip)); -// LocalFileSystem localFs = FileSystem.getLocal(conf); -// int count = 0; -// -// final FileStatus itemStatus = localFs.getFileStatus(itemToZip); -// if (itemStatus.isDirectory()) { -// final FileStatus[] statai = localFs.listStatus(itemToZip); -// -// // Add a directory entry to the zip file -// final String zipDirName = relativePathForZipEntry(itemToZip.toUri() -// .getPath(), baseName, root); -// final ZipEntry dirZipEntry = new ZipEntry(zipDirName -// + Path.SEPARATOR_CHAR); -// LOG.info(String.format("Adding directory %s to zip", zipDirName)); -// zos.putNextEntry(dirZipEntry); -// zos.closeEntry(); -// count++; -// -// if (statai == null || statai.length == 0) { -// LOG.info(String.format("Skipping empty directory %s", itemToZip)); -// return count; -// } -// for (FileStatus status : statai) { -// count += zipDirectory(conf, zos, baseName, root, status.getPath()); -// } -// LOG.info(String.format("Wrote %d entries for directory %s", count, -// itemToZip)); -// return count; -// } -// -// final String inZipPath = relativePathForZipEntry(itemToZip.toUri() -// .getPath(), baseName, root); -// -// if (inZipPath.length() == 0) { -// LOG.warn(String.format("Skipping empty zip file path for %s (%s %s)", -// itemToZip, root, baseName)); -// return 0; -// } -// -// // Take empty files in case the place holder is needed -// FSDataInputStream in = null; -// try { -// in = localFs.open(itemToZip); -// final ZipEntry ze = new ZipEntry(inZipPath); -// ze.setTime(itemStatus.getModificationTime()); -// // Comments confuse looking at the zip file -// // ze.setComment(itemToZip.toString()); -// zos.putNextEntry(ze); -// -// IOUtils.copyBytes(in, zos, conf, false); -// zos.closeEntry(); -// LOG.info(String.format("Wrote %d entries for file %s", count, itemToZip)); -// return 1; -// } finally { -// in.close(); -// } -// -// } -// -// static String relativePathForZipEntry(final String rawPath, -// final String baseName, final String root) { -// String relativePath = rawPath.replaceFirst(Pattern.quote(root.toString()), -// ""); -// LOG.info(String.format("RawPath %s, baseName %s, root %s, first %s", -// rawPath, baseName, root, relativePath)); -// -// if (relativePath.startsWith(Path.SEPARATOR)) { -// relativePath = relativePath.substring(1); -// } -// LOG.info(String.format( -// "RawPath %s, baseName %s, root %s, post leading slash %s", rawPath, -// baseName, root, relativePath)); -// if (relativePath.isEmpty()) { -// LOG.warn(String.format( -// "No data after root (%s) removal from raw path %s", root, rawPath)); -// return baseName; -// } -// // Construct the path that will be written to the zip file, including -// // removing any leading '/' characters -// String inZipPath = baseName + Path.SEPARATOR_CHAR + relativePath; -// -// LOG.info(String.format("RawPath %s, baseName %s, root %s, inZip 1 %s", -// rawPath, baseName, root, inZipPath)); -// if (inZipPath.startsWith(Path.SEPARATOR)) { -// inZipPath = inZipPath.substring(1); -// } -// LOG.info(String.format("RawPath %s, baseName %s, root %s, inZip 2 %s", -// rawPath, baseName, root, inZipPath)); -// -// return inZipPath; -// -// } -// - /* - static boolean setLogLevel(String packageName, String level) { - Log logger = LogFactory.getLog(packageName); - if (logger == null) { - return false; - } - // look for: org.apache.commons.logging.impl.SLF4JLocationAwareLog - LOG.warn("logger class:"+logger.getClass().getName()); - if (logger instanceof Log4JLogger) { - process(((Log4JLogger) logger).getLogger(), level); - return true; - } - if (logger instanceof Jdk14Logger) { - process(((Jdk14Logger) logger).getLogger(), level); - return true; - } - return false; - } - - public static void process(org.apache.log4j.Logger log, String level) { - if (level != null) { - log.setLevel(org.apache.log4j.Level.toLevel(level)); - } - } - - public static void process(java.util.logging.Logger log, String level) { - if (level != null) { - log.setLevel(java.util.logging.Level.parse(level)); - } - } - */ -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrReducer.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrReducer.java deleted file mode 100644 index 5e10c9719af..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrReducer.java +++ /dev/null @@ -1,188 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.Iterator; - -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Reducer; -import org.apache.hadoop.util.ReflectionUtils; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.hadoop.dedup.NoChangeUpdateConflictResolver; -import org.apache.solr.hadoop.dedup.RetainMostRecentUpdateConflictResolver; -import org.apache.solr.hadoop.dedup.UpdateConflictResolver; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.kitesdk.morphline.api.ExceptionHandler; -import org.kitesdk.morphline.base.FaultTolerance; -import com.google.common.base.Preconditions; - -/** - * This class loads the mapper's SolrInputDocuments into one EmbeddedSolrServer - * per reducer. Each such reducer and Solr server can be seen as a (micro) - * shard. The Solr servers store their data in HDFS. - * - * More specifically, this class consumes a list of <docId, SolrInputDocument> - * pairs, sorted by docId, and sends them to an embedded Solr server to generate - * a Solr index shard from the documents. - */ -public class SolrReducer extends Reducer { - - private UpdateConflictResolver resolver; - private HeartBeater heartBeater; - private ExceptionHandler exceptionHandler; - - public static final String UPDATE_CONFLICT_RESOLVER = SolrReducer.class.getName() + ".updateConflictResolver"; - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - @Override - protected void setup(Context context) throws IOException, InterruptedException { - verifyPartitionAssignment(context); - SolrRecordWriter.addReducerContext(context); - Class resolverClass = context.getConfiguration().getClass( - UPDATE_CONFLICT_RESOLVER, RetainMostRecentUpdateConflictResolver.class, UpdateConflictResolver.class); - - this.resolver = ReflectionUtils.newInstance(resolverClass, context.getConfiguration()); - /* - * Note that ReflectionUtils.newInstance() above also implicitly calls - * resolver.configure(context.getConfiguration()) if the resolver - * implements org.apache.hadoop.conf.Configurable - */ - - this.exceptionHandler = new FaultTolerance( - context.getConfiguration().getBoolean(FaultTolerance.IS_PRODUCTION_MODE, false), - context.getConfiguration().getBoolean(FaultTolerance.IS_IGNORING_RECOVERABLE_EXCEPTIONS, false), - context.getConfiguration().get(FaultTolerance.RECOVERABLE_EXCEPTION_CLASSES, SolrServerException.class.getName())); - - this.heartBeater = new HeartBeater(context); - } - - protected void reduce(Text key, Iterable values, Context context) throws IOException, InterruptedException { - heartBeater.needHeartBeat(); - try { - values = resolve(key, values, context); - super.reduce(key, values, context); - } catch (Exception e) { - LOG.error("Unable to process key " + key, e); - context.getCounter(getClass().getName() + ".errors", e.getClass().getName()).increment(1); - exceptionHandler.handleException(e, null); - } finally { - heartBeater.cancelHeartBeat(); - } - } - - private Iterable resolve( - final Text key, final Iterable values, final Context context) { - - if (resolver instanceof NoChangeUpdateConflictResolver) { - return values; // fast path - } - return new Iterable() { - @Override - public Iterator iterator() { - return new WrapIterator(resolver.orderUpdates(key, new UnwrapIterator(values.iterator()), context)); - } - }; - } - - @Override - protected void cleanup(Context context) throws IOException, InterruptedException { - heartBeater.close(); - super.cleanup(context); - } - - /* - * Verify that if a mappers's partitioner sends an item to partition X it implies that said item - * is sent to the reducer with taskID == X. This invariant is currently required for Solr - * documents to end up in the right Solr shard. - */ - private void verifyPartitionAssignment(Context context) { - if ("true".equals(System.getProperty("verifyPartitionAssignment", "true"))) { - String partitionStr = context.getConfiguration().get("mapred.task.partition"); - if (partitionStr == null) { - partitionStr = context.getConfiguration().get("mapreduce.task.partition"); - } - int partition = Integer.parseInt(partitionStr); - int taskId = context.getTaskAttemptID().getTaskID().getId(); - Preconditions.checkArgument(partition == taskId, - "mapred.task.partition: " + partition + " not equal to reducer taskId: " + taskId); - } - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class WrapIterator implements Iterator { - - private Iterator parent; - - private WrapIterator(Iterator parent) { - this.parent = parent; - } - - @Override - public boolean hasNext() { - return parent.hasNext(); - } - - @Override - public SolrInputDocumentWritable next() { - return new SolrInputDocumentWritable(parent.next()); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class UnwrapIterator implements Iterator { - - private Iterator parent; - - private UnwrapIterator(Iterator parent) { - this.parent = parent; - } - - @Override - public boolean hasNext() { - return parent.hasNext(); - } - - @Override - public SolrInputDocument next() { - return parent.next().getSolrInputDocument(); - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ToolRunnerHelpFormatter.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ToolRunnerHelpFormatter.java deleted file mode 100644 index 7570493d49d..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ToolRunnerHelpFormatter.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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.hadoop; - -import java.io.BufferedReader; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.PrintStream; -import java.io.PrintWriter; -import java.io.StringReader; -import java.io.StringWriter; -import java.io.UnsupportedEncodingException; -import java.nio.charset.StandardCharsets; - -import net.sourceforge.argparse4j.ArgumentParsers; -import net.sourceforge.argparse4j.helper.ASCIITextWidthCounter; -import net.sourceforge.argparse4j.helper.TextHelper; - -import org.apache.hadoop.util.ToolRunner; - -/** - * Nicely formats the output of - * {@link ToolRunner#printGenericCommandUsage(PrintStream)} with the same look and feel that argparse4j uses for help text. - */ -class ToolRunnerHelpFormatter { - - public static String getGenericCommandUsage() { - ByteArrayOutputStream bout = new ByteArrayOutputStream(); - String msg; - try { - ToolRunner.printGenericCommandUsage(new PrintStream(bout, true, "UTF-8")); - msg = new String(bout.toByteArray(), StandardCharsets.UTF_8); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); // unreachable - } - - BufferedReader reader = new BufferedReader(new StringReader(msg)); - StringBuilder result = new StringBuilder(); - while (true) { - String line; - try { - line = reader.readLine(); - } catch (IOException e) { - throw new RuntimeException(e); // unreachable - } - - if (line == null) { - return result.toString(); // EOS - } - - if (!line.startsWith("-")) { - result.append(line + "\n"); - } else { - line = line.trim(); - int i = line.indexOf(" "); - if (i < 0) { - i = line.indexOf('\t'); - } - if (i < 0) { - result.append(line + "\n"); - } else { - String title = line.substring(0, i).trim(); - if (title.length() >= 3 && Character.isLetterOrDigit(title.charAt(1)) && Character.isLetterOrDigit(title.charAt(2))) { - title = "-" + title; // prefer "--libjars" long arg style over "-libjars" style but retain "-D foo" short arg style - } - String help = line.substring(i, line.length()).trim(); - StringWriter strWriter = new StringWriter(); - PrintWriter writer = new PrintWriter(strWriter, true); - TextHelper.printHelp(writer, title, help, new ASCIITextWidthCounter(), ArgumentParsers.getFormatWidth()); - result.append(strWriter.toString()); - } - } - } - } -} - diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeMapper.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeMapper.java deleted file mode 100644 index ee34a9c32d3..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeMapper.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; - -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Mapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * For the meat see {@link TreeMergeOutputFormat}. - */ -public class TreeMergeMapper extends Mapper { - - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public static final String MAX_SEGMENTS_ON_TREE_MERGE = "maxSegmentsOnTreeMerge"; - - public static final String SOLR_SHARD_NUMBER = "_solrShardNumber"; - - @Override - protected void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException { - LOGGER.trace("map key: {}, value: {}", key, value); - context.write(value, NullWritable.get()); - } - -} \ No newline at end of file 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 deleted file mode 100644 index cac57c35aae..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java +++ /dev/null @@ -1,201 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.io.OutputStream; -import java.io.OutputStreamWriter; -import java.io.Writer; -import java.lang.invoke.MethodHandles; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.RecordWriter; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.IndexWriterConfig.OpenMode; -import org.apache.lucene.index.LogMergePolicy; -import org.apache.lucene.index.MergePolicy; -import org.apache.lucene.index.TieredMergePolicy; -import org.apache.lucene.misc.IndexMergeTool; -import org.apache.lucene.store.Directory; -import org.apache.solr.store.hdfs.HdfsDirectory; -import org.apache.solr.update.SolrIndexWriter; -import org.apache.solr.util.RTimer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * See {@link IndexMergeTool}. - */ -public class TreeMergeOutputFormat extends FileOutputFormat { - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - @Override - public RecordWriter getRecordWriter(TaskAttemptContext context) throws IOException { - Utils.getLogConfigFile(context.getConfiguration()); - Path workDir = getDefaultWorkFile(context, ""); - return new TreeMergeRecordWriter(context, workDir); - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class TreeMergeRecordWriter extends RecordWriter { - - private final Path workDir; - private final List shards = new ArrayList(); - private final HeartBeater heartBeater; - private final TaskAttemptContext context; - - private static final Logger LOG = log; - - public TreeMergeRecordWriter(TaskAttemptContext context, Path workDir) { - this.workDir = new Path(workDir, "data/index"); - this.heartBeater = new HeartBeater(context); - this.context = context; - } - - @Override - public void write(Text key, NullWritable value) { - LOG.info("map key: {}", key); - heartBeater.needHeartBeat(); - try { - Path path = new Path(key.toString()); - shards.add(path); - } finally { - heartBeater.cancelHeartBeat(); - } - } - - @Override - public void close(TaskAttemptContext context) throws IOException { - LOG.debug("Task " + context.getTaskAttemptID() + " merging into dstDir: " + workDir + ", srcDirs: " + shards); - writeShardNumberFile(context); - heartBeater.needHeartBeat(); - try { - Directory mergedIndex = new HdfsDirectory(workDir, context.getConfiguration()); - - // TODO: shouldn't we pull the Version from the solrconfig.xml? - IndexWriterConfig writerConfig = new IndexWriterConfig(null) - .setOpenMode(OpenMode.CREATE).setUseCompoundFile(false) - //.setMergePolicy(mergePolicy) // TODO: grab tuned MergePolicy from solrconfig.xml? - //.setMergeScheduler(...) // TODO: grab tuned MergeScheduler from solrconfig.xml? - ; - - if (LOG.isDebugEnabled()) { - writerConfig.setInfoStream(System.out); - } -// writerConfig.setRAMBufferSizeMB(100); // improve performance -// writerConfig.setMaxThreadStates(1); - - // disable compound file to improve performance - // also see http://lucene.472066.n3.nabble.com/Questions-on-compound-file-format-td489105.html - // also see defaults in SolrIndexConfig - MergePolicy mergePolicy = writerConfig.getMergePolicy(); - LOG.debug("mergePolicy was: {}", mergePolicy); - if (mergePolicy instanceof TieredMergePolicy) { - ((TieredMergePolicy) mergePolicy).setNoCFSRatio(0.0); -// ((TieredMergePolicy) mergePolicy).setMaxMergeAtOnceExplicit(10000); -// ((TieredMergePolicy) mergePolicy).setMaxMergeAtOnce(10000); -// ((TieredMergePolicy) mergePolicy).setSegmentsPerTier(10000); - } else if (mergePolicy instanceof LogMergePolicy) { - ((LogMergePolicy) mergePolicy).setNoCFSRatio(0.0); - } - LOG.info("Using mergePolicy: {}", mergePolicy); - - IndexWriter writer = new IndexWriter(mergedIndex, writerConfig); - - Directory[] indexes = new Directory[shards.size()]; - for (int i = 0; i < shards.size(); i++) { - indexes[i] = new HdfsDirectory(shards.get(i), context.getConfiguration()); - } - - context.setStatus("Logically merging " + shards.size() + " shards into one shard"); - LOG.info("Logically merging " + shards.size() + " shards into one shard: " + workDir); - RTimer timer = new RTimer(); - - writer.addIndexes(indexes); - // TODO: avoid intermediate copying of files into dst directory; rename the files into the dir instead (cp -> rename) - // This can improve performance and turns this phase into a true "logical" merge, completing in constant time. - // See https://issues.apache.org/jira/browse/LUCENE-4746 - - timer.stop(); - if (LOG.isDebugEnabled()) { - context.getCounter(SolrCounters.class.getName(), SolrCounters.LOGICAL_TREE_MERGE_TIME.toString()).increment((long) timer.getTime()); - } - LOG.info("Logical merge took {}ms", timer.getTime()); - int maxSegments = context.getConfiguration().getInt(TreeMergeMapper.MAX_SEGMENTS_ON_TREE_MERGE, Integer.MAX_VALUE); - context.setStatus("Optimizing Solr: forcing mtree merge down to " + maxSegments + " segments"); - LOG.info("Optimizing Solr: forcing tree merge down to {} segments", maxSegments); - timer = new RTimer(); - if (maxSegments < Integer.MAX_VALUE) { - writer.forceMerge(maxSegments); - // TODO: consider perf enhancement for no-deletes merges: bulk-copy the postings data - // see http://lucene.472066.n3.nabble.com/Experience-with-large-merge-factors-tp1637832p1647046.html - } - timer.stop(); - if (LOG.isDebugEnabled()) { - context.getCounter(SolrCounters.class.getName(), SolrCounters.PHYSICAL_TREE_MERGE_TIME.toString()).increment((long) timer.getTime()); - } - LOG.info("Optimizing Solr: done forcing tree merge down to {} segments in {}ms", maxSegments, timer.getTime()); - - // 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. - //TODO no commitUpdateCommand - SolrIndexWriter.setCommitData(writer, -1); - - timer = new RTimer(); - LOG.info("Optimizing Solr: Closing index writer"); - writer.close(); - LOG.info("Optimizing Solr: Done closing index writer in {}ms", timer.getTime()); - context.setStatus("Done"); - } finally { - heartBeater.cancelHeartBeat(); - heartBeater.close(); - } - } - - /* - * For background see MapReduceIndexerTool.renameTreeMergeShardDirs() - * - * Also see MapReduceIndexerTool.run() method where it uses - * NLineInputFormat.setNumLinesPerSplit(job, options.fanout) - */ - private void writeShardNumberFile(TaskAttemptContext context) throws IOException { - Preconditions.checkArgument(shards.size() > 0); - String shard = shards.get(0).getParent().getParent().getName(); // move up from "data/index" - String taskId = shard.substring("part-m-".length(), shard.length()); // e.g. part-m-00001 - int taskNum = Integer.parseInt(taskId); - int outputShardNum = taskNum / shards.size(); - LOG.debug("Merging into outputShardNum: " + outputShardNum + " from taskId: " + taskId); - Path shardNumberFile = new Path(workDir.getParent().getParent(), TreeMergeMapper.SOLR_SHARD_NUMBER); - OutputStream out = shardNumberFile.getFileSystem(context.getConfiguration()).create(shardNumberFile); - Writer writer = new OutputStreamWriter(out, StandardCharsets.UTF_8); - writer.write(String.valueOf(outputShardNum)); - writer.flush(); - writer.close(); - } - } -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/UnbufferedDataInputInputStream.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/UnbufferedDataInputInputStream.java deleted file mode 100644 index 83823ce8873..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/UnbufferedDataInputInputStream.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.hadoop; - -import java.io.BufferedReader; -import java.io.DataInput; -import java.io.DataInputStream; -import java.io.IOException; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; - -public class UnbufferedDataInputInputStream extends org.apache.solr.common.util.DataInputInputStream { - private final DataInputStream in; - - public UnbufferedDataInputInputStream(DataInput in) { - this.in = new DataInputStream(DataInputInputStream.constructInputStream(in)); - } - - @Override - public void readFully(byte[] b) throws IOException { - in.readFully(b); - } - - @Override - public void readFully(byte[] b, int off, int len) throws IOException { - in.readFully(b, off, len); - } - - @Override - public int skipBytes(int n) throws IOException { - return in.skipBytes(n); - } - - @Override - public boolean readBoolean() throws IOException { - return in.readBoolean(); - } - - @Override - public byte readByte() throws IOException { - return in.readByte(); - } - - @Override - public int readUnsignedByte() throws IOException { - return in.readUnsignedByte(); - } - - @Override - public short readShort() throws IOException { - return in.readShort(); - } - - @Override - public int readUnsignedShort() throws IOException { - return in.readUnsignedShort(); - } - - @Override - public char readChar() throws IOException { - return in.readChar(); - } - - @Override - public int readInt() throws IOException { - return in.readInt(); - } - - @Override - public long readLong() throws IOException { - return in.readLong(); - } - - @Override - public float readFloat() throws IOException { - return in.readFloat(); - } - - @Override - public double readDouble() throws IOException { - return in.readDouble(); - } - - @Override - public String readLine() throws IOException { - BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8)); - return reader.readLine(); - } - - @Override - public String readUTF() throws IOException { - return in.readUTF(); - } - - @Override - public int read() throws IOException { - return in.read(); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/Utils.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/Utils.java deleted file mode 100644 index fa101542939..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/Utils.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.hadoop; - -import java.io.File; - -import org.apache.hadoop.conf.Configuration; -import org.apache.log4j.PropertyConfigurator; - -import com.google.common.annotations.Beta; -import org.apache.solr.common.util.SuppressForbidden; - - -@Beta -public final class Utils { - - private static final String LOG_CONFIG_FILE = "hadoop.log4j.configuration"; - - public static void setLogConfigFile(File file, Configuration conf) { - conf.set(LOG_CONFIG_FILE, file.getName()); - } - - public static void getLogConfigFile(Configuration conf) { - String log4jPropertiesFile = conf.get(LOG_CONFIG_FILE); - configureLog4jProperties(log4jPropertiesFile); - } - - @SuppressForbidden(reason = "method is specific to log4j") - public static void configureLog4jProperties(String log4jPropertiesFile) { - if (log4jPropertiesFile != null) { - PropertyConfigurator.configure(log4jPropertiesFile); - } - } - - public static String getShortClassName(Class clazz) { - return getShortClassName(clazz.getName()); - } - - public static String getShortClassName(String className) { - int i = className.lastIndexOf('.'); // regular class - int j = className.lastIndexOf('$'); // inner class - return className.substring(1 + Math.max(i, j)); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ZooKeeperInspector.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ZooKeeperInspector.java deleted file mode 100644 index 76928aa17d7..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/ZooKeeperInspector.java +++ /dev/null @@ -1,213 +0,0 @@ -/* - * 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.hadoop; - -import java.io.File; -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; - -import com.google.common.io.Files; -import org.apache.commons.io.FileUtils; -import org.apache.solr.cloud.ZkController; -import org.apache.solr.common.SolrException; -import org.apache.solr.common.cloud.Aliases; -import org.apache.solr.common.cloud.ClusterState; -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.SolrZkClient; -import org.apache.solr.common.cloud.ZkConfigManager; -import org.apache.solr.common.cloud.ZkCoreNodeProps; -import org.apache.solr.common.cloud.ZkNodeProps; -import org.apache.solr.common.cloud.ZkStateReader; -import org.apache.solr.common.util.StrUtils; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Extracts SolrCloud information from ZooKeeper. - */ -final class ZooKeeperInspector { - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public List> extractShardUrls(String zkHost, String collection) { - - DocCollection docCollection = extractDocCollection(zkHost, collection); - List slices = getSortedSlices(docCollection.getSlices()); - List> solrUrls = new ArrayList<>(slices.size()); - for (Slice slice : slices) { - if (slice.getLeader() == null) { - throw new IllegalArgumentException("Cannot find SolrCloud slice leader. " + - "It looks like not all of your shards are registered in ZooKeeper yet"); - } - Collection replicas = slice.getReplicas(); - List urls = new ArrayList<>(replicas.size()); - for (Replica replica : replicas) { - ZkCoreNodeProps props = new ZkCoreNodeProps(replica); - urls.add(props.getCoreUrl()); - } - solrUrls.add(urls); - } - return solrUrls; - } - - public DocCollection extractDocCollection(String zkHost, String collection) { - if (collection == null) { - throw new IllegalArgumentException("collection must not be null"); - } - SolrZkClient zkClient = getZkClient(zkHost); - - try (ZkStateReader zkStateReader = new ZkStateReader(zkClient)) { - try { - // first check for alias - collection = checkForAlias(zkClient, collection); - zkStateReader.createClusterStateWatchersAndUpdate(); - } catch (Exception e) { - throw new IllegalArgumentException("Cannot find expected information for SolrCloud in ZooKeeper: " + zkHost, e); - } - - try { - return zkStateReader.getClusterState().getCollection(collection); - } catch (SolrException e) { - throw new IllegalArgumentException("Cannot find collection '" + collection + "' in ZooKeeper: " + zkHost, e); - } - } finally { - zkClient.close(); - } - } - - public SolrZkClient getZkClient(String zkHost) { - if (zkHost == null) { - throw new IllegalArgumentException("zkHost must not be null"); - } - - SolrZkClient zkClient; - try { - zkClient = new SolrZkClient(zkHost, 30000); - } catch (Exception e) { - throw new IllegalArgumentException("Cannot connect to ZooKeeper: " + zkHost, e); - } - return zkClient; - } - - public List getSortedSlices(Collection slices) { - List sorted = new ArrayList(slices); - Collections.sort(sorted, (slice1, slice2) -> { - Comparator c = new AlphaNumericComparator(); - return c.compare(slice1.getName(), slice2.getName()); - }); - LOG.trace("Sorted slices: {}", sorted); - return sorted; - } - - /** - * Returns config value given collection name - * Borrowed heavily from Solr's ZKController. - */ - public String readConfigName(SolrZkClient zkClient, String collection) - throws KeeperException, InterruptedException { - if (collection == null) { - throw new IllegalArgumentException("collection must not be null"); - } - String configName = null; - - // first check for alias - collection = checkForAlias(zkClient, collection); - - String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection; - if (LOG.isInfoEnabled()) { - LOG.info("Load collection config from:" + path); - } - byte[] data = zkClient.getData(path, null, null, true); - - if(data != null) { - ZkNodeProps props = ZkNodeProps.load(data); - configName = props.getStr(ZkController.CONFIGNAME_PROP); - } - - if (configName != null && !zkClient.exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true)) { - LOG.error("Specified config does not exist in ZooKeeper:" + configName); - throw new IllegalArgumentException("Specified config does not exist in ZooKeeper:" - + configName); - } - - return configName; - } - - private String checkForAlias(SolrZkClient zkClient, String collection) - throws KeeperException, InterruptedException { - byte[] aliasData = zkClient.getData(ZkStateReader.ALIASES, null, null, true); - Aliases aliases = ClusterState.load(aliasData); - String alias = aliases.getCollectionAlias(collection); - if (alias != null) { - List aliasList = StrUtils.splitSmart(alias, ",", true); - if (aliasList.size() > 1) { - throw new IllegalArgumentException("collection cannot be an alias that maps to multiple collections"); - } - collection = aliasList.get(0); - } - return collection; - } - - /** - * Download and return the config directory from ZK - */ - public File downloadConfigDir(SolrZkClient zkClient, String configName) - throws IOException, InterruptedException, KeeperException { - File dir = Files.createTempDir(); - dir.deleteOnExit(); - ZkConfigManager configManager = new ZkConfigManager(zkClient); - configManager.downloadConfigDir(configName, dir.toPath()); - File confDir = new File(dir, "conf"); - if (!confDir.isDirectory()) { - // create a temporary directory with "conf" subdir and mv the config in there. This is - // necessary because of CDH-11188; solrctl does not generate nor accept directories with e.g. - // conf/solrconfig.xml which is necessary for proper solr operation. This should work - // even if solrctl changes. - confDir = new File(Files.createTempDir().getAbsolutePath(), "conf"); - confDir.getParentFile().deleteOnExit(); - Files.move(dir, confDir); - dir = confDir.getParentFile(); - } - FileUtils.writeStringToFile(new File(dir, "solr.xml"), "", "UTF-8"); - verifyConfigDir(confDir); - return dir; - } - - private void verifyConfigDir(File confDir) throws IOException { - File solrConfigFile = new File(confDir, "solrconfig.xml"); - if (!solrConfigFile.exists()) { - throw new IOException("Detected invalid Solr config dir in ZooKeeper - Reason: File not found: " - + solrConfigFile.getName()); - } - if (!solrConfigFile.isFile()) { - throw new IOException("Detected invalid Solr config dir in ZooKeeper - Reason: Not a file: " - + solrConfigFile.getName()); - } - if (!solrConfigFile.canRead()) { - throw new IOException("Insufficient permissions to read file: " + solrConfigFile); - } - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/NoChangeUpdateConflictResolver.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/NoChangeUpdateConflictResolver.java deleted file mode 100644 index 0eae9405717..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/NoChangeUpdateConflictResolver.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.hadoop.dedup; - -import java.util.Iterator; - -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Reducer.Context; -import org.apache.solr.common.SolrInputDocument; - -/** - * UpdateConflictResolver implementation that returns the solr documents in the - * same order as they are received on input, i.e. without change in order. - */ -public final class NoChangeUpdateConflictResolver implements UpdateConflictResolver { - - @Override - public Iterator orderUpdates(Text key, Iterator updates, Context ctx) { - return updates; - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/RejectingUpdateConflictResolver.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/RejectingUpdateConflictResolver.java deleted file mode 100644 index 60efb4c15bb..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/RejectingUpdateConflictResolver.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.hadoop.dedup; - -import java.util.Collections; -import java.util.Iterator; - -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Reducer.Context; -import org.apache.solr.common.SolrInputDocument; - -/** - * UpdateConflictResolver implementation that rejects multiple documents with - * the same key with an exception. - */ -public final class RejectingUpdateConflictResolver implements UpdateConflictResolver { - - @Override - public Iterator orderUpdates(Text key, Iterator updates, Context ctx) { - SolrInputDocument firstUpdate = null; - while (updates.hasNext()) { - if (firstUpdate == null) { - firstUpdate = updates.next(); - assert firstUpdate != null; - } else { - throw new IllegalArgumentException("Update conflict! Documents with the same unique key are forbidden: " - + key); - } - } - assert firstUpdate != null; - return Collections.singletonList(firstUpdate).iterator(); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/RetainMostRecentUpdateConflictResolver.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/RetainMostRecentUpdateConflictResolver.java deleted file mode 100644 index c2637ce2c39..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/RetainMostRecentUpdateConflictResolver.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.hadoop.dedup; - -import java.lang.invoke.MethodHandles; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; - -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Reducer.Context; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.hadoop.HdfsFileFieldNames; -import org.apache.solr.hadoop.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * UpdateConflictResolver implementation that ignores all but the most recent - * document version, based on a configurable numeric Solr field, which defaults - * to the file_last_modified timestamp. - */ -public class RetainMostRecentUpdateConflictResolver implements UpdateConflictResolver, Configurable { - - private Configuration conf; - private String orderByFieldName = ORDER_BY_FIELD_NAME_DEFAULT; - - public static final String ORDER_BY_FIELD_NAME_KEY = - RetainMostRecentUpdateConflictResolver.class.getName() + ".orderByFieldName"; - - public static final String ORDER_BY_FIELD_NAME_DEFAULT = HdfsFileFieldNames.FILE_LAST_MODIFIED; - - public static final String COUNTER_GROUP = Utils.getShortClassName(RetainMostRecentUpdateConflictResolver.class); - public static final String DUPLICATES_COUNTER_NAME = "Number of documents ignored as duplicates"; - public static final String OUTDATED_COUNTER_NAME = "Number of documents ignored as outdated"; - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - @Override - public void setConf(Configuration conf) { - this.conf = conf; - this.orderByFieldName = conf.get(ORDER_BY_FIELD_NAME_KEY, orderByFieldName); - } - - @Override - public Configuration getConf() { - return conf; - } - - protected String getOrderByFieldName() { - return orderByFieldName; - } - - @Override - public Iterator orderUpdates(Text key, Iterator updates, Context ctx) { - return getMaximum(updates, getOrderByFieldName(), new SolrInputDocumentComparator.TimeStampComparator(), ctx); - } - - /** Returns the most recent document among the colliding updates */ - protected Iterator getMaximum(Iterator updates, String fieldName, - Comparator child, Context context) { - - SolrInputDocumentComparator comp = new SolrInputDocumentComparator(fieldName, child); - SolrInputDocument max = null; - long numDupes = 0; - long numOutdated = 0; - while (updates.hasNext()) { - SolrInputDocument next = updates.next(); - assert next != null; - if (max == null) { - max = next; - } else { - int c = comp.compare(next, max); - if (c == 0) { - LOG.debug("Ignoring document version because it is a duplicate: {}", next); - numDupes++; - } else if (c > 0) { - LOG.debug("Ignoring document version because it is outdated: {}", max); - max = next; - numOutdated++; - } else { - LOG.debug("Ignoring document version because it is outdated: {}", next); - numOutdated++; - } - } - } - - assert max != null; - if (numDupes > 0) { - context.getCounter(COUNTER_GROUP, DUPLICATES_COUNTER_NAME).increment(numDupes); - } - if (numOutdated > 0) { - context.getCounter(COUNTER_GROUP, OUTDATED_COUNTER_NAME).increment(numOutdated); - } - return Collections.singletonList(max).iterator(); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/SolrInputDocumentComparator.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/SolrInputDocumentComparator.java deleted file mode 100644 index e8cfdbb52e4..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/SolrInputDocumentComparator.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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.hadoop.dedup; - -import java.util.Comparator; - -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.common.SolrInputField; - -/** - * Default mechanism of determining which of two Solr documents with the same - * key is the more recent version. - */ -public final class SolrInputDocumentComparator implements Comparator { - - private Comparator child; - private String fieldName; - - SolrInputDocumentComparator(String fieldName, Comparator child) { - this.child = child; - this.fieldName = fieldName; - } - - @Override - public int compare(SolrInputDocument doc1, SolrInputDocument doc2) { - SolrInputField f1 = doc1.getField(fieldName); - SolrInputField f2 = doc2.getField(fieldName); - if (f1 == f2) { - return 0; - } else if (f1 == null) { - return -1; - } else if (f2 == null) { - return 1; - } - - Object v1 = f1.getFirstValue(); - Object v2 = f2.getFirstValue(); - return child.compare(v1, v2); - } - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - public static final class TimeStampComparator implements Comparator { - - @Override - public int compare(Object v1, Object v2) { - if (v1 == v2) { - return 0; - } else if (v1 == null) { - return -1; - } else if (v2 == null) { - return 1; - } - long t1 = getLong(v1); - long t2 = getLong(v2); - return (t1 < t2 ? -1 : (t1==t2 ? 0 : 1)); - } - - private long getLong(Object v) { - if (v instanceof Long) { - return ((Long) v).longValue(); - } else { - return Long.parseLong(v.toString()); - } - } - - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/SortingUpdateConflictResolver.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/SortingUpdateConflictResolver.java deleted file mode 100644 index 24ea9363801..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/SortingUpdateConflictResolver.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.hadoop.dedup; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; - -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Reducer.Context; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.hadoop.HdfsFileFieldNames; - -/** - * UpdateConflictResolver implementation that orders colliding updates ascending - * from least recent to most recent (partial) update, based on a configurable - * numeric Solr field, which defaults to the file_last_modified timestamp. - */ -public class SortingUpdateConflictResolver implements UpdateConflictResolver, Configurable { - - private Configuration conf; - private String orderByFieldName = ORDER_BY_FIELD_NAME_DEFAULT; - - public static final String ORDER_BY_FIELD_NAME_KEY = - SortingUpdateConflictResolver.class.getName() + ".orderByFieldName"; - - public static final String ORDER_BY_FIELD_NAME_DEFAULT = HdfsFileFieldNames.FILE_LAST_MODIFIED; - - @Override - public void setConf(Configuration conf) { - this.conf = conf; - this.orderByFieldName = conf.get(ORDER_BY_FIELD_NAME_KEY, orderByFieldName); - } - - @Override - public Configuration getConf() { - return conf; - } - - protected String getOrderByFieldName() { - return orderByFieldName; - } - - @Override - public Iterator orderUpdates(Text key, Iterator updates, Context ctx) { - return sort(updates, getOrderByFieldName(), new SolrInputDocumentComparator.TimeStampComparator()); - } - - protected Iterator sort(Iterator updates, String fieldName, Comparator child) { - // TODO: use an external merge sort in the pathological case where there are a huge amount of collisions - List sortedUpdates = new ArrayList(1); - while (updates.hasNext()) { - sortedUpdates.add(updates.next()); - } - if (sortedUpdates.size() > 1) { // conflicts are rare - Collections.sort(sortedUpdates, new SolrInputDocumentComparator(fieldName, child)); - } - return sortedUpdates.iterator(); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/UpdateConflictResolver.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/UpdateConflictResolver.java deleted file mode 100644 index 94e23e134eb..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/UpdateConflictResolver.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.hadoop.dedup; - -import java.util.Iterator; - -import org.apache.hadoop.conf.Configurable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Reducer; -import org.apache.hadoop.mapreduce.Reducer.Context; -import org.apache.solr.common.SolrInputDocument; - -/** - * Interface that enables deduplication and ordering of a series of document - * updates for the same unique document key. - * - * For example, a MapReduce batch job might index multiple files in the same job - * where some of the files contain old and new versions of the very same - * document, using the same unique document key. - * - * Typically, implementations of this interface forbid collisions by throwing an - * exception, or ignore all but the most recent document version, or, in the - * general case, order colliding updates ascending from least recent to most - * recent (partial) update. - * - * The caller of this interface (i.e. the Hadoop Reducer) will then apply the - * updates to Solr in the order returned by the orderUpdates() method. - * - * Configuration: If an UpdateConflictResolver implementation also implements - * {@link Configurable} then the Hadoop Reducer will call - * {@link Configurable#setConf(org.apache.hadoop.conf.Configuration)} on - * instance construction and pass the standard Hadoop configuration information. - */ -public interface UpdateConflictResolver { - - /** - * Given a list of all colliding document updates for the same unique document - * key, this method returns zero or more documents in an application specific - * order. - * - * The caller will then apply the updates for this key to Solr in the order - * returned by the orderUpdate() method. - * - * @param uniqueKey - * the document key common to all collidingUpdates mentioned below - * @param collidingUpdates - * all updates in the MapReduce job that have a key equal to - * {@code uniqueKey} mentioned above. The input order is unspecified. - * @param context - * The Context passed from the {@link Reducer} - * implementations. - * @return the order in which the updates shall be applied to Solr - */ - Iterator orderUpdates( - Text uniqueKey, Iterator collidingUpdates, Context context); - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/package-info.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/package-info.java deleted file mode 100644 index a021282654a..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/dedup/package-info.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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. - */ - -/** - * Dedupe related code. - */ -package org.apache.solr.hadoop.dedup; - - - - diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineCounters.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineCounters.java deleted file mode 100644 index 5ba98ff3968..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineCounters.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.hadoop.morphline; - -import org.apache.solr.hadoop.Utils; - -public enum MorphlineCounters { - - FILES_READ (getClassName(MorphlineMapper.class) + ": Number of files read"), - - FILE_BYTES_READ (getClassName(MorphlineMapper.class) + ": Number of file bytes read"), - - DOCS_READ (getClassName(MorphlineMapper.class) + ": Number of documents read"), - - PARSER_OUTPUT_BYTES (getClassName(MorphlineMapper.class) + ": Number of document bytes generated by Tika parser"), - - ERRORS (getClassName(MorphlineMapper.class) + ": Number of errors"); - - private final String label; - - private MorphlineCounters(String label) { - this.label = label; - } - - public String toString() { - return label; - } - - private static String getClassName(Class clazz) { - return Utils.getShortClassName(clazz); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineMapRunner.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineMapRunner.java deleted file mode 100644 index 372c48ba26d..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineMapRunner.java +++ /dev/null @@ -1,268 +0,0 @@ -/* - * 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.hadoop.morphline; - -import java.io.BufferedInputStream; -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.lang.invoke.MethodHandles; -import java.util.HashMap; -import java.util.Map; -import java.util.TreeMap; -import java.util.stream.Collectors; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.mapreduce.Mapper.Context; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.hadoop.HdfsFileFieldNames; -import org.apache.solr.hadoop.PathParts; -import org.apache.solr.hadoop.Utils; -import org.apache.solr.morphlines.solr.DocumentLoader; -import org.apache.solr.morphlines.solr.SolrLocator; -import org.apache.solr.morphlines.solr.SolrMorphlineContext; -import org.apache.solr.schema.IndexSchema; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.kitesdk.morphline.api.Command; -import org.kitesdk.morphline.api.MorphlineCompilationException; -import org.kitesdk.morphline.api.MorphlineContext; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.Compiler; -import org.kitesdk.morphline.base.FaultTolerance; -import org.kitesdk.morphline.base.Fields; -import org.kitesdk.morphline.base.Metrics; -import org.kitesdk.morphline.base.Notifications; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.Timer; -import com.google.common.annotations.Beta; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; - -/** - * Internal helper for {@link MorphlineMapper} and dryRun mode; This API is for *INTERNAL* use only - * and should not be considered public. - */ -@Beta -public final class MorphlineMapRunner { - - private MorphlineContext morphlineContext; - private Command morphline; - private IndexSchema schema; - private Map commandLineMorphlineHeaders; - private boolean disableFileOpen; - private String morphlineFileAndId; - private final Timer elapsedTime; - - public static final String MORPHLINE_FILE_PARAM = "morphlineFile"; - public static final String MORPHLINE_ID_PARAM = "morphlineId"; - - /** - * Morphline variables can be passed from the CLI to the Morphline, e.g.: - * hadoop ... -D morphlineVariable.zkHost=127.0.0.1:2181/solr - */ - public static final String MORPHLINE_VARIABLE_PARAM = "morphlineVariable"; - - /** - * Headers, including MIME types, can also explicitly be passed by force from the CLI to Morphline, e.g: - * hadoop ... -D morphlineField._attachment_mimetype=text/csv - */ - public static final String MORPHLINE_FIELD_PREFIX = "morphlineField."; - - /** - * Flag to disable reading of file contents if indexing just file metadata is sufficient. - * This improves performance and confidentiality. - */ - public static final String DISABLE_FILE_OPEN = "morphlineDisableFileOpen"; - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - MorphlineContext getMorphlineContext() { - return morphlineContext; - } - - IndexSchema getSchema() { - return schema; - } - - public MorphlineMapRunner(Configuration configuration, DocumentLoader loader, String solrHomeDir) throws IOException { - if (LOG.isTraceEnabled()) { - LOG.trace("CWD is {}", new File(".").getCanonicalPath()); - TreeMap map = new TreeMap(); - for (Map.Entry entry : configuration) { - map.put(entry.getKey(), entry.getValue()); - } - LOG.trace("Configuration:\n" + - map.entrySet().stream().map(Object::toString).collect(Collectors.joining("\n"))); - } - - FaultTolerance faultTolerance = new FaultTolerance( - configuration.getBoolean(FaultTolerance.IS_PRODUCTION_MODE, false), - configuration.getBoolean(FaultTolerance.IS_IGNORING_RECOVERABLE_EXCEPTIONS, false), - configuration.get(FaultTolerance.RECOVERABLE_EXCEPTION_CLASSES, SolrServerException.class.getName()) - ); - - morphlineContext = new SolrMorphlineContext.Builder() - .setDocumentLoader(loader) - .setExceptionHandler(faultTolerance) - .setMetricRegistry(new MetricRegistry()) - .build(); - - class MySolrLocator extends SolrLocator { // trick to access protected ctor - public MySolrLocator(MorphlineContext ctx) { - super(ctx); - } - } - - SolrLocator locator = new MySolrLocator(morphlineContext); - locator.setSolrHomeDir(solrHomeDir); - schema = locator.getIndexSchema(); - - // rebuild context, now with schema - morphlineContext = new SolrMorphlineContext.Builder() - .setIndexSchema(schema) - .setDocumentLoader(loader) - .setExceptionHandler(faultTolerance) - .setMetricRegistry(morphlineContext.getMetricRegistry()) - .build(); - - String morphlineFile = configuration.get(MORPHLINE_FILE_PARAM); - String morphlineId = configuration.get(MORPHLINE_ID_PARAM); - if (morphlineFile == null || morphlineFile.trim().length() == 0) { - throw new MorphlineCompilationException("Missing parameter: " + MORPHLINE_FILE_PARAM, null); - } - Map morphlineVariables = new HashMap(); - for (Map.Entry entry : configuration) { - String variablePrefix = MORPHLINE_VARIABLE_PARAM + "."; - if (entry.getKey().startsWith(variablePrefix)) { - morphlineVariables.put(entry.getKey().substring(variablePrefix.length()), entry.getValue()); - } - } - Config override = ConfigFactory.parseMap(morphlineVariables); - morphline = new Compiler().compile(new File(morphlineFile), morphlineId, morphlineContext, null, override); - morphlineFileAndId = morphlineFile + "@" + morphlineId; - - disableFileOpen = configuration.getBoolean(DISABLE_FILE_OPEN, false); - LOG.debug("disableFileOpen: {}", disableFileOpen); - - commandLineMorphlineHeaders = new HashMap(); - for (Map.Entry entry : configuration) { - if (entry.getKey().startsWith(MORPHLINE_FIELD_PREFIX)) { - commandLineMorphlineHeaders.put(entry.getKey().substring(MORPHLINE_FIELD_PREFIX.length()), entry.getValue()); - } - } - LOG.debug("Headers, including MIME types, passed by force from the CLI to morphline: {}", commandLineMorphlineHeaders); - - String metricName = MetricRegistry.name(Utils.getShortClassName(getClass()), Metrics.ELAPSED_TIME); - this.elapsedTime = morphlineContext.getMetricRegistry().timer(metricName); - Notifications.notifyBeginTransaction(morphline); - } - - /** - * Extract content from the path specified in the value. Key is useless. - */ - public void map(String value, Configuration configuration, Context context) throws IOException { - LOG.info("Processing file {}", value); - InputStream in = null; - Record record = null; - Timer.Context timerContext = elapsedTime.time(); - try { - PathParts parts = new PathParts(value.toString(), configuration); - record = getRecord(parts); - if (record == null) { - return; // ignore - } - for (Map.Entry entry : commandLineMorphlineHeaders.entrySet()) { - record.replaceValues(entry.getKey(), entry.getValue()); - } - long fileLength = parts.getFileStatus().getLen(); - if (disableFileOpen) { - in = new ByteArrayInputStream(new byte[0]); - } else { - in = new BufferedInputStream(parts.getFileSystem().open(parts.getUploadPath())); - } - record.put(Fields.ATTACHMENT_BODY, in); - Notifications.notifyStartSession(morphline); - if (!morphline.process(record)) { - LOG.warn("Morphline {} failed to process record: {}", morphlineFileAndId, record); - } - if (context != null) { - context.getCounter(MorphlineCounters.class.getName(), MorphlineCounters.FILES_READ.toString()).increment(1); - context.getCounter(MorphlineCounters.class.getName(), MorphlineCounters.FILE_BYTES_READ.toString()).increment(fileLength); - } - } catch (Exception e) { - LOG.error("Unable to process file " + value, e); - if (context != null) { - context.getCounter(getClass().getName() + ".errors", e.getClass().getName()).increment(1); - } - morphlineContext.getExceptionHandler().handleException(e, record); - } finally { - timerContext.stop(); - if (in != null) { - in.close(); - } - } - } - - protected Record getRecord(PathParts parts) { - FileStatus stats; - try { - stats = parts.getFileStatus(); - } catch (IOException e) { - stats = null; - } - if (stats == null) { - LOG.warn("Ignoring file that somehow has become unavailable since the job was submitted: {}", - parts.getUploadURL()); - return null; - } - - Record headers = new Record(); - //headers.put(getSchema().getUniqueKeyField().getName(), parts.getId()); // use HDFS file path as docId if no docId is specified - headers.put(Fields.BASE_ID, parts.getId()); // with sanitizeUniqueKey command, use HDFS file path as docId if no docId is specified - headers.put(Fields.ATTACHMENT_NAME, parts.getName()); // Tika can use the file name in guessing the right MIME type - - // enable indexing and storing of file meta data in Solr - headers.put(HdfsFileFieldNames.FILE_UPLOAD_URL, parts.getUploadURL()); - headers.put(HdfsFileFieldNames.FILE_DOWNLOAD_URL, parts.getDownloadURL()); - headers.put(HdfsFileFieldNames.FILE_SCHEME, parts.getScheme()); - headers.put(HdfsFileFieldNames.FILE_HOST, parts.getHost()); - headers.put(HdfsFileFieldNames.FILE_PORT, String.valueOf(parts.getPort())); - headers.put(HdfsFileFieldNames.FILE_PATH, parts.getURIPath()); - headers.put(HdfsFileFieldNames.FILE_NAME, parts.getName()); - headers.put(HdfsFileFieldNames.FILE_LAST_MODIFIED, String.valueOf(stats.getModificationTime())); // FIXME also add in SpoolDirectorySource - headers.put(HdfsFileFieldNames.FILE_LENGTH, String.valueOf(stats.getLen())); // FIXME also add in SpoolDirectorySource - headers.put(HdfsFileFieldNames.FILE_OWNER, stats.getOwner()); - headers.put(HdfsFileFieldNames.FILE_GROUP, stats.getGroup()); - headers.put(HdfsFileFieldNames.FILE_PERMISSIONS_USER, stats.getPermission().getUserAction().SYMBOL); - headers.put(HdfsFileFieldNames.FILE_PERMISSIONS_GROUP, stats.getPermission().getGroupAction().SYMBOL); - headers.put(HdfsFileFieldNames.FILE_PERMISSIONS_OTHER, stats.getPermission().getOtherAction().SYMBOL); - headers.put(HdfsFileFieldNames.FILE_PERMISSIONS_STICKYBIT, String.valueOf(stats.getPermission().getStickyBit())); - // TODO: consider to add stats.getAccessTime(), stats.getReplication(), stats.isSymlink(), stats.getBlockSize() - - return headers; - } - - public void cleanup() { - Notifications.notifyCommitTransaction(morphline); - Notifications.notifyShutdown(morphline); - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineMapper.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineMapper.java deleted file mode 100644 index 8ba2ea6483c..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/MorphlineMapper.java +++ /dev/null @@ -1,193 +0,0 @@ -/* - * 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.hadoop.morphline; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.Collection; -import java.util.Map; - -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.response.SolrPingResponse; -import org.apache.solr.client.solrj.response.UpdateResponse; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.common.SolrInputField; -import org.apache.solr.hadoop.HeartBeater; -import org.apache.solr.hadoop.SolrInputDocumentWritable; -import org.apache.solr.hadoop.SolrMapper; -import org.apache.solr.morphlines.solr.DocumentLoader; -import org.apache.solr.schema.IndexSchema; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.codahale.metrics.Counter; -import com.codahale.metrics.Counting; -import com.codahale.metrics.Histogram; -import com.codahale.metrics.Meter; -import com.codahale.metrics.MetricRegistry; -import com.codahale.metrics.Timer; - -/** - * This class takes the input files, extracts the relevant content, transforms - * it and hands SolrInputDocuments to a set of reducers. - * - * More specifically, it consumes a list of <offset, hdfsFilePath> input pairs. - * For each such pair extracts a set of zero or more SolrInputDocuments and - * sends them to a downstream Reducer. The key for the reducer is the unique id - * of the SolrInputDocument specified in Solr schema.xml. - */ -public class MorphlineMapper extends SolrMapper { - - private Context context; - private MorphlineMapRunner runner; - private HeartBeater heartBeater; - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - protected IndexSchema getSchema() { - return runner.getSchema(); - } - - protected Context getContext() { - return context; - } - - @Override - protected void setup(Context context) throws IOException, InterruptedException { - super.setup(context); - this.context = context; - heartBeater = new HeartBeater(context); - this.runner = new MorphlineMapRunner( - context.getConfiguration(), new MyDocumentLoader(), getSolrHomeDir().toString()); - } - - /** - * Extract content from the path specified in the value. Key is useless. - */ - @Override - public void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException { - heartBeater.needHeartBeat(); - try { - runner.map(value.toString(), context.getConfiguration(), context); - } finally { - heartBeater.cancelHeartBeat(); - } - } - - @Override - protected void cleanup(Context context) throws IOException, InterruptedException { - heartBeater.close(); - runner.cleanup(); - addMetricsToMRCounters(runner.getMorphlineContext().getMetricRegistry(), context); - super.cleanup(context); - } - - private void addMetricsToMRCounters(MetricRegistry metricRegistry, Context context) { - for (Map.Entry entry : metricRegistry.getCounters().entrySet()) { - addCounting(entry.getKey(), entry.getValue(), 1); - } - for (Map.Entry entry : metricRegistry.getHistograms().entrySet()) { - addCounting(entry.getKey(), entry.getValue(), 1); - } - for (Map.Entry entry : metricRegistry.getMeters().entrySet()) { - addCounting(entry.getKey(), entry.getValue(), 1); - } - for (Map.Entry entry : metricRegistry.getTimers().entrySet()) { - long nanosPerMilliSec = 1000 * 1000; - addCounting(entry.getKey(), entry.getValue(), nanosPerMilliSec); - } - } - - private void addCounting(String metricName, Counting value, long scale) { - context.getCounter("morphline", metricName).increment(value.getCount() / scale); - } - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private final class MyDocumentLoader implements DocumentLoader { - - @Override - public void beginTransaction() { - } - - @Override - public void load(SolrInputDocument doc) throws IOException, SolrServerException { - String uniqueKeyFieldName = getSchema().getUniqueKeyField().getName(); - Object id = doc.getFieldValue(uniqueKeyFieldName); - if (id == null) { - throw new IllegalArgumentException("Missing value for (required) unique document key: " + uniqueKeyFieldName - + " (see Solr schema.xml)"); - } - try { - context.write(new Text(id.toString()), new SolrInputDocumentWritable(doc)); - } catch (InterruptedException e) { - throw new IOException("Interrupted while writing " + doc, e); - } - - if (LOG.isDebugEnabled()) { - long numParserOutputBytes = 0; - for (SolrInputField field : doc.values()) { - numParserOutputBytes += sizeOf(field.getValue()); - } - context.getCounter(MorphlineCounters.class.getName(), MorphlineCounters.PARSER_OUTPUT_BYTES.toString()).increment(numParserOutputBytes); - } - context.getCounter(MorphlineCounters.class.getName(), MorphlineCounters.DOCS_READ.toString()).increment(1); - } - - // just an approximation - private long sizeOf(Object value) { - if (value instanceof CharSequence) { - return ((CharSequence) value).length(); - } else if (value instanceof Integer) { - return 4; - } else if (value instanceof Long) { - return 8; - } else if (value instanceof Collection) { - long size = 0; - for (Object val : (Collection) value) { - size += sizeOf(val); - } - return size; - } else { - return String.valueOf(value).length(); - } - } - - @Override - public void commitTransaction() { - } - - @Override - public UpdateResponse rollbackTransaction() throws SolrServerException, IOException { - return new UpdateResponse(); - } - - @Override - public void shutdown() { - } - - @Override - public SolrPingResponse ping() throws SolrServerException, IOException { - return new SolrPingResponse(); - } - - } - -} diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/package-info.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/package-info.java deleted file mode 100644 index c754f79e338..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/morphline/package-info.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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. - */ - -/** - * Morphlines related code. - */ -package org.apache.solr.hadoop.morphline; - - - - diff --git a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/package-info.java b/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/package-info.java deleted file mode 100644 index 043a150e935..00000000000 --- a/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/package-info.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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. - */ - -/** - * {@link org.apache.solr.hadoop.MapReduceIndexerTool} and related code. - */ -package org.apache.solr.hadoop; - - - - diff --git a/solr/contrib/map-reduce/src/java/overview.html b/solr/contrib/map-reduce/src/java/overview.html deleted file mode 100644 index ad7c1c0c3fe..00000000000 --- a/solr/contrib/map-reduce/src/java/overview.html +++ /dev/null @@ -1,21 +0,0 @@ - - - -Apache Solr Search Server: Solr MapReduce contrib - - diff --git a/solr/contrib/map-reduce/src/test-files/README.txt b/solr/contrib/map-reduce/src/test-files/README.txt deleted file mode 100644 index 8905df29d30..00000000000 --- a/solr/contrib/map-reduce/src/test-files/README.txt +++ /dev/null @@ -1 +0,0 @@ -The test-files by this module are located in the morphlines-core module. diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/AlphaNumericComparatorTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/AlphaNumericComparatorTest.java deleted file mode 100644 index cab29e7796c..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/AlphaNumericComparatorTest.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.hadoop; - -import java.util.Comparator; - -import org.junit.Assert; -import org.junit.Test; - -public class AlphaNumericComparatorTest extends Assert { - - @Test - public void testBasic() { - Comparator c = new AlphaNumericComparator(); - assertTrue(c.compare("a", "b") < 0); - assertTrue(c.compare("shard1", "shard1") == 0); - //assertTrue(c.compare("shard01", "shard1") == 0); - assertTrue(c.compare("shard10", "shard10") == 0); - assertTrue(c.compare("shard1", "shard2") < 0); - assertTrue(c.compare("shard9", "shard10") < 0); - assertTrue(c.compare("shard09", "shard10") < 0); - assertTrue(c.compare("shard019", "shard10") > 0); - assertTrue(c.compare("shard10", "shard11") < 0); - assertTrue(c.compare("shard10z", "shard10z") == 0); - assertTrue(c.compare("shard10z", "shard11z") < 0); - assertTrue(c.compare("shard10a", "shard10z") < 0); - assertTrue(c.compare("shard10z", "shard10a") > 0); - assertTrue(c.compare("shard1z", "shard1z") == 0); - assertTrue(c.compare("shard2", "shard1") > 0); - } - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/IdentityMapper.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/IdentityMapper.java deleted file mode 100644 index 9ba70d59975..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/IdentityMapper.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; - -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Mapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class IdentityMapper extends Mapper { - - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - @Override - protected void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException { - LOGGER.info("map key: {}, value: {}", key, value); - context.write(value, NullWritable.get()); - } -} \ No newline at end of file diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/IdentityReducer.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/IdentityReducer.java deleted file mode 100644 index 1c248c64477..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/IdentityReducer.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; - -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Reducer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class IdentityReducer extends Reducer { - - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - @Override - protected void reduce(Text key, Iterable values, Context context) throws IOException, InterruptedException { - LOGGER.info("reduce key: {}, value: {}", key, values); - context.write(key, NullWritable.get()); - } -} \ No newline at end of file diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/LineRandomizerMapperReducerTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/LineRandomizerMapperReducerTest.java deleted file mode 100644 index 2354fdd6633..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/LineRandomizerMapperReducerTest.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * 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.hadoop; - -import java.io.IOException; -import java.util.Arrays; - -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mrunit.mapreduce.MapReduceDriver; -import org.apache.hadoop.mrunit.types.Pair; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class LineRandomizerMapperReducerTest extends Assert { - - private MapReduceDriver mapReduceDriver; - - @Before - public void setUp() { - LineRandomizerMapper mapper = new LineRandomizerMapper(); - LineRandomizerReducer reducer = new LineRandomizerReducer(); - mapReduceDriver = MapReduceDriver.newMapReduceDriver(mapper, reducer); - } - - @Test - public void testMapReduce1Item() throws IOException { - mapReduceDriver.withInput(new LongWritable(0), new Text("hello")); - mapReduceDriver.withOutput(new Text("hello"), NullWritable.get()); - mapReduceDriver.runTest(); - } - - @Test - public void testMapReduce2Items() throws IOException { - mapReduceDriver.withAll(Arrays.asList( - new Pair<>(new LongWritable(0), new Text("hello")), - new Pair<>(new LongWritable(1), new Text("world")) - )); - mapReduceDriver.withAllOutput(Arrays.asList( - new Pair<>(new Text("world"), NullWritable.get()), - new Pair<>(new Text("hello"), NullWritable.get()) - )); - mapReduceDriver.runTest(); - } - - @Test - public void testMapReduce3Items() throws IOException { - mapReduceDriver.withAll(Arrays.asList( - new Pair<>(new LongWritable(0), new Text("hello")), - new Pair<>(new LongWritable(1), new Text("world")), - new Pair<>(new LongWritable(2), new Text("nadja")) - )); - mapReduceDriver.withAllOutput(Arrays.asList( - new Pair<>(new Text("nadja"), NullWritable.get()), - new Pair<>(new Text("world"), NullWritable.get()), - new Pair<>(new Text("hello"), NullWritable.get()) - )); - mapReduceDriver.runTest(); - } - - @Test - public void testMapReduce4Items() throws IOException { - mapReduceDriver.withAll(Arrays.asList( - new Pair<>(new LongWritable(0), new Text("hello")), - new Pair<>(new LongWritable(1), new Text("world")), - new Pair<>(new LongWritable(2), new Text("nadja")), - new Pair<>(new LongWritable(3), new Text("basti")) - )); - mapReduceDriver.withAllOutput(Arrays.asList( - new Pair<>(new Text("nadja"), NullWritable.get()), - new Pair<>(new Text("world"), NullWritable.get()), - new Pair<>(new Text("basti"), NullWritable.get()), - new Pair<>(new Text("hello"), NullWritable.get()) - )); - mapReduceDriver.runTest(); - } - -} \ No newline at end of file diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java deleted file mode 100644 index 558d66255f2..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.hadoop; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.util.Locale; - -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.hadoop.morphline.MorphlineMapRunner; -import org.apache.solr.morphlines.solr.AbstractSolrMorphlineTestBase; -import org.junit.AfterClass; -import org.junit.BeforeClass; - -public abstract class MRUnitBase extends SolrTestCaseJ4 { - - protected static final String RESOURCES_DIR = getFile("morphlines-core.marker").getParent(); - protected static final String DOCUMENTS_DIR = RESOURCES_DIR + "/test-documents"; - protected static File solrHomeZip; - - @BeforeClass - public static void setupClass() throws Exception { - assumeFalse("This test fails on UNIX with Turkish default locale (https://issues.apache.org/jira/browse/SOLR-6387)", - new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage())); - solrHomeZip = SolrOutputFormat.createSolrHomeZip(new File(RESOURCES_DIR + "/solr/mrunit")); - assertNotNull(solrHomeZip); - } - - @AfterClass - public static void teardownClass() throws Exception { - if (solrHomeZip != null) Files.delete(solrHomeZip.toPath()); - solrHomeZip = null; - } - - protected void setupHadoopConfig(Configuration config) throws IOException { - - String tempDir = createTempDir().toFile().getAbsolutePath(); - - FileUtils.copyFile(new File(RESOURCES_DIR + "/custom-mimetypes.xml"), new File(tempDir + "/custom-mimetypes.xml")); - - AbstractSolrMorphlineTestBase.setupMorphline(tempDir, "test-morphlines/solrCellDocumentTypes", true); - - config.set(MorphlineMapRunner.MORPHLINE_FILE_PARAM, tempDir + "/test-morphlines/solrCellDocumentTypes.conf"); - config.set(SolrOutputFormat.ZIP_NAME, solrHomeZip.getName()); - } - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java deleted file mode 100644 index 1aebcf749b1..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MapReduceIndexerToolArgumentParserTest.java +++ /dev/null @@ -1,468 +0,0 @@ -/* - * 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.hadoop; - -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.PrintStream; -import java.io.UnsupportedEncodingException; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.Collections; -import java.util.Locale; - -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.lucene.util.Constants; -import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.cloud.AbstractZkTestCase; -import org.apache.solr.hadoop.dedup.NoChangeUpdateConflictResolver; -import org.apache.solr.hadoop.dedup.RetainMostRecentUpdateConflictResolver; -import org.junit.After; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -public class MapReduceIndexerToolArgumentParserTest extends SolrTestCaseJ4 { - - private Configuration conf; - private MapReduceIndexerTool.MyArgumentParser parser; - private MapReduceIndexerTool.Options opts; - private PrintStream oldSystemOut; - private PrintStream oldSystemErr; - private ByteArrayOutputStream bout; - private ByteArrayOutputStream berr; - - private static final String RESOURCES_DIR = getFile("morphlines-core.marker").getParent(); - private static final File MINIMR_INSTANCE_DIR = new File(RESOURCES_DIR + "/solr/minimr"); - - private static final String MORPHLINE_FILE = RESOURCES_DIR + "/test-morphlines/solrCellDocumentTypes.conf"; - - private final File solrHomeDirectory = createTempDir().toFile(); - - @BeforeClass - public static void beforeClass() { - assumeFalse("Does not work on Windows, because it uses UNIX shell commands or POSIX paths", Constants.WINDOWS); - assumeFalse("This test fails on UNIX with Turkish default locale (https://issues.apache.org/jira/browse/SOLR-6387)", - new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage())); - } - - @Before - public void setUp() throws Exception { - super.setUp(); - AbstractZkTestCase.SOLRHOME = solrHomeDirectory; - FileUtils.copyDirectory(MINIMR_INSTANCE_DIR, solrHomeDirectory); - - conf = new Configuration(); - parser = new MapReduceIndexerTool.MyArgumentParser(); - opts = new MapReduceIndexerTool.Options(); - oldSystemOut = System.out; - bout = new ByteArrayOutputStream(); - System.setOut(new PrintStream(bout, true, "UTF-8")); - oldSystemErr = System.err; - berr = new ByteArrayOutputStream(); - System.setErr(new PrintStream(berr, true, "UTF-8")); - } - - @After - public void tearDown() throws Exception { - super.tearDown(); - System.setOut(oldSystemOut); - System.setErr(oldSystemErr); - } - - @Test - public void testArgsParserTypicalUse() { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--morphline-id", "morphline_xyz", - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--mappers", "10", - "--reducers", "9", - "--fanout", "8", - "--max-segments", "7", - "--shards", "1", - "--verbose", - "file:///home", - "file:///dev", - }; - Integer res = parser.parseArgs(args, conf, opts); - assertNull(res != null ? res.toString() : "", res); - assertEquals(Collections.singletonList(new Path("file:///tmp")), opts.inputLists); - assertEquals(new Path("file:/tmp/foo"), opts.outputDir); - assertEquals(new File(MINIMR_INSTANCE_DIR.getPath()), opts.solrHomeDir); - assertEquals(10, opts.mappers); - assertEquals(9, opts.reducers); - assertEquals(8, opts.fanout); - assertEquals(7, opts.maxSegments); - assertEquals(new Integer(1), opts.shards); - assertEquals(null, opts.fairSchedulerPool); - assertTrue(opts.isVerbose); - assertEquals(Arrays.asList(new Path("file:///home"), new Path("file:///dev")), opts.inputFiles); - assertEquals(RetainMostRecentUpdateConflictResolver.class.getName(), opts.updateConflictResolver); - assertEquals(MORPHLINE_FILE, opts.morphlineFile.getPath()); - assertEquals("morphline_xyz", opts.morphlineId); - assertEmptySystemErrAndEmptySystemOut(); - } - - @Test - public void testArgsParserMultipleSpecsOfSameKind() { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--input-list", "file:///", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shards", "1", - "file:///home", - "file:///dev", - }; - assertNull(parser.parseArgs(args, conf, opts)); - assertEquals(Arrays.asList(new Path("file:///tmp"), new Path("file:///")), opts.inputLists); - assertEquals(Arrays.asList(new Path("file:///home"), new Path("file:///dev")), opts.inputFiles); - assertEquals(new Path("file:/tmp/foo"), opts.outputDir); - assertEquals(new File(MINIMR_INSTANCE_DIR.getPath()), opts.solrHomeDir); - assertEmptySystemErrAndEmptySystemOut(); - } - - @Test - public void testArgsParserTypicalUseWithEqualsSign() { - String[] args = new String[] { - "--input-list=file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir=file:/tmp/foo", - "--solr-home-dir=" + MINIMR_INSTANCE_DIR.getPath(), - "--mappers=10", - "--shards", "1", - "--verbose", - "file:///home", - "file:///dev", - }; - assertNull(parser.parseArgs(args, conf, opts)); - assertEquals(Collections.singletonList(new Path("file:///tmp")), opts.inputLists); - assertEquals(new Path("file:/tmp/foo"), opts.outputDir); - assertEquals(new File(MINIMR_INSTANCE_DIR.getPath()), opts.solrHomeDir); - assertEquals(10, opts.mappers); - assertEquals(new Integer(1), opts.shards); - assertEquals(null, opts.fairSchedulerPool); - assertTrue(opts.isVerbose); - assertEquals(Arrays.asList(new Path("file:///home"), new Path("file:///dev")), opts.inputFiles); - assertEmptySystemErrAndEmptySystemOut(); - } - - @Test - public void testArgsParserMultipleSpecsOfSameKindWithEqualsSign() { - String[] args = new String[] { - "--input-list=file:///tmp", - "--input-list=file:///", - "--morphline-file", MORPHLINE_FILE, - "--output-dir=file:/tmp/foo", - "--solr-home-dir=" + MINIMR_INSTANCE_DIR.getPath(), - "--shards", "1", - "file:///home", - "file:///dev", - }; - assertNull(parser.parseArgs(args, conf, opts)); - assertEquals(Arrays.asList(new Path("file:///tmp"), new Path("file:///")), opts.inputLists); - assertEquals(Arrays.asList(new Path("file:///home"), new Path("file:///dev")), opts.inputFiles); - assertEquals(new Path("file:/tmp/foo"), opts.outputDir); - assertEquals(new File(MINIMR_INSTANCE_DIR.getPath()), opts.solrHomeDir); - assertEmptySystemErrAndEmptySystemOut(); - } - - @Test - public void testArgsParserHelp() throws UnsupportedEncodingException { - String[] args = new String[] { "--help" }; - assertEquals(new Integer(0), parser.parseArgs(args, conf, opts)); - String helpText = new String(bout.toByteArray(), StandardCharsets.UTF_8); - assertTrue(helpText.contains("MapReduce batch job driver that ")); - assertTrue(helpText.contains("bin/hadoop command")); - assertEquals(0, berr.toByteArray().length); - } - - @Test - public void testArgsParserOk() { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shards", "1", - }; - assertNull(parser.parseArgs(args, conf, opts)); - assertEquals(new Integer(1), opts.shards); - assertEmptySystemErrAndEmptySystemOut(); - } - - @Test - public void testArgsParserUpdateConflictResolver() { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shards", "1", - "--update-conflict-resolver", NoChangeUpdateConflictResolver.class.getName(), - }; - assertNull(parser.parseArgs(args, conf, opts)); - assertEquals(NoChangeUpdateConflictResolver.class.getName(), opts.updateConflictResolver); - assertEmptySystemErrAndEmptySystemOut(); - } - - @Test - public void testArgsParserUnknownArgName() throws Exception { - String[] args = new String[] { - "--xxxxxxxxinputlist", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shards", "1", - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsParserFileNotFound1() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/fileNotFound/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shards", "1", - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsParserFileNotFound2() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", "/fileNotFound", - "--shards", "1", - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsParserIntOutOfRange() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shards", "1", - "--mappers", "-20" - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsParserIllegalFanout() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shards", "1", - "--fanout", "1" // must be >= 2 - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsParserSolrHomeMustContainSolrConfigFile() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--shards", "1", - "--solr-home-dir", "/", - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsShardUrlOk() { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shard-url", "http://localhost:8983/solr/collection1", - "--shard-url", "http://localhost:8983/solr/collection2", - }; - assertNull(parser.parseArgs(args, conf, opts)); - assertEquals(Arrays.asList( - Collections.singletonList("http://localhost:8983/solr/collection1"), - Collections.singletonList("http://localhost:8983/solr/collection2")), - opts.shardUrls); - assertEquals(new Integer(2), opts.shards); - assertEmptySystemErrAndEmptySystemOut(); - } - - @Test - public void testArgsShardUrlMustHaveAParam() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shard-url", - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsShardUrlAndShardsSucceeds() { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shards", "1", - "--shard-url", "http://localhost:8983/solr/collection1", - }; - assertNull(parser.parseArgs(args, conf, opts)); - assertEmptySystemErrAndEmptySystemOut(); - } - - @Test - public void testArgsShardUrlNoGoLive() { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shard-url", "http://localhost:8983/solr/collection1" - }; - assertNull(parser.parseArgs(args, conf, opts)); - assertEmptySystemErrAndEmptySystemOut(); - assertEquals(new Integer(1), opts.shards); - } - - @Test - public void testArgsShardUrlsAndZkhostAreMutuallyExclusive() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shard-url", "http://localhost:8983/solr/collection1", - "--shard-url", "http://localhost:8983/solr/collection1", - "--zk-host", "http://localhost:2185", - "--go-live" - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsGoLiveAndSolrUrl() { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--shard-url", "http://localhost:8983/solr/collection1", - "--shard-url", "http://localhost:8983/solr/collection1", - "--go-live" - }; - Integer result = parser.parseArgs(args, conf, opts); - assertNull(result); - assertEmptySystemErrAndEmptySystemOut(); - } - - @Test - public void testArgsZkHostNoGoLive() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--zk-host", "http://localhost:2185", - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsGoLiveZkHostNoCollection() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--zk-host", "http://localhost:2185", - "--go-live" - }; - assertArgumentParserException(args); - } - - @Test - public void testArgsGoLiveNoZkHostOrSolrUrl() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--solr-home-dir", MINIMR_INSTANCE_DIR.getPath(), - "--go-live" - }; - assertArgumentParserException(args); - } - - @Test - public void testNoSolrHomeDirOrZKHost() throws Exception { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--shards", "1", - }; - assertArgumentParserException(args); - } - - @Test - public void testZKHostNoSolrHomeDirOk() { - String[] args = new String[] { - "--input-list", "file:///tmp", - "--morphline-file", MORPHLINE_FILE, - "--output-dir", "file:/tmp/foo", - "--zk-host", "http://localhost:2185", - "--collection", "collection1", - }; - assertNull(parser.parseArgs(args, conf, opts)); - assertEmptySystemErrAndEmptySystemOut(); - } - - private void assertEmptySystemErrAndEmptySystemOut() { - assertEquals(0, bout.toByteArray().length); - assertEquals(0, berr.toByteArray().length); - } - - private void assertArgumentParserException(String[] args) throws UnsupportedEncodingException { - assertEquals("should have returned fail code", new Integer(1), parser.parseArgs(args, conf, opts)); - assertEquals("no sys out expected:" + new String(bout.toByteArray(), StandardCharsets.UTF_8), 0, bout.toByteArray().length); - String usageText; - usageText = new String(berr.toByteArray(), StandardCharsets.UTF_8); - - assertTrue("should start with usage msg \"usage: hadoop \":" + usageText, usageText.startsWith("usage: hadoop ")); - } - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java deleted file mode 100644 index 6479a20d2e1..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineBasicMiniMRTest.java +++ /dev/null @@ -1,415 +0,0 @@ -/* - * 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.hadoop; - -import java.io.File; -import java.io.IOException; -import java.io.OutputStream; -import java.io.OutputStreamWriter; -import java.io.Writer; -import java.lang.reflect.Array; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; - -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.security.authorize.ProxyUsers; -import org.apache.hadoop.util.JarFinder; -import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.lucene.util.Constants; -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; -import org.apache.lucene.util.LuceneTestCase.Slow; -import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.cloud.AbstractZkTestCase; -import org.apache.solr.hadoop.hack.MiniMRCluster; -import org.apache.solr.morphlines.solr.AbstractSolrMorphlineTestBase; -import org.apache.solr.util.BadHdfsThreadsFilter; -import org.apache.solr.util.BadMrClusterThreadsFilter; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -import com.carrotsearch.randomizedtesting.annotations.Nightly; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction.Action; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Consequence; - -@ThreadLeakAction({Action.WARN}) -@ThreadLeakLingering(linger = 0) -@ThreadLeakZombies(Consequence.CONTINUE) -@ThreadLeakFilters(defaultFilters = true, filters = { - BadHdfsThreadsFilter.class, BadMrClusterThreadsFilter.class // hdfs currently leaks thread(s) -}) -@Slow -@Nightly -@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-9076") -public class MorphlineBasicMiniMRTest extends SolrTestCaseJ4 { - - private static final boolean ENABLE_LOCAL_JOB_RUNNER = false; // for debugging only - private static final String RESOURCES_DIR = getFile("morphlines-core.marker").getParent(); - private static final String DOCUMENTS_DIR = RESOURCES_DIR + "/test-documents"; - private static final File MINIMR_CONF_DIR = new File(RESOURCES_DIR + "/solr/minimr"); - - private static String SEARCH_ARCHIVES_JAR; - - private static MiniDFSCluster dfsCluster = null; - private static MiniMRCluster mrCluster = null; - private static int numRuns = 0; - - private final String inputAvroFile; - private final int count; - - private static String tempDir; - - private static File solrHomeDirectory; - - protected MapReduceIndexerTool createTool() { - return new MapReduceIndexerTool(); - } - - public MorphlineBasicMiniMRTest() { - int data = random().nextInt(3); - switch (data) { - case 0: - this.inputAvroFile = "sample-statuses-20120906-141433.avro"; - this.count = 2; - break; - case 1: - this.inputAvroFile = "sample-statuses-20120521-100919.avro"; - this.count = 20; - break; - case 2: - this.inputAvroFile = "sample-statuses-20120906-141433-medium.avro"; - this.count = 2104; - break; - default: - throw new RuntimeException("Test setup is broken"); - } - - } - - @BeforeClass - public static void setupClass() throws Exception { - solrHomeDirectory = createTempDir().toFile(); - - assumeFalse("HDFS tests were disabled by -Dtests.disableHdfs", - Boolean.parseBoolean(System.getProperty("tests.disableHdfs", "false"))); - - assumeFalse("FIXME: This test does not work with Windows because of native library requirements", Constants.WINDOWS); - - AbstractZkTestCase.SOLRHOME = solrHomeDirectory; - FileUtils.copyDirectory(MINIMR_CONF_DIR, solrHomeDirectory); - File dataDir = createTempDir().toFile(); - tempDir = dataDir.getAbsolutePath(); - new File(tempDir).mkdirs(); - FileUtils.copyFile(new File(RESOURCES_DIR + "/custom-mimetypes.xml"), new File(tempDir + "/custom-mimetypes.xml")); - - AbstractSolrMorphlineTestBase.setupMorphline(tempDir, "test-morphlines/solrCellDocumentTypes", true); - - System.setProperty("hadoop.log.dir", new File(solrHomeDirectory, "logs").getAbsolutePath()); - - int taskTrackers = 1; - int dataNodes = 2; -// String proxyUser = System.getProperty("user.name"); -// String proxyGroup = "g"; -// StringBuilder sb = new StringBuilder(); -// sb.append("127.0.0.1,localhost"); -// for (InetAddress i : InetAddress.getAllByName(InetAddress.getLocalHost().getHostName())) { -// sb.append(",").append(i.getCanonicalHostName()); -// } - - new File(dataDir, "nm-local-dirs").mkdirs(); - - System.setProperty("solr.hdfs.blockcache.enabled", "false"); - - System.setProperty("test.build.dir", dataDir + File.separator + "hdfs" + File.separator + "test-build-dir"); - System.setProperty("test.build.data", dataDir + File.separator + "hdfs" + File.separator + "build"); - System.setProperty("test.cache.data", dataDir + File.separator + "hdfs" + File.separator + "cache"); - - // Initialize AFTER test.build.dir is set, JarFinder uses it. - SEARCH_ARCHIVES_JAR = JarFinder.getJar(MapReduceIndexerTool.class); - - JobConf conf = new JobConf(); - conf.set("dfs.block.access.token.enable", "false"); - conf.set("dfs.permissions", "true"); - conf.set("hadoop.security.authentication", "simple"); - conf.set(YarnConfiguration.NM_LOCAL_DIRS, dataDir.getPath() + File.separator + "nm-local-dirs"); - conf.set(YarnConfiguration.DEFAULT_NM_LOG_DIRS, dataDir + File.separator + "nm-logs"); - conf.set("testWorkDir", dataDir.getPath() + File.separator + "testWorkDir"); - conf.set("mapreduce.jobhistory.minicluster.fixed.ports", "false"); - conf.set("mapreduce.jobhistory.admin.address", "0.0.0.0:0"); - - dfsCluster = new MiniDFSCluster(conf, dataNodes, true, null); - FileSystem fileSystem = dfsCluster.getFileSystem(); - fileSystem.mkdirs(new Path("/tmp")); - fileSystem.mkdirs(new Path("/user")); - fileSystem.mkdirs(new Path("/hadoop/mapred/system")); - fileSystem.setPermission(new Path("/tmp"), FsPermission.valueOf("-rwxrwxrwx")); - fileSystem.setPermission(new Path("/user"), FsPermission.valueOf("-rwxrwxrwx")); - fileSystem.setPermission(new Path("/hadoop/mapred/system"), FsPermission.valueOf("-rwx------")); - String nnURI = fileSystem.getUri().toString(); - int numDirs = 1; - String[] racks = null; - String[] hosts = null; - - mrCluster = new MiniMRCluster(0, 0, taskTrackers, nnURI, numDirs, racks, hosts, null, conf); - ProxyUsers.refreshSuperUserGroupsConfiguration(conf); - } - - @AfterClass - public static void teardownClass() throws Exception { - System.clearProperty("solr.hdfs.blockcache.enabled"); - System.clearProperty("test.build.dir"); - System.clearProperty("test.build.data"); - System.clearProperty("test.cache.data"); - - if (mrCluster != null) { - mrCluster.shutdown(); - mrCluster = null; - } - if (dfsCluster != null) { - dfsCluster.shutdown(); - dfsCluster = null; - } - - FileSystem.closeAll(); - } - - @After - public void tearDown() throws Exception { - System.clearProperty("hadoop.log.dir"); - System.clearProperty("solr.hdfs.blockcache.enabled"); - - super.tearDown(); - } - - private JobConf getJobConf() { - return mrCluster.createJobConf(); - } - - @Test - public void testPathParts() throws Exception { // see PathParts - FileSystem fs = dfsCluster.getFileSystem(); - int dfsClusterPort = fs.getWorkingDirectory().toUri().getPort(); - assertTrue(dfsClusterPort > 0); - JobConf jobConf = getJobConf(); - Configuration simpleConf = new Configuration(); - - for (Configuration conf : Arrays.asList(jobConf, simpleConf)) { - for (String queryAndFragment : Arrays.asList("", "?key=value#fragment")) { - for (String up : Arrays.asList("", "../")) { - String down = up.length() == 0 ? "foo/" : ""; - String uploadURL = "hdfs://localhost:12345/user/foo/" + up + "bar.txt" + queryAndFragment; - PathParts parts = new PathParts(uploadURL, conf); - assertEquals(uploadURL, parts.getUploadURL()); - assertEquals("/user/" + down + "bar.txt", parts.getURIPath()); - assertEquals("bar.txt", parts.getName()); - assertEquals("hdfs", parts.getScheme()); - assertEquals("localhost", parts.getHost()); - assertEquals(12345, parts.getPort()); - assertEquals("hdfs://localhost:12345/user/" + down + "bar.txt", parts.getId()); - assertEquals(parts.getId(), parts.getDownloadURL()); - assertFileNotFound(parts); - - uploadURL = "hdfs://localhost/user/foo/" + up + "bar.txt" + queryAndFragment; - parts = new PathParts(uploadURL, conf); - assertEquals(uploadURL, parts.getUploadURL()); - assertEquals("/user/" + down + "bar.txt", parts.getURIPath()); - assertEquals("bar.txt", parts.getName()); - assertEquals("hdfs", parts.getScheme()); - assertEquals("localhost", parts.getHost()); - assertEquals(8020, parts.getPort()); - assertEquals("hdfs://localhost:8020/user/" + down + "bar.txt", parts.getId()); - assertEquals(parts.getId(), parts.getDownloadURL()); - assertFileNotFound(parts); - } - } - } - - for (Configuration conf : Arrays.asList(jobConf)) { - for (String queryAndFragment : Arrays.asList("", "?key=value#fragment")) { - for (String up : Arrays.asList("", "../")) { - // verify using absolute path - String down = up.length() == 0 ? "foo/" : ""; - String uploadURL = "/user/foo/" + up + "bar.txt" + queryAndFragment; - PathParts parts = new PathParts(uploadURL, conf); - assertEquals(uploadURL, parts.getUploadURL()); - assertEquals("/user/" + down + "bar.txt", parts.getURIPath()); - assertEquals("bar.txt", parts.getName()); - assertEquals("hdfs", parts.getScheme()); - assertTrue("localhost".equals(parts.getHost()) || "localhost.localdomain".equals(parts.getHost())); - assertEquals(dfsClusterPort, parts.getPort()); - assertTrue(parts.getId().equals("hdfs://localhost:" + dfsClusterPort + "/user/" + down + "bar.txt") - || parts.getId().equals("hdfs://localhost.localdomain:" + dfsClusterPort + "/user/" + down + "bar.txt") - ); - assertFileNotFound(parts); - - // verify relative path is interpreted to be relative to user's home dir and resolved to an absolute path - uploadURL = "xuser/foo/" + up + "bar.txt" + queryAndFragment; - parts = new PathParts(uploadURL, conf); - assertEquals(uploadURL, parts.getUploadURL()); - String homeDir = "/user/" + System.getProperty("user.name"); - assertEquals(homeDir + "/xuser/" + down + "bar.txt", parts.getURIPath()); - assertEquals("bar.txt", parts.getName()); - assertEquals("hdfs", parts.getScheme()); - assertTrue("localhost".equals(parts.getHost()) || "localhost.localdomain".equals(parts.getHost())); - assertEquals(dfsClusterPort, parts.getPort()); - assertTrue(parts.getId().equals("hdfs://localhost:" + dfsClusterPort + homeDir + "/xuser/" + down + "bar.txt") - || parts.getId().equals("hdfs://localhost.localdomain:" + dfsClusterPort + homeDir + "/xuser/" + down + "bar.txt") - ); - assertFileNotFound(parts); - } - } - } - - try { - new PathParts("/user/foo/bar.txt", simpleConf); - fail("host/port resolution requires minimr conf, not a simple conf"); - } catch (IllegalArgumentException e) { - ; // expected - } - } - - private void assertFileNotFound(PathParts parts) { - try { - parts.getFileSystem().getFileStatus(parts.getUploadPath()); - fail(); - } catch (IOException e) { - ; // expected - } - } - - @Test - public void mrRun() throws Exception { - FileSystem fs = dfsCluster.getFileSystem(); - Path inDir = fs.makeQualified(new Path("/user/testing/testMapperReducer/input")); - fs.delete(inDir, true); - String DATADIR = "/user/testing/testMapperReducer/data"; - Path dataDir = fs.makeQualified(new Path(DATADIR)); - fs.delete(dataDir, true); - Path outDir = fs.makeQualified(new Path("/user/testing/testMapperReducer/output")); - fs.delete(outDir, true); - - assertTrue(fs.mkdirs(inDir)); - Path INPATH = new Path(inDir, "input.txt"); - OutputStream os = fs.create(INPATH); - Writer wr = new OutputStreamWriter(os, StandardCharsets.UTF_8); - wr.write(DATADIR + "/" + inputAvroFile); - wr.close(); - - assertTrue(fs.mkdirs(dataDir)); - fs.copyFromLocalFile(new Path(DOCUMENTS_DIR, inputAvroFile), dataDir); - - JobConf jobConf = getJobConf(); - jobConf.set("jobclient.output.filter", "ALL"); - if (ENABLE_LOCAL_JOB_RUNNER) { // enable Hadoop LocalJobRunner; this enables to run in debugger and set breakpoints - jobConf.set("mapred.job.tracker", "local"); - } - jobConf.setMaxMapAttempts(1); - jobConf.setMaxReduceAttempts(1); - jobConf.setJar(SEARCH_ARCHIVES_JAR); - - int shards = 2; - int maxReducers = Integer.MAX_VALUE; - if (ENABLE_LOCAL_JOB_RUNNER) { - // local job runner has a couple of limitations: only one reducer is supported and the DistributedCache doesn't work. - // see http://blog.cloudera.com/blog/2009/07/advice-on-qa-testing-your-mapreduce-jobs/ - maxReducers = 1; - shards = 1; - } - - String[] args = new String[] { - "--morphline-file=" + tempDir + "/test-morphlines/solrCellDocumentTypes.conf", - "--morphline-id=morphline1", - "--solr-home-dir=" + MINIMR_CONF_DIR.getAbsolutePath(), - "--output-dir=" + outDir.toString(), - "--shards=" + shards, - "--verbose", - numRuns % 2 == 0 ? "--input-list=" + INPATH.toString() : dataDir.toString(), - numRuns % 3 == 0 ? "--reducers=" + shards : (numRuns % 3 == 1 ? "--reducers=-1" : "--reducers=" + Math.min(8, maxReducers)) - }; - if (numRuns % 3 == 2) { - args = concat(args, new String[] {"--fanout=2"}); - } - if (numRuns == 0) { - // force (slow) MapReduce based randomization to get coverage for that as well - args = concat(new String[] {"-D", MapReduceIndexerTool.MAIN_MEMORY_RANDOMIZATION_THRESHOLD + "=-1"}, args); - } - MapReduceIndexerTool tool = createTool(); - int res = ToolRunner.run(jobConf, tool, args); - assertEquals(0, res); - Job job = tool.job; - assertTrue(job.isComplete()); - assertTrue(job.isSuccessful()); - - if (numRuns % 3 != 2) { - // Only run this check if mtree merge is disabled. - // With mtree merge enabled the BatchWriter counters aren't available anymore because - // variable "job" now refers to the merge job rather than the indexing job - assertEquals("Invalid counter " + SolrRecordWriter.class.getName() + "." + SolrCounters.DOCUMENTS_WRITTEN, - count, job.getCounters().findCounter(SolrCounters.class.getName(), SolrCounters.DOCUMENTS_WRITTEN.toString()).getValue()); - } - - // Check the output is as expected - outDir = new Path(outDir, MapReduceIndexerTool.RESULTS_DIR); - Path[] outputFiles = FileUtil.stat2Paths(fs.listStatus(outDir)); - - System.out.println("outputfiles:" + Arrays.toString(outputFiles)); - - UtilsForTests.validateSolrServerDocumentCount(MINIMR_CONF_DIR, fs, outDir, count, shards); - - // run again with --dryrun mode: - tool = createTool(); - args = concat(args, new String[] {"--dry-run"}); - res = ToolRunner.run(jobConf, tool, args); - assertEquals(0, res); - - numRuns++; - } - - protected static T[] concat(T[]... arrays) { - if (arrays.length <= 0) { - throw new IllegalArgumentException(); - } - Class clazz = null; - int length = 0; - for (T[] array : arrays) { - clazz = array.getClass(); - length += array.length; - } - T[] result = (T[]) Array.newInstance(clazz.getComponentType(), length); - int pos = 0; - for (T[] array : arrays) { - System.arraycopy(array, 0, result, pos, array.length); - pos += array.length; - } - return result; - } - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java deleted file mode 100644 index adc8d8851df..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java +++ /dev/null @@ -1,881 +0,0 @@ -/* - * 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.hadoop; - -import java.io.File; -import java.io.IOException; -import java.io.OutputStream; -import java.io.OutputStreamWriter; -import java.io.UnsupportedEncodingException; -import java.io.Writer; -import java.lang.reflect.Array; -import java.net.URI; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.security.authorize.ProxyUsers; -import org.apache.hadoop.util.JarFinder; -import org.apache.hadoop.util.ToolRunner; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.lucene.util.Constants; -import org.apache.lucene.util.LuceneTestCase; -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; -import org.apache.lucene.util.LuceneTestCase.Slow; -import org.apache.solr.SolrTestCaseJ4.SuppressSSL; -import org.apache.solr.client.solrj.SolrClient; -import org.apache.solr.client.solrj.SolrQuery; -import org.apache.solr.client.solrj.SolrQuery.ORDER; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.embedded.JettySolrRunner; -import org.apache.solr.client.solrj.impl.HttpSolrClient; -import org.apache.solr.client.solrj.request.QueryRequest; -import org.apache.solr.client.solrj.response.QueryResponse; -import org.apache.solr.cloud.AbstractFullDistribZkTestBase; -import org.apache.solr.cloud.AbstractZkTestCase; -import org.apache.solr.common.SolrDocument; -import org.apache.solr.common.SolrDocumentList; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.common.cloud.Replica; -import org.apache.solr.common.cloud.Slice; -import org.apache.solr.common.cloud.SolrZkClient; -import org.apache.solr.common.cloud.ZkCoreNodeProps; -import org.apache.solr.common.params.CollectionParams.CollectionAction; -import org.apache.solr.common.params.CoreAdminParams; -import org.apache.solr.common.params.ModifiableSolrParams; -import org.apache.solr.common.util.NamedList; -import org.apache.solr.hadoop.hack.MiniMRClientCluster; -import org.apache.solr.hadoop.hack.MiniMRClientClusterFactory; -import org.apache.solr.morphlines.solr.AbstractSolrMorphlineTestBase; -import org.apache.solr.util.BadHdfsThreadsFilter; -import org.apache.solr.util.BadMrClusterThreadsFilter; -import org.apache.solr.util.TimeOut; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; - -import com.carrotsearch.randomizedtesting.annotations.Nightly; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction.Action; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies; -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakZombies.Consequence; - -@ThreadLeakAction({Action.WARN}) -@ThreadLeakLingering(linger = 0) -@ThreadLeakZombies(Consequence.CONTINUE) -@ThreadLeakFilters(defaultFilters = true, filters = { - BadHdfsThreadsFilter.class, BadMrClusterThreadsFilter.class // hdfs currently leaks thread(s) -}) -@SuppressSSL // SSL does not work with this test for currently unknown reasons -@Slow -@Nightly -@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-9076") -public class MorphlineGoLiveMiniMRTest extends AbstractFullDistribZkTestBase { - - private static final int RECORD_COUNT = 2104; - private static final String RESOURCES_DIR = getFile("morphlines-core.marker").getParent(); - private static final String DOCUMENTS_DIR = RESOURCES_DIR + "/test-documents"; - private static final File MINIMR_INSTANCE_DIR = new File(RESOURCES_DIR + "/solr/minimr"); - private static final File MINIMR_CONF_DIR = new File(RESOURCES_DIR + "/solr/minimr"); - - private static String SEARCH_ARCHIVES_JAR; - - private static MiniDFSCluster dfsCluster = null; - private static MiniMRClientCluster mrCluster = null; - private static String tempDir; - - private final String inputAvroFile1; - private final String inputAvroFile2; - private final String inputAvroFile3; - - private static File solrHomeDirectory; - - @Override - public String getSolrHome() { - return solrHomeDirectory.getPath(); - } - - public MorphlineGoLiveMiniMRTest() { - this.inputAvroFile1 = "sample-statuses-20120521-100919.avro"; - this.inputAvroFile2 = "sample-statuses-20120906-141433.avro"; - this.inputAvroFile3 = "sample-statuses-20120906-141433-medium.avro"; - - sliceCount = TEST_NIGHTLY ? 5 : 3; - fixShardCount(TEST_NIGHTLY ? 5 : 3); - } - - @BeforeClass - public static void setupClass() throws Exception { - System.setProperty("solr.hdfs.blockcache.global", Boolean.toString(LuceneTestCase.random().nextBoolean())); - System.setProperty("solr.hdfs.blockcache.enabled", Boolean.toString(LuceneTestCase.random().nextBoolean())); - System.setProperty("solr.hdfs.blockcache.blocksperbank", "2048"); - - solrHomeDirectory = createTempDir().toFile(); - - assumeFalse("HDFS tests were disabled by -Dtests.disableHdfs", - Boolean.parseBoolean(System.getProperty("tests.disableHdfs", "false"))); - - assumeFalse("FIXME: This test does not work with Windows because of native library requirements", Constants.WINDOWS); - - AbstractZkTestCase.SOLRHOME = solrHomeDirectory; - FileUtils.copyDirectory(MINIMR_INSTANCE_DIR, AbstractZkTestCase.SOLRHOME); - tempDir = createTempDir().toFile().getAbsolutePath(); - - new File(tempDir).mkdirs(); - - FileUtils.copyFile(new File(RESOURCES_DIR + "/custom-mimetypes.xml"), new File(tempDir + "/custom-mimetypes.xml")); - - AbstractSolrMorphlineTestBase.setupMorphline(tempDir, "test-morphlines/solrCellDocumentTypes", true); - - - System.setProperty("hadoop.log.dir", new File(tempDir, "logs").getAbsolutePath()); - - int dataNodes = 2; - - JobConf conf = new JobConf(); - conf.set("dfs.block.access.token.enable", "false"); - conf.set("dfs.permissions", "true"); - conf.set("hadoop.security.authentication", "simple"); - conf.set("mapreduce.jobhistory.minicluster.fixed.ports", "false"); - conf.set("mapreduce.jobhistory.admin.address", "0.0.0.0:0"); - - conf.set(YarnConfiguration.NM_LOCAL_DIRS, tempDir + File.separator + "nm-local-dirs"); - conf.set(YarnConfiguration.DEFAULT_NM_LOG_DIRS, tempDir + File.separator + "nm-logs"); - - - new File(tempDir + File.separator + "nm-local-dirs").mkdirs(); - - System.setProperty("test.build.dir", tempDir + File.separator + "hdfs" + File.separator + "test-build-dir"); - System.setProperty("test.build.data", tempDir + File.separator + "hdfs" + File.separator + "build"); - System.setProperty("test.cache.data", tempDir + File.separator + "hdfs" + File.separator + "cache"); - - // Initialize AFTER test.build.dir is set, JarFinder uses it. - SEARCH_ARCHIVES_JAR = JarFinder.getJar(MapReduceIndexerTool.class); - - dfsCluster = new MiniDFSCluster(conf, dataNodes, true, null); - FileSystem fileSystem = dfsCluster.getFileSystem(); - fileSystem.mkdirs(new Path("/tmp")); - fileSystem.mkdirs(new Path("/user")); - fileSystem.mkdirs(new Path("/hadoop/mapred/system")); - fileSystem.setPermission(new Path("/tmp"), - FsPermission.valueOf("-rwxrwxrwx")); - fileSystem.setPermission(new Path("/user"), - FsPermission.valueOf("-rwxrwxrwx")); - fileSystem.setPermission(new Path("/hadoop/mapred/system"), - FsPermission.valueOf("-rwx------")); - - mrCluster = MiniMRClientClusterFactory.create(MorphlineGoLiveMiniMRTest.class, 1, conf, new File(tempDir, "mrCluster")); - - //new MiniMRCluster(0, 0, taskTrackers, nnURI, numDirs, racks, - //hosts, null, conf); - - ProxyUsers.refreshSuperUserGroupsConfiguration(conf); - } - - @Override - public void distribSetUp() throws Exception { - super.distribSetUp(); - System.setProperty("host", "127.0.0.1"); - System.setProperty("numShards", Integer.toString(sliceCount)); - URI uri = dfsCluster.getFileSystem().getUri(); - System.setProperty("solr.hdfs.home", uri.toString() + "/" + this.getClass().getName()); - uploadConfFiles(); - } - - @Override - public void distribTearDown() throws Exception { - super.distribTearDown(); - System.clearProperty("host"); - System.clearProperty("numShards"); - System.clearProperty("solr.hdfs.home"); - } - - @AfterClass - public static void teardownClass() throws Exception { - System.clearProperty("solr.hdfs.blockcache.global"); - System.clearProperty("solr.hdfs.blockcache.blocksperbank"); - System.clearProperty("solr.hdfs.blockcache.enabled"); - System.clearProperty("hadoop.log.dir"); - System.clearProperty("test.build.dir"); - System.clearProperty("test.build.data"); - System.clearProperty("test.cache.data"); - - if (mrCluster != null) { - mrCluster.stop(); - mrCluster = null; - } - if (dfsCluster != null) { - dfsCluster.shutdown(); - dfsCluster = null; - } - FileSystem.closeAll(); - } - - private JobConf getJobConf() throws IOException { - JobConf jobConf = new JobConf(mrCluster.getConfig()); - return jobConf; - } - - @Test - public void testBuildShardUrls() throws Exception { - // 2x3 - Integer numShards = 2; - List urls = new ArrayList<>(); - urls.add("shard1"); - urls.add("shard2"); - urls.add("shard3"); - urls.add("shard4"); - urls.add("shard5"); - urls.add("shard6"); - List> shardUrls = MapReduceIndexerTool.buildShardUrls(urls , numShards); - - assertEquals(shardUrls.toString(), 2, shardUrls.size()); - - for (List u : shardUrls) { - assertEquals(3, u.size()); - } - - // 1x6 - numShards = 1; - shardUrls = MapReduceIndexerTool.buildShardUrls(urls , numShards); - - assertEquals(shardUrls.toString(), 1, shardUrls.size()); - - for (List u : shardUrls) { - assertEquals(6, u.size()); - } - - // 6x1 - numShards = 6; - shardUrls = MapReduceIndexerTool.buildShardUrls(urls , numShards); - - assertEquals(shardUrls.toString(), 6, shardUrls.size()); - - for (List u : shardUrls) { - assertEquals(1, u.size()); - } - - // 3x2 - numShards = 3; - shardUrls = MapReduceIndexerTool.buildShardUrls(urls , numShards); - - assertEquals(shardUrls.toString(), 3, shardUrls.size()); - - for (List u : shardUrls) { - assertEquals(2, u.size()); - } - - // null shards, 6x1 - numShards = null; - shardUrls = MapReduceIndexerTool.buildShardUrls(urls , numShards); - - assertEquals(shardUrls.toString(), 6, shardUrls.size()); - - for (List u : shardUrls) { - assertEquals(1, u.size()); - } - - // null shards 3x1 - numShards = null; - - urls = new ArrayList<>(); - urls.add("shard1"); - urls.add("shard2"); - urls.add("shard3"); - - shardUrls = MapReduceIndexerTool.buildShardUrls(urls , numShards); - - assertEquals(shardUrls.toString(), 3, shardUrls.size()); - - for (List u : shardUrls) { - assertEquals(1, u.size()); - } - - // 2x(2,3) off balance - numShards = 2; - urls = new ArrayList<>(); - urls.add("shard1"); - urls.add("shard2"); - urls.add("shard3"); - urls.add("shard4"); - urls.add("shard5"); - shardUrls = MapReduceIndexerTool.buildShardUrls(urls , numShards); - - assertEquals(shardUrls.toString(), 2, shardUrls.size()); - - Set counts = new HashSet<>(); - counts.add(shardUrls.get(0).size()); - counts.add(shardUrls.get(1).size()); - - assertTrue(counts.contains(2)); - assertTrue(counts.contains(3)); - } - - private String[] prependInitialArgs(String[] args) { - String[] head = new String[] { - "--morphline-file=" + tempDir + "/test-morphlines/solrCellDocumentTypes.conf", - "--morphline-id=morphline1", - }; - return concat(head, args); - } - - @Nightly - @Test - public void test() throws Exception { - - waitForRecoveriesToFinish(false); - - FileSystem fs = dfsCluster.getFileSystem(); - Path inDir = fs.makeQualified(new Path( - "/user/testing/testMapperReducer/input")); - fs.delete(inDir, true); - String DATADIR = "/user/testing/testMapperReducer/data"; - Path dataDir = fs.makeQualified(new Path(DATADIR)); - fs.delete(dataDir, true); - Path outDir = fs.makeQualified(new Path( - "/user/testing/testMapperReducer/output")); - fs.delete(outDir, true); - - assertTrue(fs.mkdirs(inDir)); - Path INPATH = upAvroFile(fs, inDir, DATADIR, dataDir, inputAvroFile1); - - JobConf jobConf = getJobConf(); - jobConf.set("jobclient.output.filter", "ALL"); - // enable mapred.job.tracker = local to run in debugger and set breakpoints - // jobConf.set("mapred.job.tracker", "local"); - jobConf.setMaxMapAttempts(1); - jobConf.setMaxReduceAttempts(1); - jobConf.setJar(SEARCH_ARCHIVES_JAR); - - MapReduceIndexerTool tool; - int res; - QueryResponse results; - String[] args = new String[]{}; - List argList = new ArrayList<>(); - - try (HttpSolrClient server = getHttpSolrClient(cloudJettys.get(0).url)) { - - args = new String[]{ - "--solr-home-dir=" + MINIMR_CONF_DIR.getAbsolutePath(), - "--output-dir=" + outDir.toString(), - "--log4j=" + getFile("log4j.properties").getAbsolutePath(), - "--mappers=3", - random().nextBoolean() ? "--input-list=" + INPATH.toString() : dataDir.toString(), - "--go-live-threads", Integer.toString(random().nextInt(15) + 1), - "--verbose", - "--go-live" - }; - args = prependInitialArgs(args); - getShardUrlArgs(argList); - args = concat(args, argList.toArray(new String[0])); - - if (true) { - tool = new MapReduceIndexerTool(); - res = ToolRunner.run(jobConf, tool, args); - assertEquals(0, res); - assertTrue(tool.job.isComplete()); - assertTrue(tool.job.isSuccessful()); - results = server.query(new SolrQuery("*:*")); - assertEquals(20, results.getResults().getNumFound()); - } - - fs.delete(inDir, true); - fs.delete(outDir, true); - fs.delete(dataDir, true); - assertTrue(fs.mkdirs(inDir)); - INPATH = upAvroFile(fs, inDir, DATADIR, dataDir, inputAvroFile2); - - args = new String[]{ - "--solr-home-dir=" + MINIMR_CONF_DIR.getAbsolutePath(), - "--output-dir=" + outDir.toString(), - "--mappers=3", - "--verbose", - "--go-live", - random().nextBoolean() ? "--input-list=" + INPATH.toString() : dataDir.toString(), - "--go-live-threads", Integer.toString(random().nextInt(15) + 1) - }; - args = prependInitialArgs(args); - - getShardUrlArgs(argList); - args = concat(args, argList.toArray(new String[0])); - - if (true) { - tool = new MapReduceIndexerTool(); - res = ToolRunner.run(jobConf, tool, args); - assertEquals(0, res); - assertTrue(tool.job.isComplete()); - assertTrue(tool.job.isSuccessful()); - results = server.query(new SolrQuery("*:*")); - - assertEquals(22, results.getResults().getNumFound()); - } - - // try using zookeeper - String collection = "collection1"; - if (random().nextBoolean()) { - // sometimes, use an alias - createAlias("updatealias", "collection1"); - collection = "updatealias"; - } - - fs.delete(inDir, true); - fs.delete(outDir, true); - fs.delete(dataDir, true); - INPATH = upAvroFile(fs, inDir, DATADIR, dataDir, inputAvroFile3); - - cloudClient.deleteByQuery("*:*"); - cloudClient.commit(); - assertEquals(0, cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound()); - - args = new String[]{ - "--output-dir=" + outDir.toString(), - "--mappers=3", - "--reducers=6", - "--fanout=2", - "--verbose", - "--go-live", - random().nextBoolean() ? "--input-list=" + INPATH.toString() : dataDir.toString(), - "--zk-host", zkServer.getZkAddress(), - "--collection", collection - }; - args = prependInitialArgs(args); - - if (true) { - tool = new MapReduceIndexerTool(); - res = ToolRunner.run(jobConf, tool, args); - assertEquals(0, res); - assertTrue(tool.job.isComplete()); - assertTrue(tool.job.isSuccessful()); - - SolrDocumentList resultDocs = executeSolrQuery(cloudClient, "*:*"); - assertEquals(RECORD_COUNT, resultDocs.getNumFound()); - assertEquals(RECORD_COUNT, resultDocs.size()); - - // perform updates - for (int i = 0; i < RECORD_COUNT; i++) { - SolrDocument doc = resultDocs.get(i); - SolrInputDocument update = new SolrInputDocument(); - for (Map.Entry entry : doc.entrySet()) { - update.setField(entry.getKey(), entry.getValue()); - } - update.setField("user_screen_name", "Nadja" + i); - update.removeField("_version_"); - cloudClient.add(update); - } - cloudClient.commit(); - - // verify updates - SolrDocumentList resultDocs2 = executeSolrQuery(cloudClient, "*:*"); - assertEquals(RECORD_COUNT, resultDocs2.getNumFound()); - assertEquals(RECORD_COUNT, resultDocs2.size()); - for (int i = 0; i < RECORD_COUNT; i++) { - SolrDocument doc = resultDocs.get(i); - SolrDocument doc2 = resultDocs2.get(i); - assertEquals(doc.getFirstValue("id"), doc2.getFirstValue("id")); - assertEquals("Nadja" + i, doc2.getFirstValue("user_screen_name")); - assertEquals(doc.getFirstValue("text"), doc2.getFirstValue("text")); - - // perform delete - cloudClient.deleteById((String) doc.getFirstValue("id")); - } - cloudClient.commit(); - - // verify deletes - assertEquals(0, executeSolrQuery(cloudClient, "*:*").size()); - } - - cloudClient.deleteByQuery("*:*"); - cloudClient.commit(); - assertEquals(0, cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound()); - } - - // try using zookeeper with replication - String replicatedCollection = "replicated_collection"; - if (TEST_NIGHTLY) { - createCollection(replicatedCollection, 3, 3, 3); - } else { - createCollection(replicatedCollection, 2, 3, 2); - } - waitForRecoveriesToFinish(false); - cloudClient.setDefaultCollection(replicatedCollection); - fs.delete(inDir, true); - fs.delete(outDir, true); - fs.delete(dataDir, true); - assertTrue(fs.mkdirs(dataDir)); - INPATH = upAvroFile(fs, inDir, DATADIR, dataDir, inputAvroFile3); - - args = new String[] { - "--solr-home-dir=" + MINIMR_CONF_DIR.getAbsolutePath(), - "--output-dir=" + outDir.toString(), - "--mappers=3", - "--reducers=12", - "--fanout=2", - "--verbose", - "--go-live", - "--zk-host", zkServer.getZkAddress(), - "--collection", replicatedCollection, dataDir.toString() - }; - args = prependInitialArgs(args); - - if (true) { - tool = new MapReduceIndexerTool(); - res = ToolRunner.run(jobConf, tool, args); - assertEquals(0, res); - assertTrue(tool.job.isComplete()); - assertTrue(tool.job.isSuccessful()); - - SolrDocumentList resultDocs = executeSolrQuery(cloudClient, "*:*"); - assertEquals(RECORD_COUNT, resultDocs.getNumFound()); - assertEquals(RECORD_COUNT, resultDocs.size()); - - checkConsistency(replicatedCollection); - - // perform updates - for (int i = 0; i < RECORD_COUNT; i++) { - SolrDocument doc = resultDocs.get(i); - SolrInputDocument update = new SolrInputDocument(); - for (Map.Entry entry : doc.entrySet()) { - update.setField(entry.getKey(), entry.getValue()); - } - update.setField("user_screen_name", "@Nadja" + i); - update.removeField("_version_"); - cloudClient.add(update); - } - cloudClient.commit(); - - // verify updates - SolrDocumentList resultDocs2 = executeSolrQuery(cloudClient, "*:*"); - assertEquals(RECORD_COUNT, resultDocs2.getNumFound()); - assertEquals(RECORD_COUNT, resultDocs2.size()); - for (int i = 0; i < RECORD_COUNT; i++) { - SolrDocument doc = resultDocs.get(i); - SolrDocument doc2 = resultDocs2.get(i); - assertEquals(doc.getFieldValues("id"), doc2.getFieldValues("id")); - assertEquals(1, doc.getFieldValues("id").size()); - assertEquals(Arrays.asList("@Nadja" + i), doc2.getFieldValues("user_screen_name")); - assertEquals(doc.getFieldValues("text"), doc2.getFieldValues("text")); - - // perform delete - cloudClient.deleteById((String)doc.getFirstValue("id")); - } - cloudClient.commit(); - - // verify deletes - assertEquals(0, executeSolrQuery(cloudClient, "*:*").size()); - } - - // try using solr_url with replication - cloudClient.deleteByQuery("*:*"); - cloudClient.commit(); - assertEquals(0, executeSolrQuery(cloudClient, "*:*").getNumFound()); - assertEquals(0, executeSolrQuery(cloudClient, "*:*").size()); - fs.delete(inDir, true); - fs.delete(dataDir, true); - assertTrue(fs.mkdirs(dataDir)); - INPATH = upAvroFile(fs, inDir, DATADIR, dataDir, inputAvroFile3); - - args = new String[] { - "--solr-home-dir=" + MINIMR_CONF_DIR.getAbsolutePath(), - "--output-dir=" + outDir.toString(), - "--shards", "2", - "--mappers=3", - "--verbose", - "--go-live", - "--go-live-threads", Integer.toString(random().nextInt(15) + 1), dataDir.toString() - }; - args = prependInitialArgs(args); - - argList = new ArrayList<>(); - getShardUrlArgs(argList, replicatedCollection); - args = concat(args, argList.toArray(new String[0])); - - if (true) { - tool = new MapReduceIndexerTool(); - res = ToolRunner.run(jobConf, tool, args); - assertEquals(0, res); - assertTrue(tool.job.isComplete()); - assertTrue(tool.job.isSuccessful()); - - checkConsistency(replicatedCollection); - - assertEquals(RECORD_COUNT, executeSolrQuery(cloudClient, "*:*").size()); - } - - // delete collection - ModifiableSolrParams params = new ModifiableSolrParams(); - params.set("action", CollectionAction.DELETE.toString()); - params.set(CoreAdminParams.DELETE_INSTANCE_DIR, true); - params.set(CoreAdminParams.DELETE_DATA_DIR, true); - params.set(CoreAdminParams.DELETE_INDEX, true); - params.set("name", replicatedCollection); - QueryRequest request = new QueryRequest(params); - request.setPath("/admin/collections"); - cloudClient.request(request); - - final TimeOut timeout = new TimeOut(10, TimeUnit.SECONDS); - while (cloudClient.getZkStateReader().getClusterState().hasCollection(replicatedCollection)) { - if (timeout.hasTimedOut()) { - throw new AssertionError("Timeout waiting to see removed collection leave clusterstate"); - } - - Thread.sleep(200); - } - - if (TEST_NIGHTLY) { - createCollection(replicatedCollection, 3, 3, 3); - } else { - createCollection(replicatedCollection, 2, 3, 2); - } - - waitForRecoveriesToFinish(replicatedCollection, false); - printLayout(); - assertEquals(0, executeSolrQuery(cloudClient, "*:*").getNumFound()); - - - args = new String[] { - "--solr-home-dir=" + MINIMR_CONF_DIR.getAbsolutePath(), - "--output-dir=" + outDir.toString(), - "--shards", "2", - "--mappers=3", - "--verbose", - "--go-live", - "--go-live-threads", Integer.toString(random().nextInt(15) + 1), dataDir.toString() - }; - args = prependInitialArgs(args); - - argList = new ArrayList<>(); - getShardUrlArgs(argList, replicatedCollection); - args = concat(args, argList.toArray(new String[0])); - - tool = new MapReduceIndexerTool(); - res = ToolRunner.run(jobConf, tool, args); - assertEquals(0, res); - assertTrue(tool.job.isComplete()); - assertTrue(tool.job.isSuccessful()); - - checkConsistency(replicatedCollection); - - assertEquals(RECORD_COUNT, executeSolrQuery(cloudClient, "*:*").size()); - } - - private void getShardUrlArgs(List args) { - for (int i = 0; i < getShardCount(); i++) { - args.add("--shard-url"); - args.add(cloudJettys.get(i).url); - } - } - - private SolrDocumentList executeSolrQuery(SolrClient collection, String queryString) throws SolrServerException, IOException { - SolrQuery query = new SolrQuery(queryString).setRows(2 * RECORD_COUNT).addSort("id", ORDER.asc); - QueryResponse response = collection.query(query); - return response.getResults(); - } - - private void checkConsistency(String replicatedCollection) - throws Exception { - Collection slices = cloudClient.getZkStateReader().getClusterState() - .getSlices(replicatedCollection); - for (Slice slice : slices) { - Collection replicas = slice.getReplicas(); - long found = -1; - for (Replica replica : replicas) { - try (HttpSolrClient client = getHttpSolrClient(new ZkCoreNodeProps(replica).getCoreUrl())) { - SolrQuery query = new SolrQuery("*:*"); - query.set("distrib", false); - QueryResponse replicaResults = client.query(query); - long count = replicaResults.getResults().getNumFound(); - if (found != -1) { - assertEquals(slice.getName() + " is inconsistent " - + new ZkCoreNodeProps(replica).getCoreUrl(), found, count); - } - found = count; - } - } - } - } - - private void getShardUrlArgs(List args, String replicatedCollection) { - Collection slices = cloudClient.getZkStateReader().getClusterState().getSlices(replicatedCollection); - for (Slice slice : slices) { - Collection replicas = slice.getReplicas(); - for (Replica replica : replicas) { - args.add("--shard-url"); - args.add(new ZkCoreNodeProps(replica).getCoreUrl()); - } - } - } - - private Path upAvroFile(FileSystem fs, Path inDir, String DATADIR, - Path dataDir, String localFile) throws IOException, UnsupportedEncodingException { - Path INPATH = new Path(inDir, "input.txt"); - OutputStream os = fs.create(INPATH); - Writer wr = new OutputStreamWriter(os, StandardCharsets.UTF_8); - wr.write(DATADIR + File.separator + localFile); - wr.close(); - - assertTrue(fs.mkdirs(dataDir)); - fs.copyFromLocalFile(new Path(DOCUMENTS_DIR, localFile), dataDir); - return INPATH; - } - - @Override - public JettySolrRunner createJetty(File solrHome, String dataDir, - String shardList, String solrConfigOverride, String schemaOverride) - throws Exception { - - Properties props = new Properties(); - if (solrConfigOverride != null) - props.setProperty("solrconfig", solrConfigOverride); - if (schemaOverride != null) - props.setProperty("schema", schemaOverride); - if (shardList != null) - props.setProperty("shards", shardList); - - String collection = System.getProperty("collection"); - if (collection == null) - collection = "collection1"; - props.setProperty("collection", collection); - - JettySolrRunner jetty = new JettySolrRunner(solrHome.getAbsolutePath(), props, buildJettyConfig(context)); - jetty.start(); - - return jetty; - } - - private static void putConfig(SolrZkClient zkClient, File solrhome, String name) throws Exception { - putConfig(zkClient, solrhome, name, name); - } - - private static void putConfig(SolrZkClient zkClient, File solrhome, String srcName, String destName) - throws Exception { - - File file = new File(solrhome, "conf" + File.separator + srcName); - if (!file.exists()) { - // LOG.info("skipping " + file.getAbsolutePath() + - // " because it doesn't exist"); - return; - } - - String destPath = "/configs/conf1/" + destName; - // LOG.info("put " + file.getAbsolutePath() + " to " + destPath); - zkClient.makePath(destPath, file, false, true); - } - - private void uploadConfFiles() throws Exception { - // upload our own config files - SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(), 10000); - putConfig(zkClient, new File(RESOURCES_DIR + "/solr/solrcloud"), - "solrconfig.xml"); - putConfig(zkClient, MINIMR_CONF_DIR, "schema.xml"); - putConfig(zkClient, MINIMR_CONF_DIR, "elevate.xml"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_en.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_ar.txt"); - - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_bg.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_ca.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_cz.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_da.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_el.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_es.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_eu.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_de.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_fa.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_fi.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_fr.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_ga.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_gl.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_hi.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_hu.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_hy.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_id.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_it.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_ja.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_lv.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_nl.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_no.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_pt.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_ro.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_ru.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_sv.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_th.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stopwords_tr.txt"); - - putConfig(zkClient, MINIMR_CONF_DIR, "lang/contractions_ca.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/contractions_fr.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/contractions_ga.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "lang/contractions_it.txt"); - - putConfig(zkClient, MINIMR_CONF_DIR, "lang/stemdict_nl.txt"); - - putConfig(zkClient, MINIMR_CONF_DIR, "lang/hyphenations_ga.txt"); - - putConfig(zkClient, MINIMR_CONF_DIR, "stopwords.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "protwords.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "currency.xml"); - putConfig(zkClient, MINIMR_CONF_DIR, "open-exchange-rates.json"); - putConfig(zkClient, MINIMR_CONF_DIR, "mapping-ISOLatin1Accent.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "old_synonyms.txt"); - putConfig(zkClient, MINIMR_CONF_DIR, "synonyms.txt"); - zkClient.close(); - } - - protected static T[] concat(T[]... arrays) { - if (arrays.length <= 0) { - throw new IllegalArgumentException(); - } - Class clazz = null; - int length = 0; - for (T[] array : arrays) { - clazz = array.getClass(); - length += array.length; - } - T[] result = (T[]) Array.newInstance(clazz.getComponentType(), length); - int pos = 0; - for (T[] array : arrays) { - System.arraycopy(array, 0, result, pos, array.length); - pos += array.length; - } - return result; - } - - private NamedList createAlias(String alias, String collections) throws SolrServerException, IOException { - ModifiableSolrParams params = new ModifiableSolrParams(); - params.set("collections", collections); - params.set("name", alias); - params.set("action", CollectionAction.CREATEALIAS.toString()); - QueryRequest request = new QueryRequest(params); - request.setPath("/admin/collections"); - return cloudClient.request(request); - } - - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineMapperTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineMapperTest.java deleted file mode 100644 index 4f93a664fe9..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineMapperTest.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.hadoop; - -import java.net.URLEncoder; -import java.util.List; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mrunit.mapreduce.MapDriver; -import org.apache.hadoop.mrunit.types.Pair; -import org.apache.lucene.util.Constants; -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.hadoop.morphline.MorphlineMapper; -import org.apache.solr.util.BadHdfsThreadsFilter; -import org.junit.BeforeClass; -import org.junit.Test; - -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; - -@ThreadLeakFilters(defaultFilters = true, filters = { - BadHdfsThreadsFilter.class // hdfs currently leaks thread(s) -}) -@AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-9220") -public class MorphlineMapperTest extends MRUnitBase { - - @BeforeClass - public static void beforeClass() { - assumeFalse("Does not work on Windows, because it uses UNIX shell commands or POSIX paths", Constants.WINDOWS); - } - - @Test - public void testMapper() throws Exception { - MorphlineMapper mapper = new MorphlineMapper(); - MapDriver mapDriver = MapDriver.newMapDriver(mapper);; - - Configuration config = mapDriver.getConfiguration(); - setupHadoopConfig(config); - - mapDriver.withInput(new LongWritable(0L), new Text("hdfs://localhost/" + - URLEncoder.encode(DOCUMENTS_DIR, "UTF-8").replace("+", "%20") + - "/sample-statuses-20120906-141433.avro")); - - SolrInputDocument sid = new SolrInputDocument(); - sid.addField("id", "uniqueid1"); - sid.addField("user_name", "user1"); - sid.addField("text", "content of record one"); - SolrInputDocumentWritable sidw = new SolrInputDocumentWritable(sid); - - mapDriver - .withCacheArchive(solrHomeZip.getAbsolutePath()) - .withOutput(new Text("0"), sidw); - //mapDriver.runTest(); - List> result = mapDriver.run(); - for (Pair p: result) { - System.out.println(p.getFirst()); - System.out.println(p.getSecond()); - } - } -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineReducerTest.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineReducerTest.java deleted file mode 100644 index 31616d8944a..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineReducerTest.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.hadoop; - -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.TaskID; -import org.apache.hadoop.mapreduce.InputFormat; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.mrunit.mapreduce.ReduceDriver; -import org.apache.lucene.util.Constants; -import org.apache.solr.common.SolrInputDocument; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -import com.google.common.collect.Lists; - -@Ignore("This test cannot currently work because it uses a local filesystem output path for the indexes and Solr requires hdfs output paths") -public class MorphlineReducerTest extends MRUnitBase { - - @BeforeClass - public static void beforeClass2() { - assumeFalse("Does not work on Windows, because it uses UNIX shell commands or POSIX paths", Constants.WINDOWS); - - System.setProperty("verifyPartitionAssignment", "false"); - } - - @AfterClass - public static void afterClass2() { - System.clearProperty("verifyPartitionAssignment"); - } - - public static class MySolrReducer extends SolrReducer { - Context context; - - @Override - protected void setup(Context context) throws IOException, InterruptedException { - this.context = context; - - // handle a bug in MRUnit - should be fixed in MRUnit 1.0.0 - when(context.getTaskAttemptID()).thenAnswer(new Answer() { - @Override - public TaskAttemptID answer(final InvocationOnMock invocation) { - // FIXME MRUNIT seems to pass taskid to the reduce task as mapred.TaskID rather than mapreduce.TaskID - return new TaskAttemptID(new TaskID("000000000000", 0, true, 0), 0); - } - }); - - super.setup(context); - } - - } - - public static class NullInputFormat extends InputFormat { - @Override - public List getSplits(JobContext context) throws IOException, - InterruptedException { - return Lists.newArrayList(); - } - - @Override - public RecordReader createRecordReader(InputSplit split, - TaskAttemptContext context) throws IOException, InterruptedException { - return null; - } - - } - - @Test - public void testReducer() throws Exception { - MySolrReducer myReducer = new MySolrReducer(); - try { - ReduceDriver reduceDriver = ReduceDriver - .newReduceDriver(myReducer); - - Configuration config = reduceDriver.getConfiguration(); - setupHadoopConfig(config); - - List values = new ArrayList<>(); - SolrInputDocument sid = new SolrInputDocument(); - String id = "myid1"; - sid.addField("id", id); - sid.addField("text", "some unique text"); - SolrInputDocumentWritable sidw = new SolrInputDocumentWritable(sid); - values.add(sidw); - reduceDriver.withInput(new Text(id), values); - - reduceDriver.withCacheArchive(solrHomeZip.getAbsolutePath()); - - reduceDriver.withOutputFormat(SolrOutputFormat.class, - NullInputFormat.class); - - reduceDriver.run(); - - assertEquals("Expected 1 counter increment", 1, - reduceDriver.getCounters().findCounter(SolrCounters.class.getName(), - SolrCounters.DOCUMENTS_WRITTEN.toString()).getValue()); - } finally { - myReducer.cleanup(myReducer.context); - } - } - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/UtilsForTests.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/UtilsForTests.java deleted file mode 100644 index bc5148f5ae3..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/UtilsForTests.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.hadoop; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.solr.client.solrj.SolrQuery; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer; -import org.apache.solr.client.solrj.response.QueryResponse; - -import java.io.File; -import java.io.IOException; - -import static org.junit.Assert.assertEquals; - - -public class UtilsForTests { - - public static void validateSolrServerDocumentCount(File solrHomeDir, FileSystem fs, Path outDir, int expectedDocs, int expectedShards) - throws IOException, SolrServerException { - - long actualDocs = 0; - int actualShards = 0; - for (FileStatus dir : fs.listStatus(outDir)) { // for each shard - if (dir.getPath().getName().startsWith("part") && dir.isDirectory()) { - actualShards++; - try (EmbeddedSolrServer solr - = SolrRecordWriter.createEmbeddedSolrServer(new Path(solrHomeDir.getAbsolutePath()), fs, dir.getPath())) { - SolrQuery query = new SolrQuery(); - query.setQuery("*:*"); - QueryResponse resp = solr.query(query); - long numDocs = resp.getResults().getNumFound(); - actualDocs += numDocs; - } - } - } - assertEquals(expectedShards, actualShards); - assertEquals(expectedDocs, actualDocs); - } - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRClientCluster.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRClientCluster.java deleted file mode 100644 index be5ea01cd29..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRClientCluster.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.hadoop.hack; - -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; - -/* - * A simple interface for a client MR cluster used for testing. This interface - * provides basic methods which are independent of the underlying Mini Cluster ( - * either through MR1 or MR2). - */ -public interface MiniMRClientCluster { - - public void start() throws IOException; - - /** - * Stop and start back the cluster using the same configuration. - */ - public void restart() throws IOException; - - public void stop() throws IOException; - - public Configuration getConfig() throws IOException; - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRClientClusterFactory.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRClientClusterFactory.java deleted file mode 100644 index 2bf721b7a6c..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRClientClusterFactory.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.hadoop.hack; - -import java.io.File; -import java.io.IOException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.util.JarFinder; - -/** - * A MiniMRCluster factory. In MR2, it provides a wrapper MiniMRClientCluster - * interface around the MiniMRYarnCluster. While in MR1, it provides such - * wrapper around MiniMRCluster. This factory should be used in tests to provide - * an easy migration of tests across MR1 and MR2. - */ -public class MiniMRClientClusterFactory { - - public static MiniMRClientCluster create(Class caller, int noOfNMs, - Configuration conf, File testWorkDir) throws IOException { - return create(caller, caller.getSimpleName(), noOfNMs, conf, testWorkDir); - } - - public static MiniMRClientCluster create(Class caller, String identifier, - int noOfNMs, Configuration conf, File testWorkDir) throws IOException { - - if (conf == null) { - conf = new Configuration(); - } - - FileSystem fs = FileSystem.get(conf); - - Path testRootDir = new Path(testWorkDir.getPath(), identifier + "-tmpDir") - .makeQualified(fs); - Path appJar = new Path(testRootDir, "MRAppJar.jar"); - - // Copy MRAppJar and make it private. - Path appMasterJar = new Path(MiniMRYarnCluster.APPJAR); - - fs.copyFromLocalFile(appMasterJar, appJar); - fs.setPermission(appJar, new FsPermission("744")); - - Job job = Job.getInstance(conf); - - job.addFileToClassPath(appJar); - - Path callerJar = new Path(JarFinder.getJar(caller)); - Path remoteCallerJar = new Path(testRootDir, callerJar.getName()); - fs.copyFromLocalFile(callerJar, remoteCallerJar); - fs.setPermission(remoteCallerJar, new FsPermission("744")); - job.addFileToClassPath(remoteCallerJar); - - MiniMRYarnCluster miniMRYarnCluster; - try { - miniMRYarnCluster = new MiniMRYarnCluster(identifier, - noOfNMs, testWorkDir); - } catch (Exception e) { - throw new RuntimeException(e); - } - job.getConfiguration().set("minimrclientcluster.caller.name", - identifier); - job.getConfiguration().setInt("minimrclientcluster.nodemanagers.number", - noOfNMs); - miniMRYarnCluster.init(job.getConfiguration()); - miniMRYarnCluster.start(); - - return new MiniMRYarnClusterAdapter(miniMRYarnCluster, testWorkDir); - } - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRCluster.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRCluster.java deleted file mode 100644 index cf872abdb1c..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRCluster.java +++ /dev/null @@ -1,266 +0,0 @@ -/* - * 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.hadoop.hack; - -import java.io.File; -import java.io.IOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.JobID; -import org.apache.hadoop.mapred.JobPriority; -import org.apache.hadoop.mapred.MapTaskCompletionEventsUpdate; -import org.apache.hadoop.mapred.TaskCompletionEvent; -import org.apache.hadoop.security.AccessControlException; -import org.apache.hadoop.security.UserGroupInformation; -import org.apache.lucene.util.LuceneTestCase; - - -/** - * This class is an MR2 replacement for older MR1 MiniMRCluster, that was used - * by tests prior to MR2. This replacement class uses the new MiniMRYarnCluster - * in MR2 but provides the same old MR1 interface, so tests can be migrated from - * MR1 to MR2 with minimal changes. - * - * Due to major differences between MR1 and MR2, a number of methods are either - * unimplemented/unsupported or were re-implemented to provide wrappers around - * MR2 functionality. - * - * @deprecated Use {@link org.apache.hadoop.mapred.MiniMRClientClusterFactory} - * instead - */ -@Deprecated -public class MiniMRCluster { - private static final Log LOG = LogFactory.getLog(MiniMRCluster.class); - - private MiniMRClientCluster mrClientCluster; - - public String getTaskTrackerLocalDir(int taskTracker) { - throw new UnsupportedOperationException(); - } - - public String[] getTaskTrackerLocalDirs(int taskTracker) { - throw new UnsupportedOperationException(); - } - - class JobTrackerRunner { - // Mock class - } - - class TaskTrackerRunner { - // Mock class - } - - public JobTrackerRunner getJobTrackerRunner() { - throw new UnsupportedOperationException(); - } - - TaskTrackerRunner getTaskTrackerRunner(int id) { - throw new UnsupportedOperationException(); - } - - public int getNumTaskTrackers() { - throw new UnsupportedOperationException(); - } - - public void setInlineCleanupThreads() { - throw new UnsupportedOperationException(); - } - - public void waitUntilIdle() { - throw new UnsupportedOperationException(); - } - - private void waitTaskTrackers() { - throw new UnsupportedOperationException(); - } - - public int getJobTrackerPort() { - throw new UnsupportedOperationException(); - } - - public JobConf createJobConf() { - JobConf jobConf = null; - try { - jobConf = new JobConf(mrClientCluster.getConfig()); - } catch (IOException e) { - LOG.error(e); - } - return jobConf; - } - - public JobConf createJobConf(JobConf conf) { - JobConf jobConf = null; - try { - jobConf = new JobConf(mrClientCluster.getConfig()); - } catch (IOException e) { - LOG.error(e); - } - return jobConf; - } - - static JobConf configureJobConf(JobConf conf, String namenode, - int jobTrackerPort, int jobTrackerInfoPort, UserGroupInformation ugi) { - throw new UnsupportedOperationException(); - } - - public MiniMRCluster(int numTaskTrackers, String namenode, int numDir, - String[] racks, String[] hosts) throws Exception { - this(0, 0, numTaskTrackers, namenode, numDir, racks, hosts); - } - - public MiniMRCluster(int numTaskTrackers, String namenode, int numDir, - String[] racks, String[] hosts, JobConf conf) throws Exception { - this(0, 0, numTaskTrackers, namenode, numDir, racks, hosts, null, conf); - } - - public MiniMRCluster(int numTaskTrackers, String namenode, int numDir) - throws Exception { - this(0, 0, numTaskTrackers, namenode, numDir); - } - - public MiniMRCluster(int jobTrackerPort, int taskTrackerPort, - int numTaskTrackers, String namenode, int numDir) throws Exception { - this(jobTrackerPort, taskTrackerPort, numTaskTrackers, namenode, numDir, - null); - } - - public MiniMRCluster(int jobTrackerPort, int taskTrackerPort, - int numTaskTrackers, String namenode, int numDir, String[] racks) - throws Exception { - this(jobTrackerPort, taskTrackerPort, numTaskTrackers, namenode, numDir, - racks, null); - } - - public MiniMRCluster(int jobTrackerPort, int taskTrackerPort, - int numTaskTrackers, String namenode, int numDir, String[] racks, - String[] hosts) throws Exception { - this(jobTrackerPort, taskTrackerPort, numTaskTrackers, namenode, numDir, - racks, hosts, null); - } - - public MiniMRCluster(int jobTrackerPort, int taskTrackerPort, - int numTaskTrackers, String namenode, int numDir, String[] racks, - String[] hosts, UserGroupInformation ugi) throws Exception { - this(jobTrackerPort, taskTrackerPort, numTaskTrackers, namenode, numDir, - racks, hosts, ugi, null); - } - - public MiniMRCluster(int jobTrackerPort, int taskTrackerPort, - int numTaskTrackers, String namenode, int numDir, String[] racks, - String[] hosts, UserGroupInformation ugi, JobConf conf) - throws Exception { - this(jobTrackerPort, taskTrackerPort, numTaskTrackers, namenode, numDir, - racks, hosts, ugi, conf, 0); - } - - public MiniMRCluster(int jobTrackerPort, int taskTrackerPort, - int numTaskTrackers, String namenode, int numDir, String[] racks, - String[] hosts, UserGroupInformation ugi, JobConf conf, - int numTrackerToExclude) throws Exception { - if (conf == null) conf = new JobConf(); - FileSystem.setDefaultUri(conf, namenode); - String identifier = this.getClass().getSimpleName() + "_" - + Integer.toString(LuceneTestCase.random().nextInt(Integer.MAX_VALUE)); - mrClientCluster = MiniMRClientClusterFactory.create(this.getClass(), - identifier, numTaskTrackers, conf, new File(conf.get("testWorkDir"))); - } - - public UserGroupInformation getUgi() { - throw new UnsupportedOperationException(); - } - - public TaskCompletionEvent[] getTaskCompletionEvents(JobID id, int from, - int max) throws IOException { - throw new UnsupportedOperationException(); - } - - public void setJobPriority(JobID jobId, JobPriority priority) - throws AccessControlException, IOException { - throw new UnsupportedOperationException(); - } - - public JobPriority getJobPriority(JobID jobId) { - throw new UnsupportedOperationException(); - } - - public long getJobFinishTime(JobID jobId) { - throw new UnsupportedOperationException(); - } - - public void initializeJob(JobID jobId) throws IOException { - throw new UnsupportedOperationException(); - } - - public MapTaskCompletionEventsUpdate getMapTaskCompletionEventsUpdates( - int index, JobID jobId, int max) throws IOException { - throw new UnsupportedOperationException(); - } - - public JobConf getJobTrackerConf() { - JobConf jobConf = null; - try { - jobConf = new JobConf(mrClientCluster.getConfig()); - } catch (IOException e) { - LOG.error(e); - } - return jobConf; - } - - public int getFaultCount(String hostName) { - throw new UnsupportedOperationException(); - } - - public void startJobTracker() { - // Do nothing - } - - public void startJobTracker(boolean wait) { - // Do nothing - } - - public void stopJobTracker() { - // Do nothing - } - - public void stopTaskTracker(int id) { - // Do nothing - } - - public void startTaskTracker(String host, String rack, int idx, int numDir) - throws IOException { - // Do nothing - } - - void addTaskTracker(TaskTrackerRunner taskTracker) { - throw new UnsupportedOperationException(); - } - - int getTaskTrackerID(String trackerName) { - throw new UnsupportedOperationException(); - } - - public void shutdown() { - try { - mrClientCluster.stop(); - } catch (IOException e) { - LOG.error(e); - } - } -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRYarnCluster.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRYarnCluster.java deleted file mode 100644 index 8fa1b3132bc..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRYarnCluster.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * 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.hadoop.hack; - -import java.io.File; -import java.io.IOException; -import java.util.Locale; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.fs.FileContext; -import org.apache.hadoop.fs.LocalFileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.LocalContainerLauncher; -import org.apache.hadoop.mapred.ShuffleHandler; -import org.apache.hadoop.mapreduce.MRConfig; -import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; -import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.service.Service; -import org.apache.hadoop.util.JarFinder; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; -import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor; - -/** - * Configures and starts the MR-specific components in the YARN cluster. - * - */ -public class MiniMRYarnCluster extends MiniYARNCluster { - - public static final String APPJAR = JarFinder.getJar(LocalContainerLauncher.class); - - private static final Log LOG = LogFactory.getLog(MiniMRYarnCluster.class); - private JobHistoryServer historyServer; - private JobHistoryServerWrapper historyServerWrapper; - - public MiniMRYarnCluster(String testName, File testWorkDir) { - this(testName, 1, testWorkDir); - } - - public MiniMRYarnCluster(String testName, int noOfNMs, File testWorkDir) { - super(testName, noOfNMs, 4, 4, testWorkDir); - //TODO: add the history server - historyServerWrapper = new JobHistoryServerWrapper(); - addService(historyServerWrapper); - } - - @Override - public void serviceInit(Configuration conf) throws Exception { - conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME); - if (conf.get(MRJobConfig.MR_AM_STAGING_DIR) == null) { - conf.set(MRJobConfig.MR_AM_STAGING_DIR, new File(getTestWorkDir(), - "apps_staging_dir/").getAbsolutePath()); - } - - // By default, VMEM monitoring disabled, PMEM monitoring enabled. - if (!conf.getBoolean( - MRConfig.MAPREDUCE_MINICLUSTER_CONTROL_RESOURCE_MONITORING, - MRConfig.DEFAULT_MAPREDUCE_MINICLUSTER_CONTROL_RESOURCE_MONITORING)) { - conf.setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false); - conf.setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false); - } - - conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, "000"); - - try { - Path stagingPath = FileContext.getFileContext(conf).makeQualified( - new Path(conf.get(MRJobConfig.MR_AM_STAGING_DIR))); - /* - * Re-configure the staging path on Windows if the file system is localFs. - * We need to use a absolute path that contains the drive letter. The unit - * test could run on a different drive than the AM. We can run into the - * issue that job files are localized to the drive where the test runs on, - * while the AM starts on a different drive and fails to find the job - * metafiles. Using absolute path can avoid this ambiguity. - */ - if (Path.WINDOWS) { - if (LocalFileSystem.class.isInstance(stagingPath.getFileSystem(conf))) { - conf.set(MRJobConfig.MR_AM_STAGING_DIR, - new File(conf.get(MRJobConfig.MR_AM_STAGING_DIR)) - .getAbsolutePath()); - } - } - FileContext fc=FileContext.getFileContext(stagingPath.toUri(), conf); - if (fc.util().exists(stagingPath)) { - LOG.info(stagingPath + " exists! deleting..."); - fc.delete(stagingPath, true); - } - LOG.info("mkdir: " + stagingPath); - //mkdir the staging directory so that right permissions are set while running as proxy user - fc.mkdir(stagingPath, null, true); - //mkdir done directory as well - String doneDir = JobHistoryUtils.getConfiguredHistoryServerDoneDirPrefix(conf); - Path doneDirPath = fc.makeQualified(new Path(doneDir)); - fc.mkdir(doneDirPath, null, true); - } catch (IOException e) { - throw new YarnRuntimeException("Could not create staging directory. ", e); - } - conf.set(MRConfig.MASTER_ADDRESS, "test"); // The default is local because of - // which shuffle doesn't happen - //configure the shuffle service in NM - conf.setStrings(YarnConfiguration.NM_AUX_SERVICES, - new String[] { ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID }); - conf.setClass(String.format(Locale.ENGLISH, YarnConfiguration.NM_AUX_SERVICE_FMT, - ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID), ShuffleHandler.class, - Service.class); - - // Non-standard shuffle port - conf.setInt(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY, 0); - - conf.setClass(YarnConfiguration.NM_CONTAINER_EXECUTOR, - DefaultContainerExecutor.class, ContainerExecutor.class); - - // TestMRJobs is for testing non-uberized operation only; see TestUberAM - // for corresponding uberized tests. - conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false); - - super.serviceInit(conf); - } - - private class JobHistoryServerWrapper extends AbstractService { - public JobHistoryServerWrapper() { - super(JobHistoryServerWrapper.class.getName()); - } - - @Override - public synchronized void serviceStart() throws Exception { - try { - if (!getConfig().getBoolean( - JHAdminConfig.MR_HISTORY_MINICLUSTER_FIXED_PORTS, - JHAdminConfig.DEFAULT_MR_HISTORY_MINICLUSTER_FIXED_PORTS)) { - // pick free random ports. - getConfig().set(JHAdminConfig.MR_HISTORY_ADDRESS, - MiniYARNCluster.getHostname() + ":0"); - getConfig().set(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS, - MiniYARNCluster.getHostname() + ":0"); - } - historyServer = new JobHistoryServer(); - historyServer.init(getConfig()); - new Thread() { - public void run() { - historyServer.start(); - }; - }.start(); - while (historyServer.getServiceState() == STATE.INITED) { - LOG.info("Waiting for HistoryServer to start..."); - Thread.sleep(1500); - } - //TODO Add a timeout. State.STOPPED check ? - if (historyServer.getServiceState() != STATE.STARTED) { - throw new IOException("HistoryServer failed to start"); - } - super.serviceStart(); - } catch (Throwable t) { - throw new YarnRuntimeException(t); - } - //need to do this because historyServer.init creates a new Configuration - getConfig().set(JHAdminConfig.MR_HISTORY_ADDRESS, - historyServer.getConfig().get(JHAdminConfig.MR_HISTORY_ADDRESS)); - getConfig().set(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS, - historyServer.getConfig().get(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS)); - - LOG.info("MiniMRYARN ResourceManager address: " + - getConfig().get(YarnConfiguration.RM_ADDRESS)); - LOG.info("MiniMRYARN ResourceManager web address: " + - getConfig().get(YarnConfiguration.RM_WEBAPP_ADDRESS)); - LOG.info("MiniMRYARN HistoryServer address: " + - getConfig().get(JHAdminConfig.MR_HISTORY_ADDRESS)); - LOG.info("MiniMRYARN HistoryServer web address: " + - getConfig().get(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS)); - } - - @Override - public synchronized void serviceStop() throws Exception { - if (historyServer != null) { - historyServer.stop(); - } - super.serviceStop(); - } - } - - public JobHistoryServer getHistoryServer() { - return this.historyServer; - } -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRYarnClusterAdapter.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRYarnClusterAdapter.java deleted file mode 100644 index 08ab881005b..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniMRYarnClusterAdapter.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.hadoop.hack; - -import java.io.File; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; -import org.apache.hadoop.service.Service.STATE; -import org.apache.hadoop.yarn.conf.YarnConfiguration; - -/** - * An adapter for MiniMRYarnCluster providing a MiniMRClientCluster interface. - * This interface could be used by tests across both MR1 and MR2. - */ -public class MiniMRYarnClusterAdapter implements MiniMRClientCluster { - - private MiniMRYarnCluster miniMRYarnCluster; - - private File testWorkDir; - - private static final Log LOG = LogFactory.getLog(MiniMRYarnClusterAdapter.class); - - public MiniMRYarnClusterAdapter(MiniMRYarnCluster miniMRYarnCluster, File testWorkDir) { - this.miniMRYarnCluster = miniMRYarnCluster; - this.testWorkDir = testWorkDir; - } - - @Override - public Configuration getConfig() { - return miniMRYarnCluster.getConfig(); - } - - @Override - public void start() { - miniMRYarnCluster.start(); - } - - @Override - public void stop() { - miniMRYarnCluster.stop(); - } - - @Override - public void restart() { - if (!miniMRYarnCluster.getServiceState().equals(STATE.STARTED)){ - LOG.warn("Cannot restart the mini cluster, start it first"); - return; - } - Configuration oldConf = new Configuration(getConfig()); - String callerName = oldConf.get("minimrclientcluster.caller.name", - this.getClass().getName()); - int noOfNMs = oldConf.getInt("minimrclientcluster.nodemanagers.number", 1); - oldConf.setBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, true); - oldConf.setBoolean(JHAdminConfig.MR_HISTORY_MINICLUSTER_FIXED_PORTS, true); - stop(); - miniMRYarnCluster = new MiniMRYarnCluster(callerName, noOfNMs, testWorkDir); - miniMRYarnCluster.init(oldConf); - miniMRYarnCluster.start(); - } - -} diff --git a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniYARNCluster.java b/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniYARNCluster.java deleted file mode 100644 index 6931f87e2ce..00000000000 --- a/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/hack/MiniYARNCluster.java +++ /dev/null @@ -1,409 +0,0 @@ -/* - * 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.hadoop.hack; - -import java.io.File; -import java.io.IOException; -import java.net.InetAddress; -import java.net.UnknownHostException; -import java.util.Locale; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileContext; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; -import org.apache.hadoop.service.AbstractService; -import org.apache.hadoop.service.CompositeService; -import org.apache.hadoop.util.Shell; -import org.apache.hadoop.util.Shell.ShellCommandExecutor; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.event.Dispatcher; -import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.apache.hadoop.yarn.factories.RecordFactory; -import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.server.api.ResourceTracker; -import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest; -import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; -import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; -import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; -import org.apache.hadoop.yarn.server.nodemanager.Context; -import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService; -import org.apache.hadoop.yarn.server.nodemanager.NodeManager; -import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater; -import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService; - -public class MiniYARNCluster extends CompositeService { - - private static final Log LOG = LogFactory.getLog(MiniYARNCluster.class); - - // temp fix until metrics system can auto-detect itself running in unit test: - static { - DefaultMetricsSystem.setMiniClusterMode(true); - } - - private NodeManager[] nodeManagers; - private ResourceManager resourceManager; - - private ResourceManagerWrapper resourceManagerWrapper; - - private File testWorkDir; - - // Number of nm-local-dirs per nodemanager - private int numLocalDirs; - // Number of nm-log-dirs per nodemanager - private int numLogDirs; - - /** - * @param testName name of the test - * @param noOfNodeManagers the number of node managers in the cluster - * @param numLocalDirs the number of nm-local-dirs per nodemanager - * @param numLogDirs the number of nm-log-dirs per nodemanager - */ - public MiniYARNCluster(String testName, int noOfNodeManagers, - int numLocalDirs, int numLogDirs, File testWorkDir) { - super(testName.replace("$", "")); - this.numLocalDirs = numLocalDirs; - this.numLogDirs = numLogDirs; - String testSubDir = testName.replace("$", ""); - File targetWorkDir = new File(testWorkDir, testSubDir); - try { - FileContext.getLocalFSFileContext().delete( - new Path(targetWorkDir.getAbsolutePath()), true); - } catch (Exception e) { - LOG.warn("COULD NOT CLEANUP", e); - throw new YarnRuntimeException("could not cleanup test dir: "+ e, e); - } - - if (Shell.WINDOWS) { - // The test working directory can exceed the maximum path length supported - // by some Windows APIs and cmd.exe (260 characters). To work around this, - // create a symlink in temporary storage with a much shorter path, - // targeting the full path to the test working directory. Then, use the - // symlink as the test working directory. - String targetPath = targetWorkDir.getAbsolutePath(); - File link = new File(System.getProperty("java.io.tmpdir"), - String.valueOf(System.nanoTime())); - String linkPath = link.getAbsolutePath(); - - try { - FileContext.getLocalFSFileContext().delete(new Path(linkPath), true); - } catch (IOException e) { - throw new YarnRuntimeException("could not cleanup symlink: " + linkPath, e); - } - - // Guarantee target exists before creating symlink. - targetWorkDir.mkdirs(); - - ShellCommandExecutor shexec = new ShellCommandExecutor( - Shell.getSymlinkCommand(targetPath, linkPath)); - try { - shexec.execute(); - } catch (IOException e) { - throw new YarnRuntimeException(String.format(Locale.ENGLISH, - "failed to create symlink from %s to %s, shell output: %s", linkPath, - targetPath, shexec.getOutput()), e); - } - - this.testWorkDir = link; - } else { - this.testWorkDir = targetWorkDir; - } - - resourceManagerWrapper = new ResourceManagerWrapper(); - addService(resourceManagerWrapper); - nodeManagers = new CustomNodeManager[noOfNodeManagers]; - for(int index = 0; index < noOfNodeManagers; index++) { - addService(new NodeManagerWrapper(index)); - nodeManagers[index] = new CustomNodeManager(); - } - } - - @Override - public void serviceInit(Configuration conf) throws Exception { - super.serviceInit(conf instanceof YarnConfiguration ? conf - : new YarnConfiguration( - conf)); - } - - public File getTestWorkDir() { - return testWorkDir; - } - - public ResourceManager getResourceManager() { - return this.resourceManager; - } - - public NodeManager getNodeManager(int i) { - return this.nodeManagers[i]; - } - - public static String getHostname() { - try { - return InetAddress.getLocalHost().getHostName(); - } - catch (UnknownHostException ex) { - throw new RuntimeException(ex); - } - } - - private class ResourceManagerWrapper extends AbstractService { - public ResourceManagerWrapper() { - super(ResourceManagerWrapper.class.getName()); - } - - @Override - public synchronized void serviceStart() throws Exception { - try { - getConfig().setBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, true); - if (!getConfig().getBoolean( - YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, - YarnConfiguration.DEFAULT_YARN_MINICLUSTER_FIXED_PORTS)) { - // pick free random ports. - String hostname = MiniYARNCluster.getHostname(); - getConfig().set(YarnConfiguration.RM_ADDRESS, - hostname + ":0"); - getConfig().set(YarnConfiguration.RM_ADMIN_ADDRESS, - hostname + ":0"); - getConfig().set(YarnConfiguration.RM_SCHEDULER_ADDRESS, - hostname + ":0"); - getConfig().set(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS, - hostname + ":0"); - getConfig().set(YarnConfiguration.RM_WEBAPP_ADDRESS, - hostname + ":0"); - } - resourceManager = new ResourceManager() { - @Override - protected void doSecureLogin() throws IOException { - // Don't try to login using keytab in the testcase. - }; - }; - resourceManager.init(getConfig()); - new Thread() { - public void run() { - resourceManager.start(); - }; - }.start(); - int waitCount = 0; - while (resourceManager.getServiceState() == STATE.INITED - && waitCount++ < 60) { - LOG.info("Waiting for RM to start..."); - Thread.sleep(1500); - } - if (resourceManager.getServiceState() != STATE.STARTED) { - // RM could have failed. - throw new IOException( - "ResourceManager failed to start. Final state is " - + resourceManager.getServiceState()); - } - super.serviceStart(); - } catch (Throwable t) { - throw new YarnRuntimeException(t); - } - LOG.info("MiniYARN ResourceManager address: " + - getConfig().get(YarnConfiguration.RM_ADDRESS)); - LOG.info("MiniYARN ResourceManager web address: " + - getConfig().get(YarnConfiguration.RM_WEBAPP_ADDRESS)); - } - - @Override - public synchronized void serviceStop() throws Exception { - if (resourceManager != null) { - resourceManager.stop(); - } - super.serviceStop(); - - if (Shell.WINDOWS) { - // On Windows, clean up the short temporary symlink that was created to - // work around path length limitation. - String testWorkDirPath = testWorkDir.getAbsolutePath(); - try { - FileContext.getLocalFSFileContext().delete(new Path(testWorkDirPath), - true); - } catch (IOException e) { - LOG.warn("could not cleanup symlink: " + - testWorkDir.getAbsolutePath()); - } - } - } - } - - private class NodeManagerWrapper extends AbstractService { - int index = 0; - - public NodeManagerWrapper(int i) { - super(NodeManagerWrapper.class.getName() + "_" + i); - index = i; - } - - public synchronized void serviceInit(Configuration conf) throws Exception { - Configuration config = new YarnConfiguration(conf); - super.serviceInit(config); - } - - /** - * Create local/log directories - * @param dirType type of directories i.e. local dirs or log dirs - * @param numDirs number of directories - * @return the created directories as a comma delimited String - */ - private String prepareDirs(String dirType, int numDirs) { - File []dirs = new File[numDirs]; - String dirsString = ""; - for (int i = 0; i < numDirs; i++) { - dirs[i]= new File(testWorkDir, MiniYARNCluster.this.getName() - + "-" + dirType + "Dir-nm-" + index + "_" + i); - dirs[i].mkdirs(); - LOG.info("Created " + dirType + "Dir in " + dirs[i].getAbsolutePath()); - String delimiter = (i > 0) ? "," : ""; - dirsString = dirsString.concat(delimiter + dirs[i].getAbsolutePath()); - } - return dirsString; - } - - public synchronized void serviceStart() throws Exception { - try { - // create nm-local-dirs and configure them for the nodemanager - String localDirsString = prepareDirs("local", numLocalDirs); - getConfig().set(YarnConfiguration.NM_LOCAL_DIRS, localDirsString); - // create nm-log-dirs and configure them for the nodemanager - String logDirsString = prepareDirs("log", numLogDirs); - getConfig().set(YarnConfiguration.NM_LOG_DIRS, logDirsString); - - File remoteLogDir = - new File(testWorkDir, MiniYARNCluster.this.getName() - + "-remoteLogDir-nm-" + index); - remoteLogDir.mkdir(); - getConfig().set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, - remoteLogDir.getAbsolutePath()); - // By default AM + 2 containers - getConfig().setInt(YarnConfiguration.NM_PMEM_MB, 4*1024); - getConfig().set(YarnConfiguration.NM_ADDRESS, - MiniYARNCluster.getHostname() + ":0"); - getConfig().set(YarnConfiguration.NM_LOCALIZER_ADDRESS, - MiniYARNCluster.getHostname() + ":0"); - getConfig().set(YarnConfiguration.NM_WEBAPP_ADDRESS, - MiniYARNCluster.getHostname() + ":0"); - - // Disable resource checks by default - if (!getConfig().getBoolean( - YarnConfiguration.YARN_MINICLUSTER_CONTROL_RESOURCE_MONITORING, - YarnConfiguration. - DEFAULT_YARN_MINICLUSTER_CONTROL_RESOURCE_MONITORING)) { - getConfig().setBoolean(YarnConfiguration.NM_PMEM_CHECK_ENABLED, false); - getConfig().setBoolean(YarnConfiguration.NM_VMEM_CHECK_ENABLED, false); - } - - LOG.info("Starting NM: " + index); - nodeManagers[index].init(getConfig()); - new Thread() { - public void run() { - nodeManagers[index].start(); - }; - }.start(); - int waitCount = 0; - while (nodeManagers[index].getServiceState() == STATE.INITED - && waitCount++ < 60) { - LOG.info("Waiting for NM " + index + " to start..."); - Thread.sleep(1000); - } - if (nodeManagers[index].getServiceState() != STATE.STARTED) { - // RM could have failed. - throw new IOException("NodeManager " + index + " failed to start"); - } - super.serviceStart(); - } catch (Throwable t) { - throw new YarnRuntimeException(t); - } - } - - @Override - public synchronized void serviceStop() throws Exception { - if (nodeManagers[index] != null) { - nodeManagers[index].stop(); - } - super.serviceStop(); - } - } - - private class CustomNodeManager extends NodeManager { - @Override - protected void doSecureLogin() throws IOException { - // Don't try to login using keytab in the testcase. - }; - - @Override - protected NodeStatusUpdater createNodeStatusUpdater(Context context, - Dispatcher dispatcher, NodeHealthCheckerService healthChecker) { - return new NodeStatusUpdaterImpl(context, dispatcher, - healthChecker, metrics) { - @Override - protected ResourceTracker getRMClient() { - final ResourceTrackerService rt = resourceManager - .getResourceTrackerService(); - final RecordFactory recordFactory = - RecordFactoryProvider.getRecordFactory(null); - - // For in-process communication without RPC - return new ResourceTracker() { - - @Override - public NodeHeartbeatResponse nodeHeartbeat( - NodeHeartbeatRequest request) throws YarnException, - IOException { - NodeHeartbeatResponse response = recordFactory.newRecordInstance( - NodeHeartbeatResponse.class); - try { - response = rt.nodeHeartbeat(request); - } catch (YarnException e) { - LOG.info("Exception in heartbeat from node " + - request.getNodeStatus().getNodeId(), e); - throw e; - } - return response; - } - - @Override - public RegisterNodeManagerResponse registerNodeManager( - RegisterNodeManagerRequest request) - throws YarnException, IOException { - RegisterNodeManagerResponse response = recordFactory. - newRecordInstance(RegisterNodeManagerResponse.class); - try { - response = rt.registerNodeManager(request); - } catch (YarnException e) { - LOG.info("Exception in node registration from " - + request.getNodeId().toString(), e); - throw e; - } - return response; - } - }; - }; - - @Override - protected void stopRMProxy() { - return; - } - }; - }; - } -} diff --git a/solr/contrib/morphlines-cell/README.txt b/solr/contrib/morphlines-cell/README.txt deleted file mode 100644 index a3a1ba971e2..00000000000 --- a/solr/contrib/morphlines-cell/README.txt +++ /dev/null @@ -1,6 +0,0 @@ -Apache Solr Morphlines-Cell - -*Experimental* - This contrib is currently subject to change in ways that may -break back compatibility. - -This contrib provides a variety of Kite Morphlines features for Solr Cell type functionality. \ No newline at end of file diff --git a/solr/contrib/morphlines-cell/build.xml b/solr/contrib/morphlines-cell/build.xml deleted file mode 100644 index 397472d3e00..00000000000 --- a/solr/contrib/morphlines-cell/build.xml +++ /dev/null @@ -1,144 +0,0 @@ - - - - - - - - Solr Cell Morphline commands. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-cell/ivy.xml b/solr/contrib/morphlines-cell/ivy.xml deleted file mode 100644 index c090f2174f9..00000000000 --- a/solr/contrib/morphlines-cell/ivy.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/SolrCellBuilder.java b/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/SolrCellBuilder.java deleted file mode 100644 index d4483a546b6..00000000000 --- a/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/SolrCellBuilder.java +++ /dev/null @@ -1,348 +0,0 @@ -/* - * 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.morphlines.cell; - -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.IllformedLocaleException; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; -import java.util.stream.Collectors; - -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ListMultimap; -import com.google.common.io.Closeables; -import com.typesafe.config.Config; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.common.SolrInputField; -import org.apache.solr.common.params.MultiMapSolrParams; -import org.apache.solr.common.params.SolrParams; -import org.apache.solr.common.util.SuppressForbidden; -import org.apache.solr.handler.extraction.ExtractingParams; -import org.apache.solr.handler.extraction.ExtractionDateUtil; -import org.apache.solr.handler.extraction.SolrContentHandler; -import org.apache.solr.handler.extraction.SolrContentHandlerFactory; -import org.apache.solr.morphlines.solr.SolrLocator; -import org.apache.solr.schema.IndexSchema; -import org.apache.tika.exception.TikaException; -import org.apache.tika.io.TikaInputStream; -import org.apache.tika.metadata.Metadata; -import org.apache.tika.mime.MediaType; -import org.apache.tika.parser.ParseContext; -import org.apache.tika.parser.Parser; -import org.apache.tika.sax.XHTMLContentHandler; -import org.apache.tika.sax.xpath.Matcher; -import org.apache.tika.sax.xpath.MatchingContentHandler; -import org.apache.tika.sax.xpath.XPathParser; -import org.kitesdk.morphline.api.Command; -import org.kitesdk.morphline.api.CommandBuilder; -import org.kitesdk.morphline.api.MorphlineCompilationException; -import org.kitesdk.morphline.api.MorphlineContext; -import org.kitesdk.morphline.api.MorphlineRuntimeException; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.Configs; -import org.kitesdk.morphline.base.Fields; -import org.kitesdk.morphline.stdio.AbstractParser; -import org.xml.sax.ContentHandler; -import org.xml.sax.SAXException; - -/** - * Command that pipes the first attachment of a record into one of the given Tika parsers, then maps - * the Tika output back to a record using SolrCell. - *

- * The Tika parser is chosen from the configurable list of parsers, depending on the MIME type - * specified in the input record. Typically, this requires an upstream DetectMimeTypeBuilder - * in a prior command. - */ -public final class SolrCellBuilder implements CommandBuilder { - - @Override - public Collection getNames() { - return Collections.singletonList("solrCell"); - } - - @Override - public Command build(Config config, Command parent, Command child, MorphlineContext context) { - return new SolrCell(this, config, parent, child, context); - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class SolrCell extends AbstractParser { - - private final IndexSchema schema; - private final List dateFormats; - private final String xpathExpr; - private final List parsers = new ArrayList<>(); - private final SolrContentHandlerFactory solrContentHandlerFactory; - private final Locale locale; - - private final SolrParams solrParams; - private final Map mediaTypeToParserMap; - - private static final XPathParser PARSER = new XPathParser("xhtml", XHTMLContentHandler.XHTML); - - public static final String ADDITIONAL_SUPPORTED_MIME_TYPES = "additionalSupportedMimeTypes"; - - public SolrCell(CommandBuilder builder, Config config, Command parent, Command child, MorphlineContext context) { - super(builder, config, parent, child, context); - - Config solrLocatorConfig = getConfigs().getConfig(config, "solrLocator"); - SolrLocator locator = new SolrLocator(solrLocatorConfig, context); - LOG.debug("solrLocator: {}", locator); - this.schema = Objects.requireNonNull(locator.getIndexSchema()); - if (LOG.isTraceEnabled()) { - LOG.trace("Solr schema: \n" + schema.getFields().entrySet().stream() - .sorted(Map.Entry.comparingByKey()).map(Map.Entry::getValue).map(Object::toString) - .collect(Collectors.joining("\n"))); - } - - ListMultimap cellParams = ArrayListMultimap.create(); - String uprefix = getConfigs().getString(config, ExtractingParams.UNKNOWN_FIELD_PREFIX, null); - if (uprefix != null) { - cellParams.put(ExtractingParams.UNKNOWN_FIELD_PREFIX, uprefix); - } - for (String capture : getConfigs().getStringList(config, ExtractingParams.CAPTURE_ELEMENTS, Collections.emptyList())) { - cellParams.put(ExtractingParams.CAPTURE_ELEMENTS, capture); - } - Config fmapConfig = getConfigs().getConfig(config, "fmap", null); - if (fmapConfig != null) { - for (Map.Entry entry : new Configs().getEntrySet(fmapConfig)) { - cellParams.put(ExtractingParams.MAP_PREFIX + entry.getKey(), entry.getValue().toString()); - } - } - String captureAttributes = getConfigs().getString(config, ExtractingParams.CAPTURE_ATTRIBUTES, null); - if (captureAttributes != null) { - cellParams.put(ExtractingParams.CAPTURE_ATTRIBUTES, captureAttributes); - } - String lowerNames = getConfigs().getString(config, ExtractingParams.LOWERNAMES, null); - if (lowerNames != null) { - cellParams.put(ExtractingParams.LOWERNAMES, lowerNames); - } - String defaultField = getConfigs().getString(config, ExtractingParams.DEFAULT_FIELD, null); - if (defaultField != null) { - cellParams.put(ExtractingParams.DEFAULT_FIELD, defaultField); - } - xpathExpr = getConfigs().getString(config, ExtractingParams.XPATH_EXPRESSION, null); - if (xpathExpr != null) { - cellParams.put(ExtractingParams.XPATH_EXPRESSION, xpathExpr); - } - - this.dateFormats = getConfigs().getStringList(config, "dateFormats", new ArrayList<>(ExtractionDateUtil.DEFAULT_DATE_FORMATS)); - - String handlerStr = getConfigs().getString(config, "solrContentHandlerFactory", TrimSolrContentHandlerFactory.class.getName()); - Class factoryClass; - try { - factoryClass = Class.forName(handlerStr).asSubclass(SolrContentHandlerFactory.class); - } catch (ClassNotFoundException cnfe) { - throw new MorphlineCompilationException("Could not find class " - + handlerStr + " to use for " + "solrContentHandlerFactory", config, cnfe); - } - this.solrContentHandlerFactory = getSolrContentHandlerFactory(factoryClass, dateFormats, config); - - this.locale = getLocale(getConfigs().getString(config, "locale", null)); - - this.mediaTypeToParserMap = new HashMap<>(); - //MimeTypes mimeTypes = MimeTypes.getDefaultMimeTypes(); // FIXME getMediaTypeRegistry.normalize() - - List parserConfigs = getConfigs().getConfigList(config, "parsers"); - for (Config parserConfig : parserConfigs) { - String parserClassName = getConfigs().getString(parserConfig, "parser"); - - Object obj; - try { - obj = Class.forName(parserClassName).newInstance(); - } catch (Throwable e) { - throw new MorphlineCompilationException("Cannot instantiate Tika parser: " + parserClassName, config, e); - } - if (!(obj instanceof Parser)) { - throw new MorphlineCompilationException("Tika parser " + obj.getClass().getName() - + " must be an instance of class " + Parser.class.getName(), config); - } - Parser parser = (Parser) obj; - this.parsers.add(parser); - - List mediaTypes = getConfigs().getStringList(parserConfig, SUPPORTED_MIME_TYPES, Collections.emptyList()); - for (String mediaTypeStr : mediaTypes) { - MediaType mediaType = parseMediaType(mediaTypeStr); - addSupportedMimeType(mediaTypeStr); - this.mediaTypeToParserMap.put(mediaType, parser); - } - - if (!parserConfig.hasPath(SUPPORTED_MIME_TYPES)) { - for (MediaType mediaType : parser.getSupportedTypes(new ParseContext())) { - mediaType = mediaType.getBaseType(); - addSupportedMimeType(mediaType.toString()); - this.mediaTypeToParserMap.put(mediaType, parser); - } - List extras = getConfigs().getStringList(parserConfig, ADDITIONAL_SUPPORTED_MIME_TYPES, Collections.emptyList()); - for (String mediaTypeStr : extras) { - MediaType mediaType = parseMediaType(mediaTypeStr); - addSupportedMimeType(mediaTypeStr); - this.mediaTypeToParserMap.put(mediaType, parser); - } - } - } - //LOG.info("mediaTypeToParserMap="+mediaTypeToParserMap); - - Map tmp = new HashMap<>(); - for (Map.Entry> entry : cellParams.asMap().entrySet()) { - tmp.put(entry.getKey(), entry.getValue().toArray(new String[entry.getValue().size()])); - } - this.solrParams = new MultiMapSolrParams(tmp); - validateArguments(); - } - - @Override - protected boolean doProcess(Record record, InputStream inputStream) { - Parser parser = detectParser(record); - if (parser == null) { - return false; - } - - ParseContext parseContext = new ParseContext(); - parseContext.set(Locale.class, locale); - - Metadata metadata = new Metadata(); - for (Entry entry : record.getFields().entries()) { - metadata.add(entry.getKey(), entry.getValue().toString()); - } - - SolrContentHandler handler = solrContentHandlerFactory.createSolrContentHandler(metadata, solrParams, schema); - try { - inputStream = TikaInputStream.get(inputStream); - - ContentHandler parsingHandler = handler; - - // String xpathExpr = "/xhtml:html/xhtml:body/xhtml:div/descendant:node()"; - if (xpathExpr != null) { - Matcher matcher = PARSER.parse(xpathExpr); - parsingHandler = new MatchingContentHandler(parsingHandler, matcher); - } - - try { - parser.parse(inputStream, parsingHandler, metadata, parseContext); - } catch (IOException | TikaException | SAXException e) { - throw new MorphlineRuntimeException("Cannot parse", e); - } - } finally { - if (inputStream != null) { - Closeables.closeQuietly(inputStream); - } - } - - SolrInputDocument doc = handler.newDocument(); - LOG.debug("solr doc: {}", doc); - Record outputRecord = toRecord(doc); - return getChild().process(outputRecord); - } - - private Parser detectParser(Record record) { - if (!hasAtLeastOneMimeType(record)) { - return null; - } - String mediaTypeStr = (String) record.getFirstValue(Fields.ATTACHMENT_MIME_TYPE); //ExtractingParams.STREAM_TYPE); - assert mediaTypeStr != null; - - MediaType mediaType = parseMediaType(mediaTypeStr).getBaseType(); - Parser parser = mediaTypeToParserMap.get(mediaType); // fast path - if (parser != null) { - return parser; - } - // wildcard matching - for (Map.Entry entry : mediaTypeToParserMap.entrySet()) { - if (isMediaTypeMatch(mediaType, entry.getKey())) { - return entry.getValue(); - } - } - if (LOG.isDebugEnabled()) { - LOG.debug("No supported MIME type parser found for " + Fields.ATTACHMENT_MIME_TYPE + "=" + mediaTypeStr); - } - return null; - } - - private boolean hasAtLeastOneMimeType(Record record) { - if (!record.getFields().containsKey(Fields.ATTACHMENT_MIME_TYPE)) { - LOG.debug("Command failed because of missing MIME type for record: {}", record); - return false; - } - return true; - } - - private MediaType parseMediaType(String mediaTypeStr) { - MediaType mediaType = MediaType.parse(mediaTypeStr.trim().toLowerCase(Locale.ROOT)); - return mediaType.getBaseType(); - }; - - /** Returns true if mediaType falls withing the given range (pattern), false otherwise */ - private boolean isMediaTypeMatch(MediaType mediaType, MediaType rangePattern) { - String WILDCARD = "*"; - String rangePatternType = rangePattern.getType(); - String rangePatternSubtype = rangePattern.getSubtype(); - return (rangePatternType.equals(WILDCARD) || rangePatternType.equals(mediaType.getType())) - && (rangePatternSubtype.equals(WILDCARD) || rangePatternSubtype.equals(mediaType.getSubtype())); - } - - private static SolrContentHandlerFactory getSolrContentHandlerFactory( - Class factoryClass, Collection dateFormats, Config config) { - try { - return factoryClass.getConstructor(Collection.class).newInstance(dateFormats); - } catch (NoSuchMethodException nsme) { - throw new MorphlineCompilationException("Unable to find valid constructor of type " - + factoryClass.getName() + " for creating SolrContentHandler", config, nsme); - } catch (Exception e) { - throw new MorphlineCompilationException("Unexpected exception when trying to create SolrContentHandlerFactory of type " - + factoryClass.getName(), config, e); - } - } - - private Record toRecord(SolrInputDocument doc) { - Record record = new Record(); - for (Entry entry : doc.entrySet()) { - record.getFields().putAll(entry.getKey(), entry.getValue().getValues()); - } - return record; - } - - @SuppressForbidden(reason = "Usage of outdated locale parsing with Locale#toString() because of backwards compatibility") - private Locale getLocale(String name) { - if (name == null) { - return Locale.ROOT; - } - for (Locale locale : Locale.getAvailableLocales()) { - if (locale.toString().equals(name)) { - return locale; - } - } - try { - return new Locale.Builder().setLanguageTag(name).build(); - } catch (IllformedLocaleException ex) { - throw new MorphlineCompilationException("Malformed / non-existent locale: " + name, getConfig(), ex); - } - } - } - -} diff --git a/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/StripNonCharSolrContentHandlerFactory.java b/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/StripNonCharSolrContentHandlerFactory.java deleted file mode 100644 index 81f49afd4e5..00000000000 --- a/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/StripNonCharSolrContentHandlerFactory.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.morphlines.cell; - -import java.util.Collection; - -import org.apache.solr.common.params.SolrParams; -import org.apache.solr.handler.extraction.SolrContentHandler; -import org.apache.solr.handler.extraction.SolrContentHandlerFactory; -import org.apache.solr.schema.IndexSchema; -import org.apache.solr.schema.SchemaField; -import org.apache.tika.metadata.Metadata; - -/** - * {@link SolrContentHandler} and associated factory that strips non-characters and trims on output. - * This prevents exceptions on parsing integer fields inside Solr server. - */ -public class StripNonCharSolrContentHandlerFactory extends SolrContentHandlerFactory { - - public StripNonCharSolrContentHandlerFactory(Collection dateFormats) { - super(dateFormats); - } - - @Override - public SolrContentHandler createSolrContentHandler(Metadata metadata, SolrParams params, IndexSchema schema) { - return new StripNonCharSolrContentHandler(metadata, params, schema, dateFormats); - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class StripNonCharSolrContentHandler extends SolrContentHandler { - - public StripNonCharSolrContentHandler(Metadata metadata, SolrParams params, IndexSchema schema, Collection dateFormats) { - super(metadata, params, schema, dateFormats); - } - - /** - * Strip all non-characters, which can cause SolrReducer problems if present. - * This is borrowed from Apache Nutch. - */ - private static String stripNonCharCodepoints(String input) { - StringBuilder stripped = new StringBuilder(input.length()); - char ch; - for (int i = 0; i < input.length(); i++) { - ch = input.charAt(i); - // Strip all non-characters http://unicode.org/cldr/utility/list-unicodeset.jsp?a=[:Noncharacter_Code_Point=True:] - // and non-printable control characters except tabulator, new line and carriage return - if (ch % 0x10000 != 0xffff && // 0xffff - 0x10ffff range step 0x10000 - ch % 0x10000 != 0xfffe && // 0xfffe - 0x10fffe range - (ch <= 0xfdd0 || ch >= 0xfdef) && // 0xfdd0 - 0xfdef - (ch > 0x1F || ch == 0x9 || ch == 0xa || ch == 0xd)) { - stripped.append(ch); - } - } - return stripped.toString(); - } - - @Override - protected String transformValue(String val, SchemaField schemaField) { - String ret = super.transformValue(val, schemaField).trim(); - ret = stripNonCharCodepoints(ret); - return ret; - } - } -} diff --git a/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/TrimSolrContentHandlerFactory.java b/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/TrimSolrContentHandlerFactory.java deleted file mode 100644 index 6e7df593ff8..00000000000 --- a/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/TrimSolrContentHandlerFactory.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.morphlines.cell; - -import java.util.Collection; - -import org.apache.solr.common.params.SolrParams; -import org.apache.solr.handler.extraction.SolrContentHandler; -import org.apache.solr.handler.extraction.SolrContentHandlerFactory; -import org.apache.solr.schema.IndexSchema; -import org.apache.solr.schema.SchemaField; -import org.apache.tika.metadata.Metadata; - -/** - * {@link SolrContentHandler} and associated factory that trims field values on output. - * This prevents exceptions on parsing integer fields inside Solr server. - */ -public class TrimSolrContentHandlerFactory extends SolrContentHandlerFactory { - - public TrimSolrContentHandlerFactory(Collection dateFormats) { - super(dateFormats); - } - - @Override - public SolrContentHandler createSolrContentHandler(Metadata metadata, SolrParams params, IndexSchema schema) { - return new TrimSolrContentHandler(metadata, params, schema, dateFormats); - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class TrimSolrContentHandler extends SolrContentHandler { - - public TrimSolrContentHandler(Metadata metadata, SolrParams params, IndexSchema schema, Collection dateFormats) { - super(metadata, params, schema, dateFormats); - } - - @Override - protected String transformValue(String val, SchemaField schemaField) { - return super.transformValue(val, schemaField).trim(); - } - } -} diff --git a/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/package-info.java b/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/package-info.java deleted file mode 100644 index 0f44a70488d..00000000000 --- a/solr/contrib/morphlines-cell/src/java/org/apache/solr/morphlines/cell/package-info.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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. - */ - -/** - * Morphlines Solr Cell related code. - */ -package org.apache.solr.morphlines.cell; - - - - diff --git a/solr/contrib/morphlines-cell/src/java/overview.html b/solr/contrib/morphlines-cell/src/java/overview.html deleted file mode 100644 index 3e25367d302..00000000000 --- a/solr/contrib/morphlines-cell/src/java/overview.html +++ /dev/null @@ -1,21 +0,0 @@ - - - -Apache Solr Search Server: Solr Cell Morphline Commands - - diff --git a/solr/contrib/morphlines-cell/src/test-files/README.txt b/solr/contrib/morphlines-cell/src/test-files/README.txt deleted file mode 100644 index 8905df29d30..00000000000 --- a/solr/contrib/morphlines-cell/src/test-files/README.txt +++ /dev/null @@ -1 +0,0 @@ -The test-files by this module are located in the morphlines-core module. diff --git a/solr/contrib/morphlines-cell/src/test/org/apache/solr/morphlines/cell/SolrCellMorphlineTest.java b/solr/contrib/morphlines-cell/src/test/org/apache/solr/morphlines/cell/SolrCellMorphlineTest.java deleted file mode 100644 index e0872b609ed..00000000000 --- a/solr/contrib/morphlines-cell/src/test/org/apache/solr/morphlines/cell/SolrCellMorphlineTest.java +++ /dev/null @@ -1,292 +0,0 @@ -/* - * 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.morphlines.cell; - -import java.io.File; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.Map; - -import org.apache.commons.io.FileUtils; -import org.apache.lucene.util.Constants; -import org.apache.solr.common.SolrInputDocument; -import org.apache.solr.common.params.MapSolrParams; -import org.apache.solr.handler.extraction.ExtractionDateUtil; -import org.apache.solr.handler.extraction.SolrContentHandler; -import org.apache.solr.morphlines.solr.AbstractSolrMorphlineTestBase; -import org.apache.solr.schema.IndexSchema; -import org.apache.tika.metadata.Metadata; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -public class SolrCellMorphlineTest extends AbstractSolrMorphlineTestBase { - - private Map expectedRecords = new HashMap<>(); - private Map> expectedRecordContents = new HashMap<>(); - - @BeforeClass - public static void beforeClass2() { - assumeFalse("FIXME: Morphlines currently has issues with Windows paths", Constants.WINDOWS); - } - - @Before - public void setUp() throws Exception { - super.setUp(); - - String path = RESOURCES_DIR + File.separator + "test-documents" + File.separator; - expectedRecords.put(path + "sample-statuses-20120906-141433.avro", 2); - expectedRecords.put(path + "sample-statuses-20120906-141433", 2); - expectedRecords.put(path + "sample-statuses-20120906-141433.gz", 2); - expectedRecords.put(path + "sample-statuses-20120906-141433.bz2", 2); - expectedRecords.put(path + "cars.csv", 6); - expectedRecords.put(path + "cars.csv.gz", 6); - expectedRecords.put(path + "cars.tar.gz", 4); - expectedRecords.put(path + "cars.tsv", 6); - expectedRecords.put(path + "cars.ssv", 6); - expectedRecords.put(path + "test-documents.7z", 9); - expectedRecords.put(path + "test-documents.cpio", 9); - expectedRecords.put(path + "test-documents.tar", 9); - expectedRecords.put(path + "test-documents.tbz2", 9); - expectedRecords.put(path + "test-documents.tgz", 9); - expectedRecords.put(path + "test-documents.zip", 9); - expectedRecords.put(path + "multiline-stacktrace.log", 4); - - { - Map record = new LinkedHashMap(); - record.put("ignored__attachment_mimetype", "image/jpeg"); - record.put("ignored_exif_isospeedratings", "400"); - record.put("ignored_meta_creation_date", "2009-08-11T09:09:45"); - record.put("ignored_tiff_model", "Canon EOS 40D"); - record.put("text", NON_EMPTY_FIELD); - expectedRecordContents.put("/testJPEG_EXIF.jpg", record); - expectedRecordContents.put("/testJPEG_EXIF.jpg.tar", record); - expectedRecordContents.put("/testJPEG_EXIF.jpg.tar.gz", record); - } - - { - String file = path + "testWORD_various.doc"; - Map record = new LinkedHashMap(); - record.put("ignored__attachment_mimetype", "application/msword"); - record.put("ignored_author", "Michael McCandless"); - record.put("ignored_creation_date", "2011-09-02T10:11:00Z"); - record.put("ignored_title", ""); - record.put("ignored_keywords", "Keyword1 Keyword2"); - record.put("ignored_subject", "Subject is here"); - record.put("text", NON_EMPTY_FIELD); - expectedRecordContents.put(file, record); - } - - { - String file = path + "testPDF.pdf"; - Map record = new LinkedHashMap(); - record.put("ignored__attachment_mimetype", "application/pdf"); - record.put("ignored_author", "Bertrand Delacrétaz"); - record.put("ignored_creation_date", "2007-09-15T09:02:31Z"); - record.put("ignored_title", "Apache Tika - Apache Tika"); - record.put("ignored_xmp_creatortool", "Firefox"); - record.put("text", NON_EMPTY_FIELD); - expectedRecordContents.put(file, record); - } - - { - String file = path + "email.eml"; - Map record = new LinkedHashMap(); - String name = "Patrick Foo "; - record.put("ignored__attachment_mimetype", "message/rfc822"); - record.put("ignored_author", name); - //record.put("ignored_content_length", "1068"); - record.put("ignored_creation_date", "2013-11-27T20:01:23Z"); - record.put("ignored_message_from", name); - record.put("ignored_message_to", name); - record.put("ignored_creator", name); - record.put("ignored_dc_creator", name); - record.put("ignored_dc_title", "Test EML"); - record.put("ignored_dcterms_created", "2013-11-27T20:01:23Z"); - record.put("ignored_meta_author", name); - record.put("ignored_meta_creation_date", "2013-11-27T20:01:23Z"); - record.put("ignored_subject", "Test EML"); - record.put("text", NON_EMPTY_FIELD); - expectedRecordContents.put(file, record); - } - - { - String file = path + "testEXCEL.xlsx"; - Map record = new LinkedHashMap(); - record.put("ignored__attachment_mimetype", "application/vnd.openxmlformats-officedocument.spreadsheetml.sheet"); - record.put("ignored_author", "Keith Bennett"); - record.put("ignored_creation_date", "2007-10-01T16:13:56Z"); - record.put("ignored_title", "Simple Excel document"); - record.put("text", NON_EMPTY_FIELD); - expectedRecordContents.put(file, record); - } - - FileUtils.copyFile(new File(RESOURCES_DIR + "/custom-mimetypes.xml"), new File(tempDir + "/custom-mimetypes.xml")); - } - - @Test - @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-6489") - public void testSolrCellJPGCompressed() throws Exception { - morphline = createMorphline("test-morphlines" + File.separator + "solrCellJPGCompressed"); - String path = RESOURCES_DIR + File.separator + "test-documents" + File.separator; - String[] files = new String[] { - path + "testJPEG_EXIF.jpg", - path + "testJPEG_EXIF.jpg.gz", - path + "testJPEG_EXIF.jpg.tar.gz", - //path + "jpeg2000.jp2", - }; - testDocumentTypesInternal(files, expectedRecords, expectedRecordContents); - } - - @Test - public void testSolrCellXML() throws Exception { - morphline = createMorphline("test-morphlines" + File.separator + "solrCellXML"); - String path = RESOURCES_DIR + File.separator + "test-documents" + File.separator; - String[] files = new String[] { - path + "testXML2.xml", - }; - testDocumentTypesInternal(files, expectedRecords, expectedRecordContents); - } - - @Test - @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-6489") - public void testSolrCellDocumentTypes() throws Exception { - AbstractSolrMorphlineTestBase.setupMorphline(tempDir, "test-morphlines/solrCellDocumentTypes", false); - - morphline = createMorphline(new File(tempDir).getAbsolutePath() + "/test-morphlines/solrCellDocumentTypes"); - String path = RESOURCES_DIR + File.separator + "test-documents" + File.separator; - String[] files = new String[] { - path + "testBMPfp.txt", - path + "boilerplate.html", - path + "NullHeader.docx", - path + "testWORD_various.doc", - path + "testPDF.pdf", - path + "testJPEG_EXIF.jpg", - path + "testJPEG_EXIF.jpg.gz", - path + "testJPEG_EXIF.jpg.tar.gz", - path + "testXML.xml", - path + "cars.csv", -// path + "cars.tsv", -// path + "cars.ssv", - path + "cars.csv.gz", - path + "cars.tar.gz", - path + "sample-statuses-20120906-141433.avro", - path + "sample-statuses-20120906-141433", - path + "sample-statuses-20120906-141433.gz", - path + "sample-statuses-20120906-141433.bz2", - path + "email.eml", - }; - testDocumentTypesInternal(files, expectedRecords, expectedRecordContents); - } - - @Test - @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-9220") - public void testSolrCellDocumentTypes2() throws Exception { - - AbstractSolrMorphlineTestBase.setupMorphline(tempDir, "test-morphlines/solrCellDocumentTypes", false); - - morphline = createMorphline(new File(tempDir).getAbsolutePath() + "/test-morphlines/solrCellDocumentTypes"); - String path = RESOURCES_DIR + File.separator + "test-documents" + File.separator; - String[] files = new String[] { - path + "testPPT_various.ppt", - path + "testPPT_various.pptx", - path + "testEXCEL.xlsx", - path + "testEXCEL.xls", - path + "testPages.pages", - //path + "testNumbers.numbers", - //path + "testKeynote.key", - - path + "testRTFVarious.rtf", - path + "complex.mbox", - path + "test-outlook.msg", - path + "testEMLX.emlx", - path + "testRFC822", - path + "rsstest.rss", -// path + "testDITA.dita", - - path + "testMP3i18n.mp3", - path + "testAIFF.aif", - path + "testFLAC.flac", -// path + "testFLAC.oga", -// path + "testVORBIS.ogg", - path + "testMP4.m4a", - path + "testWAV.wav", -// path + "testWMA.wma", - - path + "testFLV.flv", -// path + "testWMV.wmv", - - path + "testBMP.bmp", - path + "testPNG.png", - path + "testPSD.psd", - path + "testSVG.svg", - path + "testTIFF.tif", - -// path + "test-documents.7z", -// path + "test-documents.cpio", -// path + "test-documents.tar", -// path + "test-documents.tbz2", -// path + "test-documents.tgz", -// path + "test-documents.zip", -// path + "test-zip-of-zip.zip", -// path + "testJAR.jar", - -// path + "testKML.kml", -// path + "testRDF.rdf", - path + "testVISIO.vsd", -// path + "testWAR.war", -// path + "testWindows-x86-32.exe", -// path + "testWINMAIL.dat", -// path + "testWMF.wmf", - }; - testDocumentTypesInternal(files, expectedRecords, expectedRecordContents); - } - - /** - * Test that the ContentHandler properly strips the illegal characters - */ - @Test - public void testTransformValue() { - String fieldName = "user_name"; - assertFalse("foobar".equals(getFoobarWithNonChars())); - - Metadata metadata = new Metadata(); - // load illegal char string into a metadata field and generate a new document, - // which will cause the ContentHandler to be invoked. - metadata.set(fieldName, getFoobarWithNonChars()); - StripNonCharSolrContentHandlerFactory contentHandlerFactory = - new StripNonCharSolrContentHandlerFactory(ExtractionDateUtil.DEFAULT_DATE_FORMATS); - IndexSchema schema = h.getCore().getLatestSchema(); - SolrContentHandler contentHandler = - contentHandlerFactory.createSolrContentHandler(metadata, new MapSolrParams(new HashMap()), schema); - SolrInputDocument doc = contentHandler.newDocument(); - String foobar = doc.getFieldValue(fieldName).toString(); - assertTrue("foobar".equals(foobar)); - } - - /** - * Returns string "foobar" with illegal characters interspersed. - */ - private String getFoobarWithNonChars() { - char illegalChar = '\uffff'; - StringBuilder builder = new StringBuilder(); - builder.append(illegalChar).append(illegalChar).append("foo").append(illegalChar) - .append(illegalChar).append("bar").append(illegalChar).append(illegalChar); - return builder.toString(); - } - -} diff --git a/solr/contrib/morphlines-core/README.txt b/solr/contrib/morphlines-core/README.txt deleted file mode 100644 index 0efa4674258..00000000000 --- a/solr/contrib/morphlines-core/README.txt +++ /dev/null @@ -1,6 +0,0 @@ -Apache Solr Morphlines-Core - -*Experimental* - This contrib is currently subject to change in ways that may -break back compatibility. - -This contrib provides a variety of Kite Morphlines features for Solr. \ No newline at end of file diff --git a/solr/contrib/morphlines-core/build.xml b/solr/contrib/morphlines-core/build.xml deleted file mode 100644 index 2cf6261669b..00000000000 --- a/solr/contrib/morphlines-core/build.xml +++ /dev/null @@ -1,105 +0,0 @@ - - - - - - - - Solr Morphlines commands. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/ivy.xml b/solr/contrib/morphlines-core/ivy.xml deleted file mode 100644 index ad47aec4015..00000000000 --- a/solr/contrib/morphlines-core/ivy.xml +++ /dev/null @@ -1,128 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/DocumentLoader.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/DocumentLoader.java deleted file mode 100644 index f3030247065..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/DocumentLoader.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.io.IOException; - -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.response.SolrPingResponse; -import org.apache.solr.client.solrj.response.UpdateResponse; -import org.apache.solr.common.SolrInputDocument; - -/** - * A vehicle to load a list of Solr documents into some kind of destination, - * such as a SolrServer or MapReduce RecordWriter. - */ -public interface DocumentLoader { - - /** Begins a transaction */ - public void beginTransaction() throws IOException, SolrServerException; - - /** Loads the given document into the destination */ - public void load(SolrInputDocument doc) throws IOException, SolrServerException; - - /** - * Sends any outstanding documents to the destination and waits for a positive - * or negative ack (i.e. exception). Depending on the outcome the caller - * should then commit or rollback the current flume transaction - * correspondingly. - * - * @throws IOException - * If there is a low-level I/O error. - */ - public void commitTransaction() throws IOException, SolrServerException; - - /** - * Performs a rollback of all non-committed documents pending. - *

- * Note that this is not a true rollback as in databases. Content you have - * previously added may have already been committed due to autoCommit, buffer - * full, other client performing a commit etc. So this is only a best-effort - * rollback. - * - * @throws IOException - * If there is a low-level I/O error. - */ - public UpdateResponse rollbackTransaction() throws IOException, SolrServerException; - - /** Releases allocated resources */ - public void shutdown() throws IOException, SolrServerException; - - /** - * Issues a ping request to check if the server is alive - * - * @throws IOException - * If there is a low-level I/O error. - */ - public SolrPingResponse ping() throws IOException, SolrServerException; - -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/FileUtils.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/FileUtils.java deleted file mode 100644 index e979d37bbc7..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/FileUtils.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.file.Files; - - -class FileUtils { - - //----------------------------------------------------------------------- - /** - * Deletes a directory recursively. - * - * @param directory directory to delete - * @throws IOException in case deletion is unsuccessful - */ - public static void deleteDirectory(File directory) throws IOException { - if (!directory.exists()) { - return; - } - - if (!isSymlink(directory)) { - cleanDirectory(directory); - } - - Files.delete(directory.toPath()); - } - - /** - * Determines whether the specified file is a Symbolic Link rather than an actual file. - *

- * Will not return true if there is a Symbolic Link anywhere in the path, - * only if the specific file is. - * - * @param file the file to check - * @return true if the file is a Symbolic Link - * @throws IOException if an IO error occurs while checking the file - * @since Commons IO 2.0 - */ - public static boolean isSymlink(File file) throws IOException { - if (file == null) { - throw new NullPointerException("File must not be null"); - } -// if (FilenameUtils.isSystemWindows()) { - if (File.separatorChar == '\\') { - return false; - } - File fileInCanonicalDir = null; - if (file.getParent() == null) { - fileInCanonicalDir = file; - } else { - File canonicalDir = file.getParentFile().getCanonicalFile(); - fileInCanonicalDir = new File(canonicalDir, file.getName()); - } - - if (fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile())) { - return false; - } else { - return true; - } - } - - /** - * Cleans a directory without deleting it. - * - * @param directory directory to clean - * @throws IOException in case cleaning is unsuccessful - */ - public static void cleanDirectory(File directory) throws IOException { - if (!directory.exists()) { - String message = directory + " does not exist"; - throw new IllegalArgumentException(message); - } - - if (!directory.isDirectory()) { - String message = directory + " is not a directory"; - throw new IllegalArgumentException(message); - } - - File[] files = directory.listFiles(); - if (files == null) { // null if security restricted - throw new IOException("Failed to list contents of " + directory); - } - - IOException exception = null; - for (File file : files) { - try { - forceDelete(file); - } catch (IOException ioe) { - exception = ioe; - } - } - - if (null != exception) { - throw exception; - } - } - - //----------------------------------------------------------------------- - /** - * Deletes a file. If file is a directory, delete it and all sub-directories. - *

- * The difference between File.delete() and this method are: - *

    - *
  • A directory to be deleted does not have to be empty.
  • - *
  • You get exceptions when a file or directory cannot be deleted. - * (java.io.File methods returns a boolean)
  • - *
- * - * @param file file or directory to delete, must not be null - * @throws NullPointerException if the directory is null - * @throws FileNotFoundException if the file was not found - * @throws IOException in case deletion is unsuccessful - */ - public static void forceDelete(File file) throws IOException { - if (file.isDirectory()) { - deleteDirectory(file); - } else { - Files.delete(file.toPath()); - } - } - -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/GenerateSolrSequenceKeyBuilder.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/GenerateSolrSequenceKeyBuilder.java deleted file mode 100644 index be002efe8f1..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/GenerateSolrSequenceKeyBuilder.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.security.SecureRandom; -import java.util.Arrays; -import java.util.Collection; -import java.util.Random; - -import org.apache.solr.schema.IndexSchema; -import org.apache.solr.schema.SchemaField; -import org.kitesdk.morphline.api.Command; -import org.kitesdk.morphline.api.CommandBuilder; -import org.kitesdk.morphline.api.MorphlineContext; -import org.kitesdk.morphline.api.MorphlineRuntimeException; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.AbstractCommand; -import org.kitesdk.morphline.base.Fields; -import org.kitesdk.morphline.base.Notifications; - -import com.typesafe.config.Config; - -/** - * A command that assigns a record unique key that is the concatenation of the given - * baseIdField record field, followed by a running count of the record number within - * the current session. The count is reset to zero whenever a "startSession" notification is - * received. - *

- * For example, assume a CSV file containing multiple records but no unique ids, and the - * baseIdField field is the filesystem path of the file. Now this command can be used - * to assign the following record values to Solr's unique key field: - * $path#0, $path#1, ... $path#N. - *

- * The name of the unique key field is fetched from Solr's schema.xml file, as directed by the - * solrLocator configuration parameter. - */ -public final class GenerateSolrSequenceKeyBuilder implements CommandBuilder { - - @Override - public Collection getNames() { - return Arrays.asList( - "generateSolrSequenceKey", - "sanitizeUniqueSolrKey" // old name (retained for backwards compatibility) - ); - } - - @Override - public Command build(Config config, Command parent, Command child, MorphlineContext context) { - return new GenerateSolrSequenceKey(this, config, parent, child, context); - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class GenerateSolrSequenceKey extends AbstractCommand { - - private final boolean preserveExisting; - private final String baseIdFieldName; - private final String uniqueKeyName; - private long recordCounter = 0; - - private final String idPrefix; // for load testing only; enables adding same document many times with a different unique key - private final Random randomIdPrefix; // for load testing only; enables adding same document many times with a different unique key - - public GenerateSolrSequenceKey(CommandBuilder builder, Config config, Command parent, Command child, MorphlineContext context) { - super(builder, config, parent, child, context); - this.baseIdFieldName = getConfigs().getString(config, "baseIdField", Fields.BASE_ID); - this.preserveExisting = getConfigs().getBoolean(config, "preserveExisting", true); - - Config solrLocatorConfig = getConfigs().getConfig(config, "solrLocator"); - SolrLocator locator = new SolrLocator(solrLocatorConfig, context); - LOG.debug("solrLocator: {}", locator); - IndexSchema schema = locator.getIndexSchema(); - SchemaField uniqueKey = schema.getUniqueKeyField(); - uniqueKeyName = uniqueKey == null ? null : uniqueKey.getName(); - - String tmpIdPrefix = getConfigs().getString(config, "idPrefix", null); // for load testing only - Random tmpRandomIdPrefx = null; - if ("random".equals(tmpIdPrefix)) { // for load testing only - tmpRandomIdPrefx = new Random(new SecureRandom().nextLong()); - tmpIdPrefix = null; - } - idPrefix = tmpIdPrefix; - randomIdPrefix = tmpRandomIdPrefx; - validateArguments(); - } - - @Override - protected boolean doProcess(Record doc) { - long num = recordCounter++; - // LOG.debug("record #{} id before sanitizing doc: {}", num, doc); - if (uniqueKeyName == null || (preserveExisting && doc.getFields().containsKey(uniqueKeyName))) { - ; // we must preserve the existing id - } else { - Object baseId = doc.getFirstValue(baseIdFieldName); - if (baseId == null) { - throw new MorphlineRuntimeException("Record field " + baseIdFieldName - + " must not be null as it is needed as a basis for a unique key for solr doc: " + doc); - } - doc.replaceValues(uniqueKeyName, baseId.toString() + "#" + num); - } - - // for load testing only; enables adding same document many times with a different unique key - if (idPrefix != null) { - String id = doc.getFirstValue(uniqueKeyName).toString(); - id = idPrefix + id; - doc.replaceValues(uniqueKeyName, id); - } else if (randomIdPrefix != null) { - String id = doc.getFirstValue(uniqueKeyName).toString(); - id = String.valueOf(Math.abs(randomIdPrefix.nextInt())) + "#" + id; - doc.replaceValues(uniqueKeyName, id); - } - - LOG.debug("record #{} unique key sanitized to this: {}", num, doc); - - return super.doProcess(doc); - } - - @Override - protected void doNotify(Record notification) { - if (Notifications.containsLifecycleEvent(notification, Notifications.LifecycleEvent.START_SESSION)) { - recordCounter = 0; // reset - } - super.doNotify(notification); - } - - } -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/LoadSolrBuilder.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/LoadSolrBuilder.java deleted file mode 100644 index a3af6e18c5f..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/LoadSolrBuilder.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.common.SolrInputDocument; - -import org.kitesdk.morphline.api.Command; -import org.kitesdk.morphline.api.CommandBuilder; -import org.kitesdk.morphline.api.MorphlineContext; -import org.kitesdk.morphline.api.MorphlineRuntimeException; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.AbstractCommand; -import org.kitesdk.morphline.base.Configs; -import org.kitesdk.morphline.base.Metrics; -import org.kitesdk.morphline.base.Notifications; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.codahale.metrics.Timer; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; - -/** - * A command that loads a record into a SolrServer or MapReduce SolrOutputFormat. - */ -public final class LoadSolrBuilder implements CommandBuilder { - - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private static final AtomicBoolean WARNED_ABOUT_INDEX_TIME_BOOSTS = new AtomicBoolean(); - - @Override - public Collection getNames() { - return Collections.singletonList("loadSolr"); - } - - @Override - public Command build(Config config, Command parent, Command child, MorphlineContext context) { - return new LoadSolr(this, config, parent, child, context); - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class LoadSolr extends AbstractCommand { - - private final DocumentLoader loader; - private final Timer elapsedTime; - - public LoadSolr(CommandBuilder builder, Config config, Command parent, Command child, MorphlineContext context) { - super(builder, config, parent, child, context); - Config solrLocatorConfig = getConfigs().getConfig(config, "solrLocator"); - SolrLocator locator = new SolrLocator(solrLocatorConfig, context); - LOG.debug("solrLocator: {}", locator); - this.loader = locator.getLoader(); - Config boostsConfig = getConfigs().getConfig(config, "boosts", ConfigFactory.empty()); - if (new Configs().getEntrySet(boostsConfig).isEmpty() == false) { - String message = "Ignoring field boosts: as index-time boosts are not supported anymore"; - if (WARNED_ABOUT_INDEX_TIME_BOOSTS.compareAndSet(false, true)) { - log.warn(message); - } else { - log.debug(message); - } - } - validateArguments(); - this.elapsedTime = getTimer(Metrics.ELAPSED_TIME); - } - - @Override - protected void doNotify(Record notification) { - for (Object event : Notifications.getLifecycleEvents(notification)) { - if (event == Notifications.LifecycleEvent.BEGIN_TRANSACTION) { - try { - loader.beginTransaction(); - } catch (SolrServerException | IOException e) { - throw new MorphlineRuntimeException(e); - } - } else if (event == Notifications.LifecycleEvent.COMMIT_TRANSACTION) { - try { - loader.commitTransaction(); - } catch (SolrServerException | IOException e) { - throw new MorphlineRuntimeException(e); - } - } - else if (event == Notifications.LifecycleEvent.ROLLBACK_TRANSACTION) { - try { - loader.rollbackTransaction(); - } catch (SolrServerException | IOException e) { - throw new MorphlineRuntimeException(e); - } - } - else if (event == Notifications.LifecycleEvent.SHUTDOWN) { - try { - loader.shutdown(); - } catch (SolrServerException | IOException e) { - throw new MorphlineRuntimeException(e); - } - } - } - super.doNotify(notification); - } - - @Override - protected boolean doProcess(Record record) { - Timer.Context timerContext = elapsedTime.time(); - SolrInputDocument doc = convert(record); - try { - loader.load(doc); - } catch (IOException | SolrServerException e) { - throw new MorphlineRuntimeException(e); - } finally { - timerContext.stop(); - } - - // pass record to next command in chain: - return super.doProcess(record); - } - - private SolrInputDocument convert(Record record) { - Map> map = record.getFields().asMap(); - SolrInputDocument doc = new SolrInputDocument(new HashMap(2 * map.size())); - for (Map.Entry> entry : map.entrySet()) { - String key = entry.getKey(); - doc.setField(key, entry.getValue()); - } - return doc; - } - - } -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SafeConcurrentUpdateSolrClient.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SafeConcurrentUpdateSolrClient.java deleted file mode 100644 index a5fb929c92d..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SafeConcurrentUpdateSolrClient.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.lang.invoke.MethodHandles; - -import org.apache.http.client.HttpClient; -import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * ConcurrentUpdateSolrServer that propagates exceptions up to the submitter of - * requests on blockUntilFinished() - */ -final class SafeConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient { - - private Throwable currentException = null; - private final Object myLock = new Object(); - - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public SafeConcurrentUpdateSolrClient(String solrServerUrl, int queueSize, int threadCount) { - this(solrServerUrl, null, queueSize, threadCount); - } - - public SafeConcurrentUpdateSolrClient(String solrServerUrl, HttpClient client, int queueSize, int threadCount) { - super(solrServerUrl, client, queueSize, threadCount, null, false); - } - - @Override - public void handleError(Throwable ex) { - assert ex != null; - synchronized (myLock) { - currentException = ex; - } - LOGGER.error("handleError", ex); - } - - @Override - public void blockUntilFinished() { - super.blockUntilFinished(); - synchronized (myLock) { - if (currentException != null) { - throw new RuntimeException(currentException); - } - } - } - - public void clearException() { - synchronized (myLock) { - currentException = null; - } - } - -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SanitizeUnknownSolrFieldsBuilder.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SanitizeUnknownSolrFieldsBuilder.java deleted file mode 100644 index 9ede7145b40..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SanitizeUnknownSolrFieldsBuilder.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import java.util.Objects; -import java.util.stream.Collectors; - -import org.apache.solr.schema.IndexSchema; - -import org.kitesdk.morphline.api.Command; -import org.kitesdk.morphline.api.CommandBuilder; -import org.kitesdk.morphline.api.MorphlineContext; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.AbstractCommand; -import com.typesafe.config.Config; - -/** - * Command that sanitizes record fields that are unknown to Solr schema.xml by either deleting them - * (renameToPrefix is absent or a zero length string), or by moving them to a field prefixed with - * the given renameToPrefix (e.g. renameToPrefix = "ignored_" to use typical dynamic Solr fields). - *

- * Recall that Solr throws an exception on any attempt to load a document that contains a field that - * isn't specified in schema.xml. - */ -public final class SanitizeUnknownSolrFieldsBuilder implements CommandBuilder { - - @Override - public Collection getNames() { - return Collections.singletonList("sanitizeUnknownSolrFields"); - } - - @Override - public Command build(Config config, Command parent, Command child, MorphlineContext context) { - return new SanitizeUnknownSolrFields(this, config, parent, child, context); - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class SanitizeUnknownSolrFields extends AbstractCommand { - - private final IndexSchema schema; - private final String renameToPrefix; - - public SanitizeUnknownSolrFields(CommandBuilder builder, Config config, Command parent, Command child, MorphlineContext context) { - super(builder, config, parent, child, context); - - Config solrLocatorConfig = getConfigs().getConfig(config, "solrLocator"); - SolrLocator locator = new SolrLocator(solrLocatorConfig, context); - LOG.debug("solrLocator: {}", locator); - this.schema = Objects.requireNonNull(locator.getIndexSchema()); - if (LOG.isTraceEnabled()) { - LOG.trace("Solr schema: \n" + - schema.getFields().entrySet().stream().sorted(Map.Entry.comparingByKey()) - .map(Map.Entry::getValue).map(Object::toString).collect(Collectors.joining("\n")) - ); - } - - String str = getConfigs().getString(config, "renameToPrefix", "").trim(); - this.renameToPrefix = str.length() > 0 ? str : null; - validateArguments(); - } - - @Override - protected boolean doProcess(Record record) { - Collection entries = new ArrayList(record.getFields().asMap().entrySet()); - for (Map.Entry> entry : entries) { - String key = entry.getKey(); - if (schema.getFieldOrNull(key) == null) { - LOG.debug("Sanitizing unknown Solr field: {}", key); - Collection values = entry.getValue(); - if (renameToPrefix != null) { - record.getFields().putAll(renameToPrefix + key, values); - } - values.clear(); // implicitly removes key from record - } - } - return super.doProcess(record); - } - - } -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrClientDocumentLoader.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrClientDocumentLoader.java deleted file mode 100644 index ef9ea8424bd..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrClientDocumentLoader.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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.morphlines.solr; - -import org.apache.solr.client.solrj.SolrClient; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.impl.CloudSolrClient; -import org.apache.solr.client.solrj.impl.ConcurrentUpdateSolrClient; -import org.apache.solr.client.solrj.response.SolrPingResponse; -import org.apache.solr.client.solrj.response.UpdateResponse; -import org.apache.solr.common.SolrInputDocument; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.List; - -/** - * A vehicle to load a list of Solr documents into a local or remote {@link org.apache.solr.client.solrj.SolrClient}. - */ -public class SolrClientDocumentLoader implements DocumentLoader { - - private final SolrClient client; // proxy to local or remote solr server - private long numLoadedDocs = 0; // number of documents loaded in the current transaction - private final int batchSize; - private final List batch = new ArrayList(); - - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public SolrClientDocumentLoader(SolrClient client, int batchSize) { - if (client == null) { - throw new IllegalArgumentException("solr server must not be null"); - } - this.client = client; - if (batchSize <= 0) { - throw new IllegalArgumentException("batchSize must be a positive number: " + batchSize); - } - this.batchSize = batchSize; - } - - @Override - public void beginTransaction() { - LOGGER.trace("beginTransaction"); - batch.clear(); - numLoadedDocs = 0; - if (client instanceof SafeConcurrentUpdateSolrClient) { - ((SafeConcurrentUpdateSolrClient) client).clearException(); - } - } - - @Override - public void load(SolrInputDocument doc) throws IOException, SolrServerException { - LOGGER.trace("load doc: {}", doc); - batch.add(doc); - if (batch.size() >= batchSize) { - loadBatch(); - } - } - - @Override - public void commitTransaction() throws SolrServerException, IOException { - LOGGER.trace("commitTransaction"); - if (batch.size() > 0) { - loadBatch(); - } - if (numLoadedDocs > 0) { - if (client instanceof ConcurrentUpdateSolrClient) { - ((ConcurrentUpdateSolrClient) client).blockUntilFinished(); - } - } - } - - private void loadBatch() throws SolrServerException, IOException { - numLoadedDocs += batch.size(); - try { - UpdateResponse rsp = client.add(batch); - } finally { - batch.clear(); - } - } - - @Override - public UpdateResponse rollbackTransaction() throws SolrServerException, IOException { - LOGGER.trace("rollback"); - if (!(client instanceof CloudSolrClient)) { - return client.rollback(); - } else { - return new UpdateResponse(); - } - } - - @Override - public void shutdown() throws IOException { - LOGGER.trace("shutdown"); - client.close(); - } - - @Override - public SolrPingResponse ping() throws SolrServerException, IOException { - LOGGER.trace("ping"); - return client.ping(); - } - - public SolrClient getSolrClient() { - return client; - } - -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrLocator.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrLocator.java deleted file mode 100644 index 1d177a6778c..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrLocator.java +++ /dev/null @@ -1,254 +0,0 @@ -/* - * 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.morphlines.solr; - -import javax.xml.parsers.ParserConfigurationException; -import java.io.File; -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.nio.file.Paths; -import java.util.Objects; - -import com.google.common.io.Files; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; -import com.typesafe.config.ConfigRenderOptions; -import com.typesafe.config.ConfigUtil; -import org.apache.solr.client.solrj.SolrClient; -import org.apache.solr.client.solrj.impl.CloudSolrClient; -import org.apache.solr.client.solrj.impl.CloudSolrClient.Builder; -import org.apache.solr.common.cloud.SolrZkClient; -import org.apache.solr.core.SolrConfig; -import org.apache.solr.core.SolrResourceLoader; -import org.apache.solr.schema.IndexSchema; -import org.apache.solr.util.SystemIdResolver; -import org.apache.zookeeper.KeeperException; -import org.kitesdk.morphline.api.MorphlineCompilationException; -import org.kitesdk.morphline.api.MorphlineContext; -import org.kitesdk.morphline.api.MorphlineRuntimeException; -import org.kitesdk.morphline.base.Configs; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.xml.sax.InputSource; -import org.xml.sax.SAXException; - -/** - * Set of configuration parameters that identify the location and schema of a Solr server or - * SolrCloud; Based on this information this class can return the schema and a corresponding - * {@link DocumentLoader}. - */ -public class SolrLocator { - - private Config config; - private MorphlineContext context; - private String collectionName; - private String zkHost; - private String solrUrl; - private String solrHomeDir; - private int batchSize = 1000; - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - protected SolrLocator(MorphlineContext context) { - this.context = Objects.requireNonNull(context); - } - - public SolrLocator(Config config, MorphlineContext context) { - this(context); - this.config = config; - Configs configs = new Configs(); - collectionName = configs.getString(config, "collection", null); - zkHost = configs.getString(config, "zkHost", null); - solrHomeDir = configs.getString(config, "solrHomeDir", null); - solrUrl = configs.getString(config, "solrUrl", null); - batchSize = configs.getInt(config, "batchSize", batchSize); - LOG.trace("Constructed solrLocator: {}", this); - configs.validateArguments(config); - } - - public DocumentLoader getLoader() { - if (context instanceof SolrMorphlineContext) { - DocumentLoader loader = ((SolrMorphlineContext)context).getDocumentLoader(); - if (loader != null) { - return loader; - } - } - - if (zkHost != null && zkHost.length() > 0) { - if (collectionName == null || collectionName.length() == 0) { - throw new MorphlineCompilationException("Parameter 'zkHost' requires that you also pass parameter 'collection'", config); - } - CloudSolrClient cloudSolrClient = new Builder() - .withZkHost(zkHost) - .build(); - cloudSolrClient.setDefaultCollection(collectionName); - cloudSolrClient.connect(); - return new SolrClientDocumentLoader(cloudSolrClient, batchSize); - } else { - if (solrUrl == null || solrUrl.length() == 0) { - throw new MorphlineCompilationException("Missing parameter 'solrUrl'", config); - } - int solrServerNumThreads = 2; - int solrServerQueueLength = solrServerNumThreads; - SolrClient server = new SafeConcurrentUpdateSolrClient(solrUrl, solrServerQueueLength, solrServerNumThreads); - // SolrServer server = new HttpSolrServer(solrServerUrl); - // SolrServer server = new ConcurrentUpdateSolrServer(solrServerUrl, solrServerQueueLength, solrServerNumThreads); - // server.setParser(new XMLResponseParser()); // binary parser is used by default - return new SolrClientDocumentLoader(server, batchSize); - } - } - - public IndexSchema getIndexSchema() { - if (context instanceof SolrMorphlineContext) { - IndexSchema schema = ((SolrMorphlineContext)context).getIndexSchema(); - if (schema != null) { - validateSchema(schema); - return schema; - } - } - - File downloadedSolrHomeDir = null; - try { - // If solrHomeDir isn't defined and zkHost and collectionName are defined - // then download schema.xml and solrconfig.xml, etc from zk and use that as solrHomeDir - String mySolrHomeDir = solrHomeDir; - if (solrHomeDir == null || solrHomeDir.length() == 0) { - if (zkHost == null || zkHost.length() == 0) { - // TODO: implement download from solrUrl if specified - throw new MorphlineCompilationException( - "Downloading a Solr schema requires either parameter 'solrHomeDir' or parameters 'zkHost' and 'collection'", - config); - } - if (collectionName == null || collectionName.length() == 0) { - throw new MorphlineCompilationException( - "Parameter 'zkHost' requires that you also pass parameter 'collection'", config); - } - ZooKeeperDownloader zki = new ZooKeeperDownloader(); - SolrZkClient zkClient = zki.getZkClient(zkHost); - try { - String configName = zki.readConfigName(zkClient, collectionName); - downloadedSolrHomeDir = Files.createTempDir(); - downloadedSolrHomeDir = zki.downloadConfigDir(zkClient, configName, downloadedSolrHomeDir); - mySolrHomeDir = downloadedSolrHomeDir.getAbsolutePath(); - } catch (KeeperException | InterruptedException | IOException e) { - throw new MorphlineCompilationException("Cannot download schema.xml from ZooKeeper", config, e); - } finally { - zkClient.close(); - } - } - - LOG.debug("SolrLocator loading IndexSchema from dir {}", mySolrHomeDir); - try { - SolrResourceLoader loader = new SolrResourceLoader(Paths.get(mySolrHomeDir)); - SolrConfig solrConfig = new SolrConfig(loader, "solrconfig.xml", null); - InputSource is = new InputSource(loader.openSchema("schema.xml")); - is.setSystemId(SystemIdResolver.createSystemIdFromResourceName("schema.xml")); - - IndexSchema schema = new IndexSchema(solrConfig, "schema.xml", is); - validateSchema(schema); - return schema; - } catch (ParserConfigurationException | IOException | SAXException e) { - throw new MorphlineRuntimeException(e); - } - } finally { - if (downloadedSolrHomeDir != null) { - try { - FileUtils.deleteDirectory(downloadedSolrHomeDir); - } catch (IOException e) { - LOG.warn("Cannot delete tmp directory", e); - } - } - } - } - - private void validateSchema(IndexSchema schema) { - if (schema.getUniqueKeyField() == null) { - throw new MorphlineCompilationException("Solr schema.xml is missing unique key field", config); - } - if (!schema.getUniqueKeyField().isRequired()) { - throw new MorphlineCompilationException("Solr schema.xml must contain a required unique key field", config); - } - } - - @Override - public String toString() { - return toConfig(null).root().render(ConfigRenderOptions.concise()); - } - - public Config toConfig(String key) { - String json = ""; - if (key != null) { - json = toJson(key) + " : "; - } - json += - "{" + - " collection : " + toJson(collectionName) + ", " + - " zkHost : " + toJson(zkHost) + ", " + - " solrUrl : " + toJson(solrUrl) + ", " + - " solrHomeDir : " + toJson(solrHomeDir) + ", " + - " batchSize : " + toJson(batchSize) + " " + - "}"; - return ConfigFactory.parseString(json); - } - - private String toJson(Object key) { - String str = key == null ? "" : key.toString(); - str = ConfigUtil.quoteString(str); - return str; - } - - public String getCollectionName() { - return this.collectionName; - } - - public void setCollectionName(String collectionName) { - this.collectionName = collectionName; - } - - public String getZkHost() { - return this.zkHost; - } - - public void setZkHost(String zkHost) { - this.zkHost = zkHost; - } - - public String getSolrHomeDir() { - return this.solrHomeDir; - } - - public void setSolrHomeDir(String solrHomeDir) { - this.solrHomeDir = solrHomeDir; - } - - public String getServerUrl() { - return this.solrUrl; - } - - public void setServerUrl(String solrUrl) { - this.solrUrl = solrUrl; - } - - public int getBatchSize() { - return this.batchSize; - } - - public void setBatchSize(int batchSize) { - this.batchSize = batchSize; - } - -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrMorphlineContext.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrMorphlineContext.java deleted file mode 100644 index b3cd3019506..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/SolrMorphlineContext.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.morphlines.solr; - -import org.apache.solr.schema.IndexSchema; - -import org.kitesdk.morphline.api.MorphlineContext; - -/** - * A context that is specific to Solr. - */ -public class SolrMorphlineContext extends MorphlineContext { - - private DocumentLoader loader; - private IndexSchema schema; - - /** For public access use {@link Builder#build()} instead */ - protected SolrMorphlineContext() {} - - public DocumentLoader getDocumentLoader() { - return loader; - } - - public IndexSchema getIndexSchema() { - return schema; - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - /** - * Helper to construct a {@link SolrMorphlineContext} instance. - */ - public static class Builder extends MorphlineContext.Builder { - - private DocumentLoader loader; - private IndexSchema schema; - - public Builder() {} - - public Builder setDocumentLoader(DocumentLoader loader) { - this.loader = loader; - return this; - } - - public Builder setIndexSchema(IndexSchema schema) { - this.schema = schema; - return this; - } - - @Override - public SolrMorphlineContext build() { - ((SolrMorphlineContext)context).loader = loader; - ((SolrMorphlineContext)context).schema = schema; - return (SolrMorphlineContext) super.build(); - } - - @Override - protected SolrMorphlineContext create() { - return new SolrMorphlineContext(); - } - - } - -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/TokenizeTextBuilder.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/TokenizeTextBuilder.java deleted file mode 100644 index 7c96f3ff176..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/TokenizeTextBuilder.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.io.IOException; -import java.io.Reader; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.analysis.TokenStream; -import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; -import org.apache.solr.schema.FieldType; -import org.apache.solr.schema.IndexSchema; - -import org.kitesdk.morphline.api.Command; -import org.kitesdk.morphline.api.CommandBuilder; -import org.kitesdk.morphline.api.MorphlineCompilationException; -import org.kitesdk.morphline.api.MorphlineContext; -import org.kitesdk.morphline.api.MorphlineRuntimeException; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.AbstractCommand; -import com.typesafe.config.Config; - -/** - * A command that uses the embedded Solr/Lucene Analyzer library to generate tokens from a text - * string, without sending data to a Solr server. - */ -public final class TokenizeTextBuilder implements CommandBuilder { - - @Override - public Collection getNames() { - return Collections.singletonList("tokenizeText"); - } - - @Override - public Command build(Config config, Command parent, Command child, MorphlineContext context) { - return new TokenizeText(this, config, parent, child, context); - } - - - /////////////////////////////////////////////////////////////////////////////// - // Nested classes: - /////////////////////////////////////////////////////////////////////////////// - private static final class TokenizeText extends AbstractCommand { - - private final String inputFieldName; - private final String outputFieldName; - private final Analyzer analyzer; - private final CharTermAttribute token; // cached - private final ReusableStringReader reader = new ReusableStringReader(); // cached - - public TokenizeText(CommandBuilder builder, Config config, Command parent, Command child, MorphlineContext context) { - super(builder, config, parent, child, context); - this.inputFieldName = getConfigs().getString(config, "inputField"); - this.outputFieldName = getConfigs().getString(config, "outputField"); - String solrFieldType = getConfigs().getString(config, "solrFieldType"); - Config solrLocatorConfig = getConfigs().getConfig(config, "solrLocator"); - SolrLocator locator = new SolrLocator(solrLocatorConfig, context); - LOG.debug("solrLocator: {}", locator); - IndexSchema schema = locator.getIndexSchema(); - FieldType fieldType = schema.getFieldTypeByName(solrFieldType); - if (fieldType == null) { - throw new MorphlineCompilationException("Missing Solr field type in schema.xml for name: " + solrFieldType, config); - } - this.analyzer = Objects.requireNonNull(fieldType.getIndexAnalyzer()); - // register CharTermAttribute for later (implicit) reuse - this.token = Objects.requireNonNull(analyzer.tokenStream("content", reader).addAttribute(CharTermAttribute.class)); - validateArguments(); - } - - @Override - protected boolean doProcess(Record record) { - try { - List outputValues = record.get(outputFieldName); - for (Object value : record.get(inputFieldName)) { - reader.setValue(value.toString()); - TokenStream tokenStream = analyzer.tokenStream("content", reader); - tokenStream.reset(); - while (tokenStream.incrementToken()) { - if (token.length() > 0) { // incrementToken() updates the token! - String tokenStr = new String(token.buffer(), 0, token.length()); - outputValues.add(tokenStr); - } - } - tokenStream.end(); - tokenStream.close(); - } - } catch (IOException e) { - throw new MorphlineRuntimeException(e); - } - - // pass record to next command in chain: - return super.doProcess(record); - } - - } - - private static final class ReusableStringReader extends Reader { - private int pos = 0, size = 0; - private String s = null; - - void setValue(String s) { - this.s = s; - this.size = s.length(); - this.pos = 0; - } - - @Override - public int read() { - if (pos < size) { - return s.charAt(pos++); - } else { - s = null; - return -1; - } - } - - @Override - public int read(char[] c, int off, int len) { - if (pos < size) { - len = Math.min(len, size-pos); - s.getChars(pos, pos+len, c, off); - pos += len; - return len; - } else { - s = null; - return -1; - } - } - - @Override - public void close() { - pos = size; // this prevents NPE when reading after close! - s = null; - } - } -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/ZooKeeperDownloader.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/ZooKeeperDownloader.java deleted file mode 100644 index 7185531d807..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/ZooKeeperDownloader.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.io.File; -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.List; - -import org.apache.solr.cloud.ZkController; -import org.apache.solr.common.cloud.Aliases; -import org.apache.solr.common.cloud.ClusterState; -import org.apache.solr.common.cloud.SolrZkClient; -import org.apache.solr.common.cloud.ZkConfigManager; -import org.apache.solr.common.cloud.ZkNodeProps; -import org.apache.solr.common.cloud.ZkStateReader; -import org.apache.solr.common.util.StrUtils; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Preconditions; -import com.google.common.io.Files; - -/** - * Downloads SolrCloud information from ZooKeeper. - */ -final class ZooKeeperDownloader { - - private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public SolrZkClient getZkClient(String zkHost) { - if (zkHost == null) { - throw new IllegalArgumentException("zkHost must not be null"); - } - - SolrZkClient zkClient; - try { - zkClient = new SolrZkClient(zkHost, 30000); - } catch (Exception e) { - throw new IllegalArgumentException("Cannot connect to ZooKeeper: " + zkHost, e); - } - return zkClient; - } - - /** - * Returns config value given collection name - * Borrowed heavily from Solr's ZKController. - */ - public String readConfigName(SolrZkClient zkClient, String collection) - throws KeeperException, InterruptedException { - if (collection == null) { - throw new IllegalArgumentException("collection must not be null"); - } - String configName = null; - - // first check for alias - byte[] aliasData = zkClient.getData(ZkStateReader.ALIASES, null, null, true); - Aliases aliases = ClusterState.load(aliasData); - String alias = aliases.getCollectionAlias(collection); - if (alias != null) { - List aliasList = StrUtils.splitSmart(alias, ",", true); - if (aliasList.size() > 1) { - throw new IllegalArgumentException("collection cannot be an alias that maps to multiple collections"); - } - collection = aliasList.get(0); - } - - String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection; - if (LOG.isInfoEnabled()) { - LOG.info("Load collection config from:" + path); - } - byte[] data = zkClient.getData(path, null, null, true); - - if(data != null) { - ZkNodeProps props = ZkNodeProps.load(data); - configName = props.getStr(ZkController.CONFIGNAME_PROP); - } - - if (configName != null && !zkClient.exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true)) { - LOG.error("Specified config does not exist in ZooKeeper:" + configName); - throw new IllegalArgumentException("Specified config does not exist in ZooKeeper:" - + configName); - } - - return configName; - } - - /** - * Download and return the config directory from ZK - */ - public File downloadConfigDir(SolrZkClient zkClient, String configName, File dir) - throws IOException, InterruptedException, KeeperException { - Preconditions.checkArgument(dir.exists()); - Preconditions.checkArgument(dir.isDirectory()); - ZkConfigManager manager = new ZkConfigManager(zkClient); - manager.downloadConfigDir(configName, dir.toPath()); - File confDir = new File(dir, "conf"); - if (!confDir.isDirectory()) { - // create a temporary directory with "conf" subdir and mv the config in there. This is - // necessary because of CDH-11188; solrctl does not generate nor accept directories with e.g. - // conf/solrconfig.xml which is necessary for proper solr operation. This should work - // even if solrctl changes. - confDir = new File(Files.createTempDir().getAbsolutePath(), "conf"); - confDir.getParentFile().deleteOnExit(); - Files.move(dir, confDir); - dir = confDir.getParentFile(); - } - verifyConfigDir(confDir); - return dir; - } - - private void verifyConfigDir(File confDir) throws IOException { - File solrConfigFile = new File(confDir, "solrconfig.xml"); - if (!solrConfigFile.exists()) { - throw new IOException("Detected invalid Solr config dir in ZooKeeper - Reason: File not found: " - + solrConfigFile.getName()); - } - if (!solrConfigFile.isFile()) { - throw new IOException("Detected invalid Solr config dir in ZooKeeper - Reason: Not a file: " - + solrConfigFile.getName()); - } - if (!solrConfigFile.canRead()) { - throw new IOException("Insufficient permissions to read file: " + solrConfigFile); - } - } - -} diff --git a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/package-info.java b/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/package-info.java deleted file mode 100644 index f4b91eca57c..00000000000 --- a/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/package-info.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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. - */ - -/** - * Morphlines Solr related code. - */ -package org.apache.solr.morphlines.solr; - - - - diff --git a/solr/contrib/morphlines-core/src/java/overview.html b/solr/contrib/morphlines-core/src/java/overview.html deleted file mode 100644 index 7f8ad137a34..00000000000 --- a/solr/contrib/morphlines-core/src/java/overview.html +++ /dev/null @@ -1,21 +0,0 @@ - - - -Apache Solr Search Server: Solr Core Morphline Commands - - diff --git a/solr/contrib/morphlines-core/src/test-files/custom-mimetypes.xml b/solr/contrib/morphlines-core/src/test-files/custom-mimetypes.xml deleted file mode 100644 index 6891e42d616..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/custom-mimetypes.xml +++ /dev/null @@ -1,38 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/log4j.properties b/solr/contrib/morphlines-core/src/test-files/log4j.properties deleted file mode 100644 index 40fc92bbb4d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/log4j.properties +++ /dev/null @@ -1,7 +0,0 @@ -# Logging level -log4j.rootLogger=INFO, CONSOLE - -log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender -log4j.appender.CONSOLE.Target=System.err -log4j.appender.CONSOLE.layout=org.apache.log4j.EnhancedPatternLayout -log4j.appender.CONSOLE.layout.ConversionPattern=%-4r %-5p (%t) [%X{node_name} %X{collection} %X{shard} %X{replica} %X{core}] %c{1.} %m%n diff --git a/solr/contrib/morphlines-core/src/test-files/morphlines-core.marker b/solr/contrib/morphlines-core/src/test-files/morphlines-core.marker deleted file mode 100644 index f4ed7be804a..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/morphlines-core.marker +++ /dev/null @@ -1 +0,0 @@ -# Marker file, so we can lookup this file from classpath to get the resources folder for morphlines-core. \ No newline at end of file diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/currency.xml b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/currency.xml deleted file mode 100644 index 3a9c58afee8..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/currency.xml +++ /dev/null @@ -1,67 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/elevate.xml b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/elevate.xml deleted file mode 100644 index 2c09ebed669..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/elevate.xml +++ /dev/null @@ -1,42 +0,0 @@ - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_ca.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_ca.txt deleted file mode 100644 index 307a85f913d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_ca.txt +++ /dev/null @@ -1,8 +0,0 @@ -# Set of Catalan contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -l -m -n -s -t diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_fr.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_fr.txt deleted file mode 100644 index 722db588333..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_fr.txt +++ /dev/null @@ -1,9 +0,0 @@ -# Set of French contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -l -m -t -qu -n -s -j diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_ga.txt deleted file mode 100644 index 9ebe7fa349a..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -m -b diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_it.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_it.txt deleted file mode 100644 index cac04095372..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/contractions_it.txt +++ /dev/null @@ -1,23 +0,0 @@ -# Set of Italian contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -c -l -all -dall -dell -nell -sull -coll -pell -gl -agl -dagl -degl -negl -sugl -un -m -t -s -v -d diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/hyphenations_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/hyphenations_ga.txt deleted file mode 100644 index 4d2642cc5a3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/hyphenations_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish hyphenations for StopFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -h -n -t diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stemdict_nl.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stemdict_nl.txt deleted file mode 100644 index 441072971d3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stemdict_nl.txt +++ /dev/null @@ -1,6 +0,0 @@ -# Set of overrides for the dutch stemmer -# TODO: load this as a resource from the analyzer and sync it in build.xml -fiets fiets -bromfiets bromfiets -ei eier -kind kinder diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stoptags_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stoptags_ja.txt deleted file mode 100644 index 71b750845e3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stoptags_ja.txt +++ /dev/null @@ -1,420 +0,0 @@ -# -# This file defines a Japanese stoptag set for JapanesePartOfSpeechStopFilter. -# -# Any token with a part-of-speech tag that exactly matches those defined in this -# file are removed from the token stream. -# -# Set your own stoptags by uncommenting the lines below. Note that comments are -# not allowed on the same line as a stoptag. See LUCENE-3745 for frequency lists, -# etc. that can be useful for building you own stoptag set. -# -# The entire possible tagset is provided below for convenience. -# -##### -# noun: unclassified nouns -#名詞 -# -# noun-common: Common nouns or nouns where the sub-classification is undefined -#名詞-一般 -# -# noun-proper: Proper nouns where the sub-classification is undefined -#名詞-固有名詞 -# -# noun-proper-misc: miscellaneous proper nouns -#名詞-固有名詞-一般 -# -# noun-proper-person: Personal names where the sub-classification is undefined -#名詞-固有名詞-人名 -# -# noun-proper-person-misc: names that cannot be divided into surname and -# given name; foreign names; names where the surname or given name is unknown. -# e.g. お市の方 -#名詞-固有名詞-人名-一般 -# -# noun-proper-person-surname: Mainly Japanese surnames. -# e.g. 山田 -#名詞-固有名詞-人名-姓 -# -# noun-proper-person-given_name: Mainly Japanese given names. -# e.g. 太郎 -#名詞-固有名詞-人名-名 -# -# noun-proper-organization: Names representing organizations. -# e.g. 通産省, NHK -#名詞-固有名詞-組織 -# -# noun-proper-place: Place names where the sub-classification is undefined -#名詞-固有名詞-地域 -# -# noun-proper-place-misc: Place names excluding countries. -# e.g. アジア, バルセロナ, 京都 -#名詞-固有名詞-地域-一般 -# -# noun-proper-place-country: Country names. -# e.g. 日本, オーストラリア -#名詞-固有名詞-地域-国 -# -# noun-pronoun: Pronouns where the sub-classification is undefined -#名詞-代名詞 -# -# noun-pronoun-misc: miscellaneous pronouns: -# e.g. それ, ここ, あいつ, あなた, あちこち, いくつ, どこか, なに, みなさん, みんな, わたくし, われわれ -#名詞-代名詞-一般 -# -# noun-pronoun-contraction: Spoken language contraction made by combining a -# pronoun and the particle 'wa'. -# e.g. ありゃ, こりゃ, こりゃあ, そりゃ, そりゃあ -#名詞-代名詞-縮約 -# -# noun-adverbial: Temporal nouns such as names of days or months that behave -# like adverbs. Nouns that represent amount or ratios and can be used adverbially, -# e.g. 金曜, 一月, 午後, 少量 -#名詞-副詞可能 -# -# noun-verbal: Nouns that take arguments with case and can appear followed by -# 'suru' and related verbs (する, できる, なさる, くださる) -# e.g. インプット, 愛着, 悪化, 悪戦苦闘, 一安心, 下取り -#名詞-サ変接続 -# -# noun-adjective-base: The base form of adjectives, words that appear before な ("na") -# e.g. 健康, 安易, 駄目, だめ -#名詞-形容動詞語幹 -# -# noun-numeric: Arabic numbers, Chinese numerals, and counters like 何 (回), 数. -# e.g. 0, 1, 2, 何, 数, 幾 -#名詞-数 -# -# noun-affix: noun affixes where the sub-classification is undefined -#名詞-非自立 -# -# noun-affix-misc: Of adnominalizers, the case-marker の ("no"), and words that -# attach to the base form of inflectional words, words that cannot be classified -# into any of the other categories below. This category includes indefinite nouns. -# e.g. あかつき, 暁, かい, 甲斐, 気, きらい, 嫌い, くせ, 癖, こと, 事, ごと, 毎, しだい, 次第, -# 順, せい, 所為, ついで, 序で, つもり, 積もり, 点, どころ, の, はず, 筈, はずみ, 弾み, -# 拍子, ふう, ふり, 振り, ほう, 方, 旨, もの, 物, 者, ゆえ, 故, ゆえん, 所以, わけ, 訳, -# わり, 割り, 割, ん-口語/, もん-口語/ -#名詞-非自立-一般 -# -# noun-affix-adverbial: noun affixes that that can behave as adverbs. -# e.g. あいだ, 間, あげく, 挙げ句, あと, 後, 余り, 以外, 以降, 以後, 以上, 以前, 一方, うえ, -# 上, うち, 内, おり, 折り, かぎり, 限り, きり, っきり, 結果, ころ, 頃, さい, 際, 最中, さなか, -# 最中, じたい, 自体, たび, 度, ため, 為, つど, 都度, とおり, 通り, とき, 時, ところ, 所, -# とたん, 途端, なか, 中, のち, 後, ばあい, 場合, 日, ぶん, 分, ほか, 他, まえ, 前, まま, -# 儘, 侭, みぎり, 矢先 -#名詞-非自立-副詞可能 -# -# noun-affix-aux: noun affixes treated as 助動詞 ("auxiliary verb") in school grammars -# with the stem よう(だ) ("you(da)"). -# e.g. よう, やう, 様 (よう) -#名詞-非自立-助動詞語幹 -# -# noun-affix-adjective-base: noun affixes that can connect to the indeclinable -# connection form な (aux "da"). -# e.g. みたい, ふう -#名詞-非自立-形容動詞語幹 -# -# noun-special: special nouns where the sub-classification is undefined. -#名詞-特殊 -# -# noun-special-aux: The そうだ ("souda") stem form that is used for reporting news, is -# treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the base -# form of inflectional words. -# e.g. そう -#名詞-特殊-助動詞語幹 -# -# noun-suffix: noun suffixes where the sub-classification is undefined. -#名詞-接尾 -# -# noun-suffix-misc: Of the nouns or stem forms of other parts of speech that connect -# to ガル or タイ and can combine into compound nouns, words that cannot be classified into -# any of the other categories below. In general, this category is more inclusive than -# 接尾語 ("suffix") and is usually the last element in a compound noun. -# e.g. おき, かた, 方, 甲斐 (がい), がかり, ぎみ, 気味, ぐるみ, (~した) さ, 次第, 済 (ず) み, -# よう, (でき)っこ, 感, 観, 性, 学, 類, 面, 用 -#名詞-接尾-一般 -# -# noun-suffix-person: Suffixes that form nouns and attach to person names more often -# than other nouns. -# e.g. 君, 様, 著 -#名詞-接尾-人名 -# -# noun-suffix-place: Suffixes that form nouns and attach to place names more often -# than other nouns. -# e.g. 町, 市, 県 -#名詞-接尾-地域 -# -# noun-suffix-verbal: Of the suffixes that attach to nouns and form nouns, those that -# can appear before スル ("suru"). -# e.g. 化, 視, 分け, 入り, 落ち, 買い -#名詞-接尾-サ変接続 -# -# noun-suffix-aux: The stem form of そうだ (様態) that is used to indicate conditions, -# is treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the -# conjunctive form of inflectional words. -# e.g. そう -#名詞-接尾-助動詞語幹 -# -# noun-suffix-adjective-base: Suffixes that attach to other nouns or the conjunctive -# form of inflectional words and appear before the copula だ ("da"). -# e.g. 的, げ, がち -#名詞-接尾-形容動詞語幹 -# -# noun-suffix-adverbial: Suffixes that attach to other nouns and can behave as adverbs. -# e.g. 後 (ご), 以後, 以降, 以前, 前後, 中, 末, 上, 時 (じ) -#名詞-接尾-副詞可能 -# -# noun-suffix-classifier: Suffixes that attach to numbers and form nouns. This category -# is more inclusive than 助数詞 ("classifier") and includes common nouns that attach -# to numbers. -# e.g. 個, つ, 本, 冊, パーセント, cm, kg, カ月, か国, 区画, 時間, 時半 -#名詞-接尾-助数詞 -# -# noun-suffix-special: Special suffixes that mainly attach to inflecting words. -# e.g. (楽し) さ, (考え) 方 -#名詞-接尾-特殊 -# -# noun-suffix-conjunctive: Nouns that behave like conjunctions and join two words -# together. -# e.g. (日本) 対 (アメリカ), 対 (アメリカ), (3) 対 (5), (女優) 兼 (主婦) -#名詞-接続詞的 -# -# noun-verbal_aux: Nouns that attach to the conjunctive particle て ("te") and are -# semantically verb-like. -# e.g. ごらん, ご覧, 御覧, 頂戴 -#名詞-動詞非自立的 -# -# noun-quotation: text that cannot be segmented into words, proverbs, Chinese poetry, -# dialects, English, etc. Currently, the only entry for 名詞 引用文字列 ("noun quotation") -# is いわく ("iwaku"). -#名詞-引用文字列 -# -# noun-nai_adjective: Words that appear before the auxiliary verb ない ("nai") and -# behave like an adjective. -# e.g. 申し訳, 仕方, とんでも, 違い -#名詞-ナイ形容詞語幹 -# -##### -# prefix: unclassified prefixes -#接頭詞 -# -# prefix-nominal: Prefixes that attach to nouns (including adjective stem forms) -# excluding numerical expressions. -# e.g. お (水), 某 (氏), 同 (社), 故 (~氏), 高 (品質), お (見事), ご (立派) -#接頭詞-名詞接続 -# -# prefix-verbal: Prefixes that attach to the imperative form of a verb or a verb -# in conjunctive form followed by なる/なさる/くださる. -# e.g. お (読みなさい), お (座り) -#接頭詞-動詞接続 -# -# prefix-adjectival: Prefixes that attach to adjectives. -# e.g. お (寒いですねえ), バカ (でかい) -#接頭詞-形容詞接続 -# -# prefix-numerical: Prefixes that attach to numerical expressions. -# e.g. 約, およそ, 毎時 -#接頭詞-数接続 -# -##### -# verb: unclassified verbs -#動詞 -# -# verb-main: -#動詞-自立 -# -# verb-auxiliary: -#動詞-非自立 -# -# verb-suffix: -#動詞-接尾 -# -##### -# adjective: unclassified adjectives -#形容詞 -# -# adjective-main: -#形容詞-自立 -# -# adjective-auxiliary: -#形容詞-非自立 -# -# adjective-suffix: -#形容詞-接尾 -# -##### -# adverb: unclassified adverbs -#副詞 -# -# adverb-misc: Words that can be segmented into one unit and where adnominal -# modification is not possible. -# e.g. あいかわらず, 多分 -#副詞-一般 -# -# adverb-particle_conjunction: Adverbs that can be followed by の, は, に, -# な, する, だ, etc. -# e.g. こんなに, そんなに, あんなに, なにか, なんでも -#副詞-助詞類接続 -# -##### -# adnominal: Words that only have noun-modifying forms. -# e.g. この, その, あの, どの, いわゆる, なんらかの, 何らかの, いろんな, こういう, そういう, ああいう, -# どういう, こんな, そんな, あんな, どんな, 大きな, 小さな, おかしな, ほんの, たいした, -# 「(, も) さる (ことながら)」, 微々たる, 堂々たる, 単なる, いかなる, 我が」「同じ, 亡き -#連体詞 -# -##### -# conjunction: Conjunctions that can occur independently. -# e.g. が, けれども, そして, じゃあ, それどころか -接続詞 -# -##### -# particle: unclassified particles. -助詞 -# -# particle-case: case particles where the subclassification is undefined. -助詞-格助詞 -# -# particle-case-misc: Case particles. -# e.g. から, が, で, と, に, へ, より, を, の, にて -助詞-格助詞-一般 -# -# particle-case-quote: the "to" that appears after nouns, a person’s speech, -# quotation marks, expressions of decisions from a meeting, reasons, judgements, -# conjectures, etc. -# e.g. ( だ) と (述べた.), ( である) と (して執行猶予...) -助詞-格助詞-引用 -# -# particle-case-compound: Compounds of particles and verbs that mainly behave -# like case particles. -# e.g. という, といった, とかいう, として, とともに, と共に, でもって, にあたって, に当たって, に当って, -# にあたり, に当たり, に当り, に当たる, にあたる, において, に於いて,に於て, における, に於ける, -# にかけ, にかけて, にかんし, に関し, にかんして, に関して, にかんする, に関する, に際し, -# に際して, にしたがい, に従い, に従う, にしたがって, に従って, にたいし, に対し, にたいして, -# に対して, にたいする, に対する, について, につき, につけ, につけて, につれ, につれて, にとって, -# にとり, にまつわる, によって, に依って, に因って, により, に依り, に因り, による, に依る, に因る, -# にわたって, にわたる, をもって, を以って, を通じ, を通じて, を通して, をめぐって, をめぐり, をめぐる, -# って-口語/, ちゅう-関西弁「という」/, (何) ていう (人)-口語/, っていう-口語/, といふ, とかいふ -助詞-格助詞-連語 -# -# particle-conjunctive: -# e.g. から, からには, が, けれど, けれども, けど, し, つつ, て, で, と, ところが, どころか, とも, ども, -# ながら, なり, ので, のに, ば, ものの, や ( した), やいなや, (ころん) じゃ(いけない)-口語/, -# (行っ) ちゃ(いけない)-口語/, (言っ) たって (しかたがない)-口語/, (それがなく)ったって (平気)-口語/ -助詞-接続助詞 -# -# particle-dependency: -# e.g. こそ, さえ, しか, すら, は, も, ぞ -助詞-係助詞 -# -# particle-adverbial: -# e.g. がてら, かも, くらい, 位, ぐらい, しも, (学校) じゃ(これが流行っている)-口語/, -# (それ)じゃあ (よくない)-口語/, ずつ, (私) なぞ, など, (私) なり (に), (先生) なんか (大嫌い)-口語/, -# (私) なんぞ, (先生) なんて (大嫌い)-口語/, のみ, だけ, (私) だって-口語/, だに, -# (彼)ったら-口語/, (お茶) でも (いかが), 等 (とう), (今後) とも, ばかり, ばっか-口語/, ばっかり-口語/, -# ほど, 程, まで, 迄, (誰) も (が)([助詞-格助詞] および [助詞-係助詞] の前に位置する「も」) -助詞-副助詞 -# -# particle-interjective: particles with interjective grammatical roles. -# e.g. (松島) や -助詞-間投助詞 -# -# particle-coordinate: -# e.g. と, たり, だの, だり, とか, なり, や, やら -助詞-並立助詞 -# -# particle-final: -# e.g. かい, かしら, さ, ぜ, (だ)っけ-口語/, (とまってる) で-方言/, な, ナ, なあ-口語/, ぞ, ね, ネ, -# ねぇ-口語/, ねえ-口語/, ねん-方言/, の, のう-口語/, や, よ, ヨ, よぉ-口語/, わ, わい-口語/ -助詞-終助詞 -# -# particle-adverbial/conjunctive/final: The particle "ka" when unknown whether it is -# adverbial, conjunctive, or sentence final. For example: -# (a) 「A か B か」. Ex:「(国内で運用する) か,(海外で運用する) か (.)」 -# (b) Inside an adverb phrase. Ex:「(幸いという) か (, 死者はいなかった.)」 -# 「(祈りが届いたせい) か (, 試験に合格した.)」 -# (c) 「かのように」. Ex:「(何もなかった) か (のように振る舞った.)」 -# e.g. か -助詞-副助詞/並立助詞/終助詞 -# -# particle-adnominalizer: The "no" that attaches to nouns and modifies -# non-inflectional words. -助詞-連体化 -# -# particle-adnominalizer: The "ni" and "to" that appear following nouns and adverbs -# that are giongo, giseigo, or gitaigo. -# e.g. に, と -助詞-副詞化 -# -# particle-special: A particle that does not fit into one of the above classifications. -# This includes particles that are used in Tanka, Haiku, and other poetry. -# e.g. かな, けむ, ( しただろう) に, (あんた) にゃ(わからん), (俺) ん (家) -助詞-特殊 -# -##### -# auxiliary-verb: -助動詞 -# -##### -# interjection: Greetings and other exclamations. -# e.g. おはよう, おはようございます, こんにちは, こんばんは, ありがとう, どうもありがとう, ありがとうございます, -# いただきます, ごちそうさま, さよなら, さようなら, はい, いいえ, ごめん, ごめんなさい -#感動詞 -# -##### -# symbol: unclassified Symbols. -記号 -# -# symbol-misc: A general symbol not in one of the categories below. -# e.g. [○◎@$〒→+] -記号-一般 -# -# symbol-comma: Commas -# e.g. [,、] -記号-読点 -# -# symbol-period: Periods and full stops. -# e.g. [..。] -記号-句点 -# -# symbol-space: Full-width whitespace. -記号-空白 -# -# symbol-open_bracket: -# e.g. [({‘“『【] -記号-括弧開 -# -# symbol-close_bracket: -# e.g. [)}’”』」】] -記号-括弧閉 -# -# symbol-alphabetic: -#記号-アルファベット -# -##### -# other: unclassified other -#その他 -# -# other-interjection: Words that are hard to classify as noun-suffixes or -# sentence-final particles. -# e.g. (だ)ァ -その他-間投 -# -##### -# filler: Aizuchi that occurs during a conversation or sounds inserted as filler. -# e.g. あの, うんと, えと -フィラー -# -##### -# non-verbal: non-verbal sound. -非言語音 -# -##### -# fragment: -#語断片 -# -##### -# unknown: unknown part of speech. -#未知語 -# -##### End of file diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ar.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ar.txt deleted file mode 100644 index 046829db6a2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ar.txt +++ /dev/null @@ -1,125 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Cleaned on October 11, 2009 (not normalized, so use before normalization) -# This means that when modifying this list, you might need to add some -# redundant entries, for example containing forms with both أ and ا -من -ومن -منها -منه -في -وفي -فيها -فيه -و -ف -ثم -او -أو -ب -بها -به -ا -أ -اى -اي -أي -أى -لا -ولا -الا -ألا -إلا -لكن -ما -وما -كما -فما -عن -مع -اذا -إذا -ان -أن -إن -انها -أنها -إنها -انه -أنه -إنه -بان -بأن -فان -فأن -وان -وأن -وإن -التى -التي -الذى -الذي -الذين -الى -الي -إلى -إلي -على -عليها -عليه -اما -أما -إما -ايضا -أيضا -كل -وكل -لم -ولم -لن -ولن -هى -هي -هو -وهى -وهي -وهو -فهى -فهي -فهو -انت -أنت -لك -لها -له -هذه -هذا -تلك -ذلك -هناك -كانت -كان -يكون -تكون -وكانت -وكان -غير -بعض -قد -نحو -بين -بينما -منذ -ضمن -حيث -الان -الآن -خلال -بعد -قبل -حتى -عند -عندما -لدى -جميع diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_bg.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_bg.txt deleted file mode 100644 index 1ae4ba2ae38..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_bg.txt +++ /dev/null @@ -1,193 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -а -аз -ако -ала -бе -без -беше -би -бил -била -били -било -близо -бъдат -бъде -бяха -в -вас -ваш -ваша -вероятно -вече -взема -ви -вие -винаги -все -всеки -всички -всичко -всяка -във -въпреки -върху -г -ги -главно -го -д -да -дали -до -докато -докога -дори -досега -доста -е -едва -един -ето -за -зад -заедно -заради -засега -затова -защо -защото -и -из -или -им -има -имат -иска -й -каза -как -каква -какво -както -какъв -като -кога -когато -което -които -кой -който -колко -която -къде -където -към -ли -м -ме -между -мен -ми -мнозина -мога -могат -може -моля -момента -му -н -на -над -назад -най -направи -напред -например -нас -не -него -нея -ни -ние -никой -нито -но -някои -някой -няма -обаче -около -освен -особено -от -отгоре -отново -още -пак -по -повече -повечето -под -поне -поради -после -почти -прави -пред -преди -през -при -пък -първо -с -са -само -се -сега -си -скоро -след -сме -според -сред -срещу -сте -съм -със -също -т -тази -така -такива -такъв -там -твой -те -тези -ти -тн -то -това -тогава -този -той -толкова -точно -трябва -тук -тъй -тя -тях -у -харесва -ч -че -често -чрез -ще -щом -я diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ca.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ca.txt deleted file mode 100644 index 3da65deafe1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ca.txt +++ /dev/null @@ -1,220 +0,0 @@ -# Catalan stopwords from http://github.com/vcl/cue.language (Apache 2 Licensed) -a -abans -ací -ah -així -això -al -als -aleshores -algun -alguna -algunes -alguns -alhora -allà -allí -allò -altra -altre -altres -amb -ambdós -ambdues -apa -aquell -aquella -aquelles -aquells -aquest -aquesta -aquestes -aquests -aquí -baix -cada -cadascú -cadascuna -cadascunes -cadascuns -com -contra -d'un -d'una -d'unes -d'uns -dalt -de -del -dels -des -després -dins -dintre -donat -doncs -durant -e -eh -el -els -em -en -encara -ens -entre -érem -eren -éreu -es -és -esta -està -estàvem -estaven -estàveu -esteu -et -etc -ets -fins -fora -gairebé -ha -han -has -havia -he -hem -heu -hi -ho -i -igual -iguals -ja -l'hi -la -les -li -li'n -llavors -m'he -ma -mal -malgrat -mateix -mateixa -mateixes -mateixos -me -mentre -més -meu -meus -meva -meves -molt -molta -moltes -molts -mon -mons -n'he -n'hi -ne -ni -no -nogensmenys -només -nosaltres -nostra -nostre -nostres -o -oh -oi -on -pas -pel -pels -per -però -perquè -poc -poca -pocs -poques -potser -propi -qual -quals -quan -quant -que -què -quelcom -qui -quin -quina -quines -quins -s'ha -s'han -sa -semblant -semblants -ses -seu -seus -seva -seva -seves -si -sobre -sobretot -sóc -solament -sols -son -són -sons -sota -sou -t'ha -t'han -t'he -ta -tal -també -tampoc -tan -tant -tanta -tantes -teu -teus -teva -teves -ton -tons -tot -tota -totes -tots -un -una -unes -uns -us -va -vaig -vam -van -vas -veu -vosaltres -vostra -vostre -vostres diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_cz.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_cz.txt deleted file mode 100644 index 53c6097dac7..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_cz.txt +++ /dev/null @@ -1,172 +0,0 @@ -a -s -k -o -i -u -v -z -dnes -cz -tímto -budeš -budem -byli -jseš -můj -svým -ta -tomto -tohle -tuto -tyto -jej -zda -proč -máte -tato -kam -tohoto -kdo -kteří -mi -nám -tom -tomuto -mít -nic -proto -kterou -byla -toho -protože -asi -ho -naši -napište -re -což -tím -takže -svých -její -svými -jste -aj -tu -tedy -teto -bylo -kde -ke -pravé -ji -nad -nejsou -či -pod -téma -mezi -přes -ty -pak -vám -ani -když -však -neg -jsem -tento -článku -články -aby -jsme -před -pta -jejich -byl -ještě -až -bez -také -pouze -první -vaše -která -nás -nový -tipy -pokud -může -strana -jeho -své -jiné -zprávy -nové -není -vás -jen -podle -zde -už -být -více -bude -již -než -který -by -které -co -nebo -ten -tak -má -při -od -po -jsou -jak -další -ale -si -se -ve -to -jako -za -zpět -ze -do -pro -je -na -atd -atp -jakmile -přičemž -já -on -ona -ono -oni -ony -my -vy -jí -ji -mě -mne -jemu -tomu -těm -těmu -němu -němuž -jehož -jíž -jelikož -jež -jakož -načež diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_da.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_da.txt deleted file mode 100644 index a3ff5fe122c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_da.txt +++ /dev/null @@ -1,108 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/danish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Danish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - -og | and -i | in -jeg | I -det | that (dem. pronoun)/it (pers. pronoun) -at | that (in front of a sentence)/to (with infinitive) -en | a/an -den | it (pers. pronoun)/that (dem. pronoun) -til | to/at/for/until/against/by/of/into, more -er | present tense of "to be" -som | who, as -på | on/upon/in/on/at/to/after/of/with/for, on -de | they -med | with/by/in, along -han | he -af | of/by/from/off/for/in/with/on, off -for | at/for/to/from/by/of/ago, in front/before, because -ikke | not -der | who/which, there/those -var | past tense of "to be" -mig | me/myself -sig | oneself/himself/herself/itself/themselves -men | but -et | a/an/one, one (number), someone/somebody/one -har | present tense of "to have" -om | round/about/for/in/a, about/around/down, if -vi | we -min | my -havde | past tense of "to have" -ham | him -hun | she -nu | now -over | over/above/across/by/beyond/past/on/about, over/past -da | then, when/as/since -fra | from/off/since, off, since -du | you -ud | out -sin | his/her/its/one's -dem | them -os | us/ourselves -op | up -man | you/one -hans | his -hvor | where -eller | or -hvad | what -skal | must/shall etc. -selv | myself/youself/herself/ourselves etc., even -her | here -alle | all/everyone/everybody etc. -vil | will (verb) -blev | past tense of "to stay/to remain/to get/to become" -kunne | could -ind | in -når | when -være | present tense of "to be" -dog | however/yet/after all -noget | something -ville | would -jo | you know/you see (adv), yes -deres | their/theirs -efter | after/behind/according to/for/by/from, later/afterwards -ned | down -skulle | should -denne | this -end | than -dette | this -mit | my/mine -også | also -under | under/beneath/below/during, below/underneath -have | have -dig | you -anden | other -hende | her -mine | my -alt | everything -meget | much/very, plenty of -sit | his, her, its, one's -sine | his, her, its, one's -vor | our -mod | against -disse | these -hvis | if -din | your/yours -nogle | some -hos | by/at -blive | be/become -mange | many -ad | by/through -bliver | present tense of "to be/to become" -hendes | her/hers -været | be -thi | for (conj) -jer | you -sådan | such, like this/like that diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_de.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_de.txt deleted file mode 100644 index f7703841887..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_de.txt +++ /dev/null @@ -1,292 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/german/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A German stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | The number of forms in this list is reduced significantly by passing it - | through the German stemmer. - - -aber | but - -alle | all -allem -allen -aller -alles - -als | than, as -also | so -am | an + dem -an | at - -ander | other -andere -anderem -anderen -anderer -anderes -anderm -andern -anderr -anders - -auch | also -auf | on -aus | out of -bei | by -bin | am -bis | until -bist | art -da | there -damit | with it -dann | then - -der | the -den -des -dem -die -das - -daß | that - -derselbe | the same -derselben -denselben -desselben -demselben -dieselbe -dieselben -dasselbe - -dazu | to that - -dein | thy -deine -deinem -deinen -deiner -deines - -denn | because - -derer | of those -dessen | of him - -dich | thee -dir | to thee -du | thou - -dies | this -diese -diesem -diesen -dieser -dieses - - -doch | (several meanings) -dort | (over) there - - -durch | through - -ein | a -eine -einem -einen -einer -eines - -einig | some -einige -einigem -einigen -einiger -einiges - -einmal | once - -er | he -ihn | him -ihm | to him - -es | it -etwas | something - -euer | your -eure -eurem -euren -eurer -eures - -für | for -gegen | towards -gewesen | p.p. of sein -hab | have -habe | have -haben | have -hat | has -hatte | had -hatten | had -hier | here -hin | there -hinter | behind - -ich | I -mich | me -mir | to me - - -ihr | you, to her -ihre -ihrem -ihren -ihrer -ihres -euch | to you - -im | in + dem -in | in -indem | while -ins | in + das -ist | is - -jede | each, every -jedem -jeden -jeder -jedes - -jene | that -jenem -jenen -jener -jenes - -jetzt | now -kann | can - -kein | no -keine -keinem -keinen -keiner -keines - -können | can -könnte | could -machen | do -man | one - -manche | some, many a -manchem -manchen -mancher -manches - -mein | my -meine -meinem -meinen -meiner -meines - -mit | with -muss | must -musste | had to -nach | to(wards) -nicht | not -nichts | nothing -noch | still, yet -nun | now -nur | only -ob | whether -oder | or -ohne | without -sehr | very - -sein | his -seine -seinem -seinen -seiner -seines - -selbst | self -sich | herself - -sie | they, she -ihnen | to them - -sind | are -so | so - -solche | such -solchem -solchen -solcher -solches - -soll | shall -sollte | should -sondern | but -sonst | else -über | over -um | about, around -und | and - -uns | us -unse -unsem -unsen -unser -unses - -unter | under -viel | much -vom | von + dem -von | from -vor | before -während | while -war | was -waren | were -warst | wast -was | what -weg | away, off -weil | because -weiter | further - -welche | which -welchem -welchen -welcher -welches - -wenn | when -werde | will -werden | will -wie | how -wieder | again -will | want -wir | we -wird | will -wirst | willst -wo | where -wollen | want -wollte | wanted -würde | would -würden | would -zu | to -zum | zu + dem -zur | zu + der -zwar | indeed -zwischen | between - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_el.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_el.txt deleted file mode 100644 index 232681f5bd6..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_el.txt +++ /dev/null @@ -1,78 +0,0 @@ -# Lucene Greek Stopwords list -# Note: by default this file is used after GreekLowerCaseFilter, -# so when modifying this file use 'σ' instead of 'ς' -ο -η -το -οι -τα -του -τησ -των -τον -την -και -κι -κ -ειμαι -εισαι -ειναι -ειμαστε -ειστε -στο -στον -στη -στην -μα -αλλα -απο -για -προσ -με -σε -ωσ -παρα -αντι -κατα -μετα -θα -να -δε -δεν -μη -μην -επι -ενω -εαν -αν -τοτε -που -πωσ -ποιοσ -ποια -ποιο -ποιοι -ποιεσ -ποιων -ποιουσ -αυτοσ -αυτη -αυτο -αυτοι -αυτων -αυτουσ -αυτεσ -αυτα -εκεινοσ -εκεινη -εκεινο -εκεινοι -εκεινεσ -εκεινα -εκεινων -εκεινουσ -οπωσ -ομωσ -ισωσ -οσο -οτι diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_en.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_en.txt deleted file mode 100644 index 2c164c0b2a1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_en.txt +++ /dev/null @@ -1,54 +0,0 @@ -# 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. - -# a couple of test stopwords to test that the words are really being -# configured from this file: -stopworda -stopwordb - -# Standard english stop words taken from Lucene's StopAnalyzer -a -an -and -are -as -at -be -but -by -for -if -in -into -is -it -no -not -of -on -or -such -that -the -their -then -there -these -they -this -to -was -will -with diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_es.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_es.txt deleted file mode 100644 index 2db14760075..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_es.txt +++ /dev/null @@ -1,354 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/spanish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Spanish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | from, of -la | the, her -que | who, that -el | the -en | in -y | and -a | to -los | the, them -del | de + el -se | himself, from him etc -las | the, them -por | for, by, etc -un | a -para | for -con | with -no | no -una | a -su | his, her -al | a + el - | es from SER -lo | him -como | how -más | more -pero | pero -sus | su plural -le | to him, her -ya | already -o | or - | fue from SER -este | this - | ha from HABER -sí | himself etc -porque | because -esta | this - | son from SER -entre | between - | está from ESTAR -cuando | when -muy | very -sin | without -sobre | on - | ser from SER - | tiene from TENER -también | also -me | me -hasta | until -hay | there is/are -donde | where - | han from HABER -quien | whom, that - | están from ESTAR - | estado from ESTAR -desde | from -todo | all -nos | us -durante | during - | estados from ESTAR -todos | all -uno | a -les | to them -ni | nor -contra | against -otros | other - | fueron from SER -ese | that -eso | that - | había from HABER -ante | before -ellos | they -e | and (variant of y) -esto | this -mí | me -antes | before -algunos | some -qué | what? -unos | a -yo | I -otro | other -otras | other -otra | other -él | he -tanto | so much, many -esa | that -estos | these -mucho | much, many -quienes | who -nada | nothing -muchos | many -cual | who - | sea from SER -poco | few -ella | she -estar | to be - | haber from HABER -estas | these - | estaba from ESTAR - | estamos from ESTAR -algunas | some -algo | something -nosotros | we - - | other forms - -mi | me -mis | mi plural -tú | thou -te | thee -ti | thee -tu | thy -tus | tu plural -ellas | they -nosotras | we -vosotros | you -vosotras | you -os | you -mío | mine -mía | -míos | -mías | -tuyo | thine -tuya | -tuyos | -tuyas | -suyo | his, hers, theirs -suya | -suyos | -suyas | -nuestro | ours -nuestra | -nuestros | -nuestras | -vuestro | yours -vuestra | -vuestros | -vuestras | -esos | those -esas | those - - | forms of estar, to be (not including the infinitive): -estoy -estás -está -estamos -estáis -están -esté -estés -estemos -estéis -estén -estaré -estarás -estará -estaremos -estaréis -estarán -estaría -estarías -estaríamos -estaríais -estarían -estaba -estabas -estábamos -estabais -estaban -estuve -estuviste -estuvo -estuvimos -estuvisteis -estuvieron -estuviera -estuvieras -estuviéramos -estuvierais -estuvieran -estuviese -estuvieses -estuviésemos -estuvieseis -estuviesen -estando -estado -estada -estados -estadas -estad - - | forms of haber, to have (not including the infinitive): -he -has -ha -hemos -habéis -han -haya -hayas -hayamos -hayáis -hayan -habré -habrás -habrá -habremos -habréis -habrán -habría -habrías -habríamos -habríais -habrían -había -habías -habíamos -habíais -habían -hube -hubiste -hubo -hubimos -hubisteis -hubieron -hubiera -hubieras -hubiéramos -hubierais -hubieran -hubiese -hubieses -hubiésemos -hubieseis -hubiesen -habiendo -habido -habida -habidos -habidas - - | forms of ser, to be (not including the infinitive): -soy -eres -es -somos -sois -son -sea -seas -seamos -seáis -sean -seré -serás -será -seremos -seréis -serán -sería -serías -seríamos -seríais -serían -era -eras -éramos -erais -eran -fui -fuiste -fue -fuimos -fuisteis -fueron -fuera -fueras -fuéramos -fuerais -fueran -fuese -fueses -fuésemos -fueseis -fuesen -siendo -sido - | sed also means 'thirst' - - | forms of tener, to have (not including the infinitive): -tengo -tienes -tiene -tenemos -tenéis -tienen -tenga -tengas -tengamos -tengáis -tengan -tendré -tendrás -tendrá -tendremos -tendréis -tendrán -tendría -tendrías -tendríamos -tendríais -tendrían -tenía -tenías -teníamos -teníais -tenían -tuve -tuviste -tuvo -tuvimos -tuvisteis -tuvieron -tuviera -tuvieras -tuviéramos -tuvierais -tuvieran -tuviese -tuvieses -tuviésemos -tuvieseis -tuviesen -teniendo -tenido -tenida -tenidos -tenidas -tened - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_eu.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_eu.txt deleted file mode 100644 index 25f1db93460..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_eu.txt +++ /dev/null @@ -1,99 +0,0 @@ -# example set of basque stopwords -al -anitz -arabera -asko -baina -bat -batean -batek -bati -batzuei -batzuek -batzuetan -batzuk -bera -beraiek -berau -berauek -bere -berori -beroriek -beste -bezala -da -dago -dira -ditu -du -dute -edo -egin -ere -eta -eurak -ez -gainera -gu -gutxi -guzti -haiei -haiek -haietan -hainbeste -hala -han -handik -hango -hara -hari -hark -hartan -hau -hauei -hauek -hauetan -hemen -hemendik -hemengo -hi -hona -honek -honela -honetan -honi -hor -hori -horiei -horiek -horietan -horko -horra -horrek -horrela -horretan -horri -hortik -hura -izan -ni -noiz -nola -non -nondik -nongo -nor -nora -ze -zein -zen -zenbait -zenbat -zer -zergatik -ziren -zituen -zu -zuek -zuen -zuten diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fa.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fa.txt deleted file mode 100644 index 723641c6da7..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fa.txt +++ /dev/null @@ -1,313 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Note: by default this file is used after normalization, so when adding entries -# to this file, use the arabic 'ي' instead of 'ی' -انان -نداشته -سراسر -خياه -ايشان -وي -تاكنون -بيشتري -دوم -پس -ناشي -وگو -يا -داشتند -سپس -هنگام -هرگز -پنج -نشان -امسال -ديگر -گروهي -شدند -چطور -ده -و -دو -نخستين -ولي -چرا -چه -وسط -ه -كدام -قابل -يك -رفت -هفت -همچنين -در -هزار -بله -بلي -شايد -اما -شناسي -گرفته -دهد -داشته -دانست -داشتن -خواهيم -ميليارد -وقتيكه -امد -خواهد -جز -اورده -شده -بلكه -خدمات -شدن -برخي -نبود -بسياري -جلوگيري -حق -كردند -نوعي -بعري -نكرده -نظير -نبايد -بوده -بودن -داد -اورد -هست -جايي -شود -دنبال -داده -بايد -سابق -هيچ -همان -انجا -كمتر -كجاست -گردد -كسي -تر -مردم -تان -دادن -بودند -سري -جدا -ندارند -مگر -يكديگر -دارد -دهند -بنابراين -هنگامي -سمت -جا -انچه -خود -دادند -زياد -دارند -اثر -بدون -بهترين -بيشتر -البته -به -براساس -بيرون -كرد -بعضي -گرفت -توي -اي -ميليون -او -جريان -تول -بر -مانند -برابر -باشيم -مدتي -گويند -اكنون -تا -تنها -جديد -چند -بي -نشده -كردن -كردم -گويد -كرده -كنيم -نمي -نزد -روي -قصد -فقط -بالاي -ديگران -اين -ديروز -توسط -سوم -ايم -دانند -سوي -استفاده -شما -كنار -داريم -ساخته -طور -امده -رفته -نخست -بيست -نزديك -طي -كنيد -از -انها -تمامي -داشت -يكي -طريق -اش -چيست -روب -نمايد -گفت -چندين -چيزي -تواند -ام -ايا -با -ان -ايد -ترين -اينكه -ديگري -راه -هايي -بروز -همچنان -پاعين -كس -حدود -مختلف -مقابل -چيز -گيرد -ندارد -ضد -همچون -سازي -شان -مورد -باره -مرسي -خويش -برخوردار -چون -خارج -شش -هنوز -تحت -ضمن -هستيم -گفته -فكر -بسيار -پيش -براي -روزهاي -انكه -نخواهد -بالا -كل -وقتي -كي -چنين -كه -گيري -نيست -است -كجا -كند -نيز -يابد -بندي -حتي -توانند -عقب -خواست -كنند -بين -تمام -همه -ما -باشند -مثل -شد -اري -باشد -اره -طبق -بعد -اگر -صورت -غير -جاي -بيش -ريزي -اند -زيرا -چگونه -بار -لطفا -مي -درباره -من -ديده -همين -گذاري -برداري -علت -گذاشته -هم -فوق -نه -ها -شوند -اباد -همواره -هر -اول -خواهند -چهار -نام -امروز -مان -هاي -قبل -كنم -سعي -تازه -را -هستند -زير -جلوي -عنوان -بود diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fi.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fi.txt deleted file mode 100644 index addad798c4b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fi.txt +++ /dev/null @@ -1,95 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/finnish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - -| forms of BE - -olla -olen -olet -on -olemme -olette -ovat -ole | negative form - -oli -olisi -olisit -olisin -olisimme -olisitte -olisivat -olit -olin -olimme -olitte -olivat -ollut -olleet - -en | negation -et -ei -emme -ette -eivät - -|Nom Gen Acc Part Iness Elat Illat Adess Ablat Allat Ess Trans -minä minun minut minua minussa minusta minuun minulla minulta minulle | I -sinä sinun sinut sinua sinussa sinusta sinuun sinulla sinulta sinulle | you -hän hänen hänet häntä hänessä hänestä häneen hänellä häneltä hänelle | he she -me meidän meidät meitä meissä meistä meihin meillä meiltä meille | we -te teidän teidät teitä teissä teistä teihin teillä teiltä teille | you -he heidän heidät heitä heissä heistä heihin heillä heiltä heille | they - -tämä tämän tätä tässä tästä tähän tallä tältä tälle tänä täksi | this -tuo tuon tuotä tuossa tuosta tuohon tuolla tuolta tuolle tuona tuoksi | that -se sen sitä siinä siitä siihen sillä siltä sille sinä siksi | it -nämä näiden näitä näissä näistä näihin näillä näiltä näille näinä näiksi | these -nuo noiden noita noissa noista noihin noilla noilta noille noina noiksi | those -ne niiden niitä niissä niistä niihin niillä niiltä niille niinä niiksi | they - -kuka kenen kenet ketä kenessä kenestä keneen kenellä keneltä kenelle kenenä keneksi| who -ketkä keiden ketkä keitä keissä keistä keihin keillä keiltä keille keinä keiksi | (pl) -mikä minkä minkä mitä missä mistä mihin millä miltä mille minä miksi | which what -mitkä | (pl) - -joka jonka jota jossa josta johon jolla jolta jolle jona joksi | who which -jotka joiden joita joissa joista joihin joilla joilta joille joina joiksi | (pl) - -| conjunctions - -että | that -ja | and -jos | if -koska | because -kuin | than -mutta | but -niin | so -sekä | and -sillä | for -tai | or -vaan | but -vai | or -vaikka | although - - -| prepositions - -kanssa | with -mukaan | according to -noin | about -poikki | across -yli | over, across - -| other - -kun | when -niin | so -nyt | now -itse | self - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fr.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fr.txt deleted file mode 100644 index c00837ea939..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_fr.txt +++ /dev/null @@ -1,183 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/french/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A French stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -au | a + le -aux | a + les -avec | with -ce | this -ces | these -dans | with -de | of -des | de + les -du | de + le -elle | she -en | `of them' etc -et | and -eux | them -il | he -je | I -la | the -le | the -leur | their -lui | him -ma | my (fem) -mais | but -me | me -même | same; as in moi-même (myself) etc -mes | me (pl) -moi | me -mon | my (masc) -ne | not -nos | our (pl) -notre | our -nous | we -on | one -ou | where -par | by -pas | not -pour | for -qu | que before vowel -que | that -qui | who -sa | his, her (fem) -se | oneself -ses | his (pl) -son | his, her (masc) -sur | on -ta | thy (fem) -te | thee -tes | thy (pl) -toi | thee -ton | thy (masc) -tu | thou -un | a -une | a -vos | your (pl) -votre | your -vous | you - - | single letter forms - -c | c' -d | d' -j | j' -l | l' -à | to, at -m | m' -n | n' -s | s' -t | t' -y | there - - | forms of être (not including the infinitive): -été -étée -étées -étés -étant -suis -es -est -sommes -êtes -sont -serai -seras -sera -serons -serez -seront -serais -serait -serions -seriez -seraient -étais -était -étions -étiez -étaient -fus -fut -fûmes -fûtes -furent -sois -soit -soyons -soyez -soient -fusse -fusses -fût -fussions -fussiez -fussent - - | forms of avoir (not including the infinitive): -ayant -eu -eue -eues -eus -ai -as -avons -avez -ont -aurai -auras -aura -aurons -aurez -auront -aurais -aurait -aurions -auriez -auraient -avais -avait -avions -aviez -avaient -eut -eûmes -eûtes -eurent -aie -aies -ait -ayons -ayez -aient -eusse -eusses -eût -eussions -eussiez -eussent - - | Later additions (from Jean-Christophe Deschamps) -ceci | this -celà  | that -cet | this -cette | this -ici | here -ils | they -les | the (pl) -leurs | their (pl) -quel | which -quels | which -quelle | which -quelles | which -sans | without -soi | oneself - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ga.txt deleted file mode 100644 index 9ff88d747e5..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ga.txt +++ /dev/null @@ -1,110 +0,0 @@ - -a -ach -ag -agus -an -aon -ar -arna -as -b' -ba -beirt -bhúr -caoga -ceathair -ceathrar -chomh -chtó -chuig -chun -cois -céad -cúig -cúigear -d' -daichead -dar -de -deich -deichniúr -den -dhá -do -don -dtí -dá -dár -dó -faoi -faoin -faoina -faoinár -fara -fiche -gach -gan -go -gur -haon -hocht -i -iad -idir -in -ina -ins -inár -is -le -leis -lena -lenár -m' -mar -mo -mé -na -nach -naoi -naonúr -ná -ní -níor -nó -nócha -ocht -ochtar -os -roimh -sa -seacht -seachtar -seachtó -seasca -seisear -siad -sibh -sinn -sna -sé -sí -tar -thar -thú -triúr -trí -trína -trínár -tríocha -tú -um -ár -é -éis -í -ó -ón -óna -ónár diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_gl.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_gl.txt deleted file mode 100644 index d8760b12c14..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_gl.txt +++ /dev/null @@ -1,161 +0,0 @@ -# galican stopwords -a -aínda -alí -aquel -aquela -aquelas -aqueles -aquilo -aquí -ao -aos -as -así -á -ben -cando -che -co -coa -comigo -con -connosco -contigo -convosco -coas -cos -cun -cuns -cunha -cunhas -da -dalgunha -dalgunhas -dalgún -dalgúns -das -de -del -dela -delas -deles -desde -deste -do -dos -dun -duns -dunha -dunhas -e -el -ela -elas -eles -en -era -eran -esa -esas -ese -eses -esta -estar -estaba -está -están -este -estes -estiven -estou -eu -é -facer -foi -foron -fun -había -hai -iso -isto -la -las -lle -lles -lo -los -mais -me -meu -meus -min -miña -miñas -moi -na -nas -neste -nin -no -non -nos -nosa -nosas -noso -nosos -nós -nun -nunha -nuns -nunhas -o -os -ou -ó -ós -para -pero -pode -pois -pola -polas -polo -polos -por -que -se -senón -ser -seu -seus -sexa -sido -sobre -súa -súas -tamén -tan -te -ten -teñen -teño -ter -teu -teus -ti -tido -tiña -tiven -túa -túas -un -unha -unhas -uns -vos -vosa -vosas -voso -vosos -vós diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hi.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hi.txt deleted file mode 100644 index 86286bb083b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hi.txt +++ /dev/null @@ -1,235 +0,0 @@ -# Also see http://www.opensource.org/licenses/bsd-license.html -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# This file was created by Jacques Savoy and is distributed under the BSD license. -# Note: by default this file also contains forms normalized by HindiNormalizer -# for spelling variation (see section below), such that it can be used whether or -# not you enable that feature. When adding additional entries to this list, -# please add the normalized form as well. -अंदर -अत -अपना -अपनी -अपने -अभी -आदि -आप -इत्यादि -इन -इनका -इन्हीं -इन्हें -इन्हों -इस -इसका -इसकी -इसके -इसमें -इसी -इसे -उन -उनका -उनकी -उनके -उनको -उन्हीं -उन्हें -उन्हों -उस -उसके -उसी -उसे -एक -एवं -एस -ऐसे -और -कई -कर -करता -करते -करना -करने -करें -कहते -कहा -का -काफ़ी -कि -कितना -किन्हें -किन्हों -किया -किर -किस -किसी -किसे -की -कुछ -कुल -के -को -कोई -कौन -कौनसा -गया -घर -जब -जहाँ -जा -जितना -जिन -जिन्हें -जिन्हों -जिस -जिसे -जीधर -जैसा -जैसे -जो -तक -तब -तरह -तिन -तिन्हें -तिन्हों -तिस -तिसे -तो -था -थी -थे -दबारा -दिया -दुसरा -दूसरे -दो -द्वारा -न -नहीं -ना -निहायत -नीचे -ने -पर -पर -पहले -पूरा -पे -फिर -बनी -बही -बहुत -बाद -बाला -बिलकुल -भी -भीतर -मगर -मानो -मे -में -यदि -यह -यहाँ -यही -या -यिह -ये -रखें -रहा -रहे -ऱ्वासा -लिए -लिये -लेकिन -व -वर्ग -वह -वह -वहाँ -वहीं -वाले -वुह -वे -वग़ैरह -संग -सकता -सकते -सबसे -सभी -साथ -साबुत -साभ -सारा -से -सो -ही -हुआ -हुई -हुए -है -हैं -हो -होता -होती -होते -होना -होने -# additional normalized forms of the above -अपनि -जेसे -होति -सभि -तिंहों -इंहों -दवारा -इसि -किंहें -थि -उंहों -ओर -जिंहें -वहिं -अभि -बनि -हि -उंहिं -उंहें -हें -वगेरह -एसे -रवासा -कोन -निचे -काफि -उसि -पुरा -भितर -हे -बहि -वहां -कोइ -यहां -जिंहों -तिंहें -किसि -कइ -यहि -इंहिं -जिधर -इंहें -अदि -इतयादि -हुइ -कोनसा -इसकि -दुसरे -जहां -अप -किंहों -उनकि -भि -वरग -हुअ -जेसा -नहिं diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hu.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hu.txt deleted file mode 100644 index 1a96f1db6f2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hu.txt +++ /dev/null @@ -1,209 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/hungarian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - -| Hungarian stop word list -| prepared by Anna Tordai - -a -ahogy -ahol -aki -akik -akkor -alatt -által -általában -amely -amelyek -amelyekben -amelyeket -amelyet -amelynek -ami -amit -amolyan -amíg -amikor -át -abban -ahhoz -annak -arra -arról -az -azok -azon -azt -azzal -azért -aztán -azután -azonban -bár -be -belül -benne -cikk -cikkek -cikkeket -csak -de -e -eddig -egész -egy -egyes -egyetlen -egyéb -egyik -egyre -ekkor -el -elég -ellen -elő -először -előtt -első -én -éppen -ebben -ehhez -emilyen -ennek -erre -ez -ezt -ezek -ezen -ezzel -ezért -és -fel -felé -hanem -hiszen -hogy -hogyan -igen -így -illetve -ill. -ill -ilyen -ilyenkor -ison -ismét -itt -jó -jól -jobban -kell -kellett -keresztül -keressünk -ki -kívül -között -közül -legalább -lehet -lehetett -legyen -lenne -lenni -lesz -lett -maga -magát -majd -majd -már -más -másik -meg -még -mellett -mert -mely -melyek -mi -mit -míg -miért -milyen -mikor -minden -mindent -mindenki -mindig -mint -mintha -mivel -most -nagy -nagyobb -nagyon -ne -néha -nekem -neki -nem -néhány -nélkül -nincs -olyan -ott -össze -ő -ők -őket -pedig -persze -rá -s -saját -sem -semmi -sok -sokat -sokkal -számára -szemben -szerint -szinte -talán -tehát -teljes -tovább -továbbá -több -úgy -ugyanis -új -újabb -újra -után -utána -utolsó -vagy -vagyis -valaki -valami -valamint -való -vagyok -van -vannak -volt -voltam -voltak -voltunk -vissza -vele -viszont -volna diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hy.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hy.txt deleted file mode 100644 index 60c1c50fbc8..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_hy.txt +++ /dev/null @@ -1,46 +0,0 @@ -# example set of Armenian stopwords. -այդ -այլ -այն -այս -դու -դուք -եմ -են -ենք -ես -եք -է -էի -էին -էինք -էիր -էիք -էր -ըստ -թ -ի -ին -իսկ -իր -կամ -համար -հետ -հետո -մենք -մեջ -մի -ն -նա -նաև -նրա -նրանք -որ -որը -որոնք -որպես -ու -ում -պիտի -վրա -և diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_id.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_id.txt deleted file mode 100644 index 4617f83a5c5..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_id.txt +++ /dev/null @@ -1,359 +0,0 @@ -# from appendix D of: A Study of Stemming Effects on Information -# Retrieval in Bahasa Indonesia -ada -adanya -adalah -adapun -agak -agaknya -agar -akan -akankah -akhirnya -aku -akulah -amat -amatlah -anda -andalah -antar -diantaranya -antara -antaranya -diantara -apa -apaan -mengapa -apabila -apakah -apalagi -apatah -atau -ataukah -ataupun -bagai -bagaikan -sebagai -sebagainya -bagaimana -bagaimanapun -sebagaimana -bagaimanakah -bagi -bahkan -bahwa -bahwasanya -sebaliknya -banyak -sebanyak -beberapa -seberapa -begini -beginian -beginikah -beginilah -sebegini -begitu -begitukah -begitulah -begitupun -sebegitu -belum -belumlah -sebelum -sebelumnya -sebenarnya -berapa -berapakah -berapalah -berapapun -betulkah -sebetulnya -biasa -biasanya -bila -bilakah -bisa -bisakah -sebisanya -boleh -bolehkah -bolehlah -buat -bukan -bukankah -bukanlah -bukannya -cuma -percuma -dahulu -dalam -dan -dapat -dari -daripada -dekat -demi -demikian -demikianlah -sedemikian -dengan -depan -di -dia -dialah -dini -diri -dirinya -terdiri -dong -dulu -enggak -enggaknya -entah -entahlah -terhadap -terhadapnya -hal -hampir -hanya -hanyalah -harus -haruslah -harusnya -seharusnya -hendak -hendaklah -hendaknya -hingga -sehingga -ia -ialah -ibarat -ingin -inginkah -inginkan -ini -inikah -inilah -itu -itukah -itulah -jangan -jangankan -janganlah -jika -jikalau -juga -justru -kala -kalau -kalaulah -kalaupun -kalian -kami -kamilah -kamu -kamulah -kan -kapan -kapankah -kapanpun -dikarenakan -karena -karenanya -ke -kecil -kemudian -kenapa -kepada -kepadanya -ketika -seketika -khususnya -kini -kinilah -kiranya -sekiranya -kita -kitalah -kok -lagi -lagian -selagi -lah -lain -lainnya -melainkan -selaku -lalu -melalui -terlalu -lama -lamanya -selama -selama -selamanya -lebih -terlebih -bermacam -macam -semacam -maka -makanya -makin -malah -malahan -mampu -mampukah -mana -manakala -manalagi -masih -masihkah -semasih -masing -mau -maupun -semaunya -memang -mereka -merekalah -meski -meskipun -semula -mungkin -mungkinkah -nah -namun -nanti -nantinya -nyaris -oleh -olehnya -seorang -seseorang -pada -padanya -padahal -paling -sepanjang -pantas -sepantasnya -sepantasnyalah -para -pasti -pastilah -per -pernah -pula -pun -merupakan -rupanya -serupa -saat -saatnya -sesaat -saja -sajalah -saling -bersama -sama -sesama -sambil -sampai -sana -sangat -sangatlah -saya -sayalah -se -sebab -sebabnya -sebuah -tersebut -tersebutlah -sedang -sedangkan -sedikit -sedikitnya -segala -segalanya -segera -sesegera -sejak -sejenak -sekali -sekalian -sekalipun -sesekali -sekaligus -sekarang -sekarang -sekitar -sekitarnya -sela -selain -selalu -seluruh -seluruhnya -semakin -sementara -sempat -semua -semuanya -sendiri -sendirinya -seolah -seperti -sepertinya -sering -seringnya -serta -siapa -siapakah -siapapun -disini -disinilah -sini -sinilah -sesuatu -sesuatunya -suatu -sesudah -sesudahnya -sudah -sudahkah -sudahlah -supaya -tadi -tadinya -tak -tanpa -setelah -telah -tentang -tentu -tentulah -tentunya -tertentu -seterusnya -tapi -tetapi -setiap -tiap -setidaknya -tidak -tidakkah -tidaklah -toh -waduh -wah -wahai -sewaktu -walau -walaupun -wong -yaitu -yakni -yang diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_it.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_it.txt deleted file mode 100644 index 4cb5b0891b1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_it.txt +++ /dev/null @@ -1,301 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/italian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | An Italian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -ad | a (to) before vowel -al | a + il -allo | a + lo -ai | a + i -agli | a + gli -all | a + l' -agl | a + gl' -alla | a + la -alle | a + le -con | with -col | con + il -coi | con + i (forms collo, cogli etc are now very rare) -da | from -dal | da + il -dallo | da + lo -dai | da + i -dagli | da + gli -dall | da + l' -dagl | da + gll' -dalla | da + la -dalle | da + le -di | of -del | di + il -dello | di + lo -dei | di + i -degli | di + gli -dell | di + l' -degl | di + gl' -della | di + la -delle | di + le -in | in -nel | in + el -nello | in + lo -nei | in + i -negli | in + gli -nell | in + l' -negl | in + gl' -nella | in + la -nelle | in + le -su | on -sul | su + il -sullo | su + lo -sui | su + i -sugli | su + gli -sull | su + l' -sugl | su + gl' -sulla | su + la -sulle | su + le -per | through, by -tra | among -contro | against -io | I -tu | thou -lui | he -lei | she -noi | we -voi | you -loro | they -mio | my -mia | -miei | -mie | -tuo | -tua | -tuoi | thy -tue | -suo | -sua | -suoi | his, her -sue | -nostro | our -nostra | -nostri | -nostre | -vostro | your -vostra | -vostri | -vostre | -mi | me -ti | thee -ci | us, there -vi | you, there -lo | him, the -la | her, the -li | them -le | them, the -gli | to him, the -ne | from there etc -il | the -un | a -uno | a -una | a -ma | but -ed | and -se | if -perché | why, because -anche | also -come | how -dov | where (as dov') -dove | where -che | who, that -chi | who -cui | whom -non | not -più | more -quale | who, that -quanto | how much -quanti | -quanta | -quante | -quello | that -quelli | -quella | -quelle | -questo | this -questi | -questa | -queste | -si | yes -tutto | all -tutti | all - - | single letter forms: - -a | at -c | as c' for ce or ci -e | and -i | the -l | as l' -o | or - - | forms of avere, to have (not including the infinitive): - -ho -hai -ha -abbiamo -avete -hanno -abbia -abbiate -abbiano -avrò -avrai -avrà -avremo -avrete -avranno -avrei -avresti -avrebbe -avremmo -avreste -avrebbero -avevo -avevi -aveva -avevamo -avevate -avevano -ebbi -avesti -ebbe -avemmo -aveste -ebbero -avessi -avesse -avessimo -avessero -avendo -avuto -avuta -avuti -avute - - | forms of essere, to be (not including the infinitive): -sono -sei -è -siamo -siete -sia -siate -siano -sarò -sarai -sarà -saremo -sarete -saranno -sarei -saresti -sarebbe -saremmo -sareste -sarebbero -ero -eri -era -eravamo -eravate -erano -fui -fosti -fu -fummo -foste -furono -fossi -fosse -fossimo -fossero -essendo - - | forms of fare, to do (not including the infinitive, fa, fat-): -faccio -fai -facciamo -fanno -faccia -facciate -facciano -farò -farai -farà -faremo -farete -faranno -farei -faresti -farebbe -faremmo -fareste -farebbero -facevo -facevi -faceva -facevamo -facevate -facevano -feci -facesti -fece -facemmo -faceste -fecero -facessi -facesse -facessimo -facessero -facendo - - | forms of stare, to be (not including the infinitive): -sto -stai -sta -stiamo -stanno -stia -stiate -stiano -starò -starai -starà -staremo -starete -staranno -starei -staresti -starebbe -staremmo -stareste -starebbero -stavo -stavi -stava -stavamo -stavate -stavano -stetti -stesti -stette -stemmo -steste -stettero -stessi -stesse -stessimo -stessero -stando diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ja.txt deleted file mode 100644 index d4321be6b16..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ja.txt +++ /dev/null @@ -1,127 +0,0 @@ -# -# This file defines a stopword set for Japanese. -# -# This set is made up of hand-picked frequent terms from segmented Japanese Wikipedia. -# Punctuation characters and frequent kanji have mostly been left out. See LUCENE-3745 -# for frequency lists, etc. that can be useful for making your own set (if desired) -# -# Note that there is an overlap between these stopwords and the terms stopped when used -# in combination with the JapanesePartOfSpeechStopFilter. When editing this file, note -# that comments are not allowed on the same line as stopwords. -# -# Also note that stopping is done in a case-insensitive manner. Change your StopFilter -# configuration if you need case-sensitive stopping. Lastly, note that stopping is done -# using the same character width as the entries in this file. Since this StopFilter is -# normally done after a CJKWidthFilter in your chain, you would usually want your romaji -# entries to be in half-width and your kana entries to be in full-width. -# -の -に -は -を -た -が -で -て -と -し -れ -さ -ある -いる -も -する -から -な -こと -として -い -や -れる -など -なっ -ない -この -ため -その -あっ -よう -また -もの -という -あり -まで -られ -なる -へ -か -だ -これ -によって -により -おり -より -による -ず -なり -られる -において -ば -なかっ -なく -しかし -について -せ -だっ -その後 -できる -それ -う -ので -なお -のみ -でき -き -つ -における -および -いう -さらに -でも -ら -たり -その他 -に関する -たち -ます -ん -なら -に対して -特に -せる -及び -これら -とき -では -にて -ほか -ながら -うち -そして -とともに -ただし -かつて -それぞれ -または -お -ほど -ものの -に対する -ほとんど -と共に -といった -です -とも -ところ -ここ -##### End of file diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_lv.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_lv.txt deleted file mode 100644 index e21a23c06c3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_lv.txt +++ /dev/null @@ -1,172 +0,0 @@ -# Set of Latvian stopwords from A Stemming Algorithm for Latvian, Karlis Kreslins -# the original list of over 800 forms was refined: -# pronouns, adverbs, interjections were removed -# -# prepositions -aiz -ap -ar -apakš -ārpus -augšpus -bez -caur -dēļ -gar -iekš -iz -kopš -labad -lejpus -līdz -no -otrpus -pa -par -pār -pēc -pie -pirms -pret -priekš -starp -šaipus -uz -viņpus -virs -virspus -zem -apakšpus -# Conjunctions -un -bet -jo -ja -ka -lai -tomēr -tikko -turpretī -arī -kaut -gan -tādēļ -tā -ne -tikvien -vien -kā -ir -te -vai -kamēr -# Particles -ar -diezin -droši -diemžēl -nebūt -ik -it -taču -nu -pat -tiklab -iekšpus -nedz -tik -nevis -turpretim -jeb -iekam -iekām -iekāms -kolīdz -līdzko -tiklīdz -jebšu -tālab -tāpēc -nekā -itin -jā -jau -jel -nē -nezin -tad -tikai -vis -tak -iekams -vien -# modal verbs -būt -biju -biji -bija -bijām -bijāt -esmu -esi -esam -esat -būšu -būsi -būs -būsim -būsiet -tikt -tiku -tiki -tika -tikām -tikāt -tieku -tiec -tiek -tiekam -tiekat -tikšu -tiks -tiksim -tiksiet -tapt -tapi -tapāt -topat -tapšu -tapsi -taps -tapsim -tapsiet -kļūt -kļuvu -kļuvi -kļuva -kļuvām -kļuvāt -kļūstu -kļūsti -kļūst -kļūstam -kļūstat -kļūšu -kļūsi -kļūs -kļūsim -kļūsiet -# verbs -varēt -varēju -varējām -varēšu -varēsim -var -varēji -varējāt -varēsi -varēsiet -varat -varēja -varēs diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_nl.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_nl.txt deleted file mode 100644 index f4d61f5092c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_nl.txt +++ /dev/null @@ -1,117 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/dutch/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Dutch stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large sample of Dutch text. - - | Dutch stop words frequently exhibit homonym clashes. These are indicated - | clearly below. - -de | the -en | and -van | of, from -ik | I, the ego -te | (1) chez, at etc, (2) to, (3) too -dat | that, which -die | that, those, who, which -in | in, inside -een | a, an, one -hij | he -het | the, it -niet | not, nothing, naught -zijn | (1) to be, being, (2) his, one's, its -is | is -was | (1) was, past tense of all persons sing. of 'zijn' (to be) (2) wax, (3) the washing, (4) rise of river -op | on, upon, at, in, up, used up -aan | on, upon, to (as dative) -met | with, by -als | like, such as, when -voor | (1) before, in front of, (2) furrow -had | had, past tense all persons sing. of 'hebben' (have) -er | there -maar | but, only -om | round, about, for etc -hem | him -dan | then -zou | should/would, past tense all persons sing. of 'zullen' -of | or, whether, if -wat | what, something, anything -mijn | possessive and noun 'mine' -men | people, 'one' -dit | this -zo | so, thus, in this way -door | through by -over | over, across -ze | she, her, they, them -zich | oneself -bij | (1) a bee, (2) by, near, at -ook | also, too -tot | till, until -je | you -mij | me -uit | out of, from -der | Old Dutch form of 'van der' still found in surnames -daar | (1) there, (2) because -haar | (1) her, their, them, (2) hair -naar | (1) unpleasant, unwell etc, (2) towards, (3) as -heb | present first person sing. of 'to have' -hoe | how, why -heeft | present third person sing. of 'to have' -hebben | 'to have' and various parts thereof -deze | this -u | you -want | (1) for, (2) mitten, (3) rigging -nog | yet, still -zal | 'shall', first and third person sing. of verb 'zullen' (will) -me | me -zij | she, they -nu | now -ge | 'thou', still used in Belgium and south Netherlands -geen | none -omdat | because -iets | something, somewhat -worden | to become, grow, get -toch | yet, still -al | all, every, each -waren | (1) 'were' (2) to wander, (3) wares, (3) -veel | much, many -meer | (1) more, (2) lake -doen | to do, to make -toen | then, when -moet | noun 'spot/mote' and present form of 'to must' -ben | (1) am, (2) 'are' in interrogative second person singular of 'to be' -zonder | without -kan | noun 'can' and present form of 'to be able' -hun | their, them -dus | so, consequently -alles | all, everything, anything -onder | under, beneath -ja | yes, of course -eens | once, one day -hier | here -wie | who -werd | imperfect third person sing. of 'become' -altijd | always -doch | yet, but etc -wordt | present third person sing. of 'become' -wezen | (1) to be, (2) 'been' as in 'been fishing', (3) orphans -kunnen | to be able -ons | us/our -zelf | self -tegen | against, towards, at -na | after, near -reeds | already -wil | (1) present tense of 'want', (2) 'will', noun, (3) fender -kon | could; past tense of 'to be able' -niets | nothing -uw | your -iemand | somebody -geweest | been; past participle of 'be' -andere | other diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_no.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_no.txt deleted file mode 100644 index e76f36e69ed..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_no.txt +++ /dev/null @@ -1,192 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/norwegian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Norwegian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This stop word list is for the dominant bokmål dialect. Words unique - | to nynorsk are marked *. - - | Revised by Jan Bruusgaard , Jan 2005 - -og | and -i | in -jeg | I -det | it/this/that -at | to (w. inf.) -en | a/an -et | a/an -den | it/this/that -til | to -er | is/am/are -som | who/that -på | on -de | they / you(formal) -med | with -han | he -av | of -ikke | not -ikkje | not * -der | there -så | so -var | was/were -meg | me -seg | you -men | but -ett | one -har | have -om | about -vi | we -min | my -mitt | my -ha | have -hadde | had -hun | she -nå | now -over | over -da | when/as -ved | by/know -fra | from -du | you -ut | out -sin | your -dem | them -oss | us -opp | up -man | you/one -kan | can -hans | his -hvor | where -eller | or -hva | what -skal | shall/must -selv | self (reflective) -sjøl | self (reflective) -her | here -alle | all -vil | will -bli | become -ble | became -blei | became * -blitt | have become -kunne | could -inn | in -når | when -være | be -kom | come -noen | some -noe | some -ville | would -dere | you -som | who/which/that -deres | their/theirs -kun | only/just -ja | yes -etter | after -ned | down -skulle | should -denne | this -for | for/because -deg | you -si | hers/his -sine | hers/his -sitt | hers/his -mot | against -å | to -meget | much -hvorfor | why -dette | this -disse | these/those -uten | without -hvordan | how -ingen | none -din | your -ditt | your -blir | become -samme | same -hvilken | which -hvilke | which (plural) -sånn | such a -inni | inside/within -mellom | between -vår | our -hver | each -hvem | who -vors | us/ours -hvis | whose -både | both -bare | only/just -enn | than -fordi | as/because -før | before -mange | many -også | also -slik | just -vært | been -være | to be -båe | both * -begge | both -siden | since -dykk | your * -dykkar | yours * -dei | they * -deira | them * -deires | theirs * -deim | them * -di | your (fem.) * -då | as/when * -eg | I * -ein | a/an * -eit | a/an * -eitt | a/an * -elles | or * -honom | he * -hjå | at * -ho | she * -hoe | she * -henne | her -hennar | her/hers -hennes | hers -hoss | how * -hossen | how * -ikkje | not * -ingi | noone * -inkje | noone * -korleis | how * -korso | how * -kva | what/which * -kvar | where * -kvarhelst | where * -kven | who/whom * -kvi | why * -kvifor | why * -me | we * -medan | while * -mi | my * -mine | my * -mykje | much * -no | now * -nokon | some (masc./neut.) * -noka | some (fem.) * -nokor | some * -noko | some * -nokre | some * -si | his/hers * -sia | since * -sidan | since * -so | so * -somt | some * -somme | some * -um | about* -upp | up * -vere | be * -vore | was * -verte | become * -vort | become * -varte | became * -vart | became * - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_pt.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_pt.txt deleted file mode 100644 index 276c1b446f2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_pt.txt +++ /dev/null @@ -1,251 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/portuguese/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Portuguese stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | of, from -a | the; to, at; her -o | the; him -que | who, that -e | and -do | de + o -da | de + a -em | in -um | a -para | for - | é from SER -com | with -não | not, no -uma | a -os | the; them -no | em + o -se | himself etc -na | em + a -por | for -mais | more -as | the; them -dos | de + os -como | as, like -mas | but - | foi from SER -ao | a + o -ele | he -das | de + as - | tem from TER -à | a + a -seu | his -sua | her -ou | or - | ser from SER -quando | when -muito | much - | há from HAV -nos | em + os; us -já | already, now - | está from EST -eu | I -também | also -só | only, just -pelo | per + o -pela | per + a -até | up to -isso | that -ela | he -entre | between - | era from SER -depois | after -sem | without -mesmo | same -aos | a + os - | ter from TER -seus | his -quem | whom -nas | em + as -me | me -esse | that -eles | they - | estão from EST -você | you - | tinha from TER - | foram from SER -essa | that -num | em + um -nem | nor -suas | her -meu | my -às | a + as -minha | my - | têm from TER -numa | em + uma -pelos | per + os -elas | they - | havia from HAV - | seja from SER -qual | which - | será from SER -nós | we - | tenho from TER -lhe | to him, her -deles | of them -essas | those -esses | those -pelas | per + as -este | this - | fosse from SER -dele | of him - - | other words. There are many contractions such as naquele = em+aquele, - | mo = me+o, but they are rare. - | Indefinite article plural forms are also rare. - -tu | thou -te | thee -vocês | you (plural) -vos | you -lhes | to them -meus | my -minhas -teu | thy -tua -teus -tuas -nosso | our -nossa -nossos -nossas - -dela | of her -delas | of them - -esta | this -estes | these -estas | these -aquele | that -aquela | that -aqueles | those -aquelas | those -isto | this -aquilo | that - - | forms of estar, to be (not including the infinitive): -estou -está -estamos -estão -estive -esteve -estivemos -estiveram -estava -estávamos -estavam -estivera -estivéramos -esteja -estejamos -estejam -estivesse -estivéssemos -estivessem -estiver -estivermos -estiverem - - | forms of haver, to have (not including the infinitive): -hei -há -havemos -hão -houve -houvemos -houveram -houvera -houvéramos -haja -hajamos -hajam -houvesse -houvéssemos -houvessem -houver -houvermos -houverem -houverei -houverá -houveremos -houverão -houveria -houveríamos -houveriam - - | forms of ser, to be (not including the infinitive): -sou -somos -são -era -éramos -eram -fui -foi -fomos -foram -fora -fôramos -seja -sejamos -sejam -fosse -fôssemos -fossem -for -formos -forem -serei -será -seremos -serão -seria -seríamos -seriam - - | forms of ter, to have (not including the infinitive): -tenho -tem -temos -tém -tinha -tínhamos -tinham -tive -teve -tivemos -tiveram -tivera -tivéramos -tenha -tenhamos -tenham -tivesse -tivéssemos -tivessem -tiver -tivermos -tiverem -terei -terá -teremos -terão -teria -teríamos -teriam diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ro.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ro.txt deleted file mode 100644 index 4fdee90a5ba..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ro.txt +++ /dev/null @@ -1,233 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -acea -aceasta -această -aceea -acei -aceia -acel -acela -acele -acelea -acest -acesta -aceste -acestea -aceşti -aceştia -acolo -acum -ai -aia -aibă -aici -al -ăla -ale -alea -ălea -altceva -altcineva -am -ar -are -aş -aşadar -asemenea -asta -ăsta -astăzi -astea -ăstea -ăştia -asupra -aţi -au -avea -avem -aveţi -azi -bine -bucur -bună -ca -că -căci -când -care -cărei -căror -cărui -cât -câte -câţi -către -câtva -ce -cel -ceva -chiar -cînd -cine -cineva -cît -cîte -cîţi -cîtva -contra -cu -cum -cumva -curând -curînd -da -dă -dacă -dar -datorită -de -deci -deja -deoarece -departe -deşi -din -dinaintea -dintr -dintre -drept -după -ea -ei -el -ele -eram -este -eşti -eu -face -fără -fi -fie -fiecare -fii -fim -fiţi -iar -ieri -îi -îl -îmi -împotriva -în -înainte -înaintea -încât -încît -încotro -între -întrucât -întrucît -îţi -la -lângă -le -li -lîngă -lor -lui -mă -mâine -mea -mei -mele -mereu -meu -mi -mine -mult -multă -mulţi -ne -nicăieri -nici -nimeni -nişte -noastră -noastre -noi -noştri -nostru -nu -ori -oricând -oricare -oricât -orice -oricînd -oricine -oricît -oricum -oriunde -până -pe -pentru -peste -pînă -poate -pot -prea -prima -primul -prin -printr -sa -să -săi -sale -sau -său -se -şi -sînt -sîntem -sînteţi -spre -sub -sunt -suntem -sunteţi -ta -tăi -tale -tău -te -ţi -ţie -tine -toată -toate -tot -toţi -totuşi -tu -un -una -unde -undeva -unei -unele -uneori -unor -vă -vi -voastră -voastre -voi -voştri -vostru -vouă -vreo -vreun diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ru.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ru.txt deleted file mode 100644 index 64307693457..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_ru.txt +++ /dev/null @@ -1,241 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/russian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | a russian stop word list. comments begin with vertical bar. each stop - | word is at the start of a line. - - | this is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | letter `ё' is translated to `е'. - -и | and -в | in/into -во | alternative form -не | not -что | what/that -он | he -на | on/onto -я | i -с | from -со | alternative form -как | how -а | milder form of `no' (but) -то | conjunction and form of `that' -все | all -она | she -так | so, thus -его | him -но | but -да | yes/and -ты | thou -к | towards, by -у | around, chez -же | intensifier particle -вы | you -за | beyond, behind -бы | conditional/subj. particle -по | up to, along -только | only -ее | her -мне | to me -было | it was -вот | here is/are, particle -от | away from -меня | me -еще | still, yet, more -нет | no, there isnt/arent -о | about -из | out of -ему | to him -теперь | now -когда | when -даже | even -ну | so, well -вдруг | suddenly -ли | interrogative particle -если | if -уже | already, but homonym of `narrower' -или | or -ни | neither -быть | to be -был | he was -него | prepositional form of его -до | up to -вас | you accusative -нибудь | indef. suffix preceded by hyphen -опять | again -уж | already, but homonym of `adder' -вам | to you -сказал | he said -ведь | particle `after all' -там | there -потом | then -себя | oneself -ничего | nothing -ей | to her -может | usually with `быть' as `maybe' -они | they -тут | here -где | where -есть | there is/are -надо | got to, must -ней | prepositional form of ей -для | for -мы | we -тебя | thee -их | them, their -чем | than -была | she was -сам | self -чтоб | in order to -без | without -будто | as if -человек | man, person, one -чего | genitive form of `what' -раз | once -тоже | also -себе | to oneself -под | beneath -жизнь | life -будет | will be -ж | short form of intensifer particle `же' -тогда | then -кто | who -этот | this -говорил | was saying -того | genitive form of `that' -потому | for that reason -этого | genitive form of `this' -какой | which -совсем | altogether -ним | prepositional form of `его', `они' -здесь | here -этом | prepositional form of `этот' -один | one -почти | almost -мой | my -тем | instrumental/dative plural of `тот', `то' -чтобы | full form of `in order that' -нее | her (acc.) -кажется | it seems -сейчас | now -были | they were -куда | where to -зачем | why -сказать | to say -всех | all (acc., gen. preposn. plural) -никогда | never -сегодня | today -можно | possible, one can -при | by -наконец | finally -два | two -об | alternative form of `о', about -другой | another -хоть | even -после | after -над | above -больше | more -тот | that one (masc.) -через | across, in -эти | these -нас | us -про | about -всего | in all, only, of all -них | prepositional form of `они' (they) -какая | which, feminine -много | lots -разве | interrogative particle -сказала | she said -три | three -эту | this, acc. fem. sing. -моя | my, feminine -впрочем | moreover, besides -хорошо | good -свою | ones own, acc. fem. sing. -этой | oblique form of `эта', fem. `this' -перед | in front of -иногда | sometimes -лучше | better -чуть | a little -том | preposn. form of `that one' -нельзя | one must not -такой | such a one -им | to them -более | more -всегда | always -конечно | of course -всю | acc. fem. sing of `all' -между | between - - - | b: some paradigms - | - | personal pronouns - | - | я меня мне мной [мною] - | ты тебя тебе тобой [тобою] - | он его ему им [него, нему, ним] - | она ее эи ею [нее, нэи, нею] - | оно его ему им [него, нему, ним] - | - | мы нас нам нами - | вы вас вам вами - | они их им ими [них, ним, ними] - | - | себя себе собой [собою] - | - | demonstrative pronouns: этот (this), тот (that) - | - | этот эта это эти - | этого эты это эти - | этого этой этого этих - | этому этой этому этим - | этим этой этим [этою] этими - | этом этой этом этих - | - | тот та то те - | того ту то те - | того той того тех - | тому той тому тем - | тем той тем [тою] теми - | том той том тех - | - | determinative pronouns - | - | (a) весь (all) - | - | весь вся все все - | всего всю все все - | всего всей всего всех - | всему всей всему всем - | всем всей всем [всею] всеми - | всем всей всем всех - | - | (b) сам (himself etc) - | - | сам сама само сами - | самого саму само самих - | самого самой самого самих - | самому самой самому самим - | самим самой самим [самою] самими - | самом самой самом самих - | - | stems of verbs `to be', `to have', `to do' and modal - | - | быть бы буд быв есть суть - | име - | дел - | мог мож мочь - | уме - | хоч хот - | долж - | можн - | нужн - | нельзя - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_sv.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_sv.txt deleted file mode 100644 index 22bddfd8cb3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_sv.txt +++ /dev/null @@ -1,131 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/swedish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Swedish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | Swedish stop words occasionally exhibit homonym clashes. For example - | så = so, but also seed. These are indicated clearly below. - -och | and -det | it, this/that -att | to (with infinitive) -i | in, at -en | a -jag | I -hon | she -som | who, that -han | he -på | on -den | it, this/that -med | with -var | where, each -sig | him(self) etc -för | for -så | so (also: seed) -till | to -är | is -men | but -ett | a -om | if; around, about -hade | had -de | they, these/those -av | of -icke | not, no -mig | me -du | you -henne | her -då | then, when -sin | his -nu | now -har | have -inte | inte någon = no one -hans | his -honom | him -skulle | 'sake' -hennes | her -där | there -min | my -man | one (pronoun) -ej | nor -vid | at, by, on (also: vast) -kunde | could -något | some etc -från | from, off -ut | out -när | when -efter | after, behind -upp | up -vi | we -dem | them -vara | be -vad | what -över | over -än | than -dig | you -kan | can -sina | his -här | here -ha | have -mot | towards -alla | all -under | under (also: wonder) -någon | some etc -eller | or (else) -allt | all -mycket | much -sedan | since -ju | why -denna | this/that -själv | myself, yourself etc -detta | this/that -åt | to -utan | without -varit | was -hur | how -ingen | no -mitt | my -ni | you -bli | to be, become -blev | from bli -oss | us -din | thy -dessa | these/those -några | some etc -deras | their -blir | from bli -mina | my -samma | (the) same -vilken | who, that -er | you, your -sådan | such a -vår | our -blivit | from bli -dess | its -inom | within -mellan | between -sådant | such a -varför | why -varje | each -vilka | who, that -ditt | thy -vem | who -vilket | who, that -sitta | his -sådana | such a -vart | each -dina | thy -vars | whose -vårt | our -våra | our -ert | your -era | your -vilkas | whose - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_th.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_th.txt deleted file mode 100644 index 07f0fabe692..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_th.txt +++ /dev/null @@ -1,119 +0,0 @@ -# Thai stopwords from: -# "Opinion Detection in Thai Political News Columns -# Based on Subjectivity Analysis" -# Khampol Sukhum, Supot Nitsuwat, and Choochart Haruechaiyasak -ไว้ -ไม่ -ไป -ได้ -ให้ -ใน -โดย -แห่ง -แล้ว -และ -แรก -แบบ -แต่ -เอง -เห็น -เลย -เริ่ม -เรา -เมื่อ -เพื่อ -เพราะ -เป็นการ -เป็น -เปิดเผย -เปิด -เนื่องจาก -เดียวกัน -เดียว -เช่น -เฉพาะ -เคย -เข้า -เขา -อีก -อาจ -อะไร -ออก -อย่าง -อยู่ -อยาก -หาก -หลาย -หลังจาก -หลัง -หรือ -หนึ่ง -ส่วน -ส่ง -สุด -สําหรับ -ว่า -วัน -ลง -ร่วม -ราย -รับ -ระหว่าง -รวม -ยัง -มี -มาก -มา -พร้อม -พบ -ผ่าน -ผล -บาง -น่า -นี้ -นํา -นั้น -นัก -นอกจาก -ทุก -ที่สุด -ที่ -ทําให้ -ทํา -ทาง -ทั้งนี้ -ทั้ง -ถ้า -ถูก -ถึง -ต้อง -ต่างๆ -ต่าง -ต่อ -ตาม -ตั้งแต่ -ตั้ง -ด้าน -ด้วย -ดัง -ซึ่ง -ช่วง -จึง -จาก -จัด -จะ -คือ -ความ -ครั้ง -คง -ขึ้น -ของ -ขอ -ขณะ -ก่อน -ก็ -การ -กับ -กัน -กว่า -กล่าว diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_tr.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_tr.txt deleted file mode 100644 index 84d9408d4ea..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/stopwords_tr.txt +++ /dev/null @@ -1,212 +0,0 @@ -# Turkish stopwords from LUCENE-559 -# merged with the list from "Information Retrieval on Turkish Texts" -# (http://www.users.muohio.edu/canf/papers/JASIST2008offPrint.pdf) -acaba -altmış -altı -ama -ancak -arada -aslında -ayrıca -bana -bazı -belki -ben -benden -beni -benim -beri -beş -bile -bin -bir -birçok -biri -birkaç -birkez -birşey -birşeyi -biz -bize -bizden -bizi -bizim -böyle -böylece -bu -buna -bunda -bundan -bunlar -bunları -bunların -bunu -bunun -burada -çok -çünkü -da -daha -dahi -de -defa -değil -diğer -diye -doksan -dokuz -dolayı -dolayısıyla -dört -edecek -eden -ederek -edilecek -ediliyor -edilmesi -ediyor -eğer -elli -en -etmesi -etti -ettiği -ettiğini -gibi -göre -halen -hangi -hatta -hem -henüz -hep -hepsi -her -herhangi -herkesin -hiç -hiçbir -için -iki -ile -ilgili -ise -işte -itibaren -itibariyle -kadar -karşın -katrilyon -kendi -kendilerine -kendini -kendisi -kendisine -kendisini -kez -ki -kim -kimden -kime -kimi -kimse -kırk -milyar -milyon -mu -mü -mı -nasıl -ne -neden -nedenle -nerde -nerede -nereye -niye -niçin -o -olan -olarak -oldu -olduğu -olduğunu -olduklarını -olmadı -olmadığı -olmak -olması -olmayan -olmaz -olsa -olsun -olup -olur -olursa -oluyor -on -ona -ondan -onlar -onlardan -onları -onların -onu -onun -otuz -oysa -öyle -pek -rağmen -sadece -sanki -sekiz -seksen -sen -senden -seni -senin -siz -sizden -sizi -sizin -şey -şeyden -şeyi -şeyler -şöyle -şu -şuna -şunda -şundan -şunları -şunu -tarafından -trilyon -tüm -üç -üzere -var -vardı -ve -veya -ya -yani -yapacak -yapılan -yapılması -yapıyor -yapmak -yaptı -yaptığı -yaptığını -yaptıkları -yedi -yerine -yetmiş -yine -yirmi -yoksa -yüz -zaten diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/userdict_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/userdict_ja.txt deleted file mode 100644 index 6f0368e4d81..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/lang/userdict_ja.txt +++ /dev/null @@ -1,29 +0,0 @@ -# -# This is a sample user dictionary for Kuromoji (JapaneseTokenizer) -# -# Add entries to this file in order to override the statistical model in terms -# of segmentation, readings and part-of-speech tags. Notice that entries do -# not have weights since they are always used when found. This is by-design -# in order to maximize ease-of-use. -# -# Entries are defined using the following CSV format: -# , ... , ... , -# -# Notice that a single half-width space separates tokens and readings, and -# that the number tokens and readings must match exactly. -# -# Also notice that multiple entries with the same is undefined. -# -# Whitespace only lines are ignored. Comments are not allowed on entry lines. -# - -# Custom segmentation for kanji compounds -日本経済新聞,日本 経済 新聞,ニホン ケイザイ シンブン,カスタム名詞 -関西国際空港,関西 国際 空港,カンサイ コクサイ クウコウ,カスタム名詞 - -# Custom segmentation for compound katakana -トートバッグ,トート バッグ,トート バッグ,かずカナ名詞 -ショルダーバッグ,ショルダー バッグ,ショルダー バッグ,かずカナ名詞 - -# Custom reading for former sumo wrestler -朝青龍,朝青龍,アサショウリュウ,カスタム人名 diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/protwords.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/protwords.txt deleted file mode 100644 index 1dfc0abecbf..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/protwords.txt +++ /dev/null @@ -1,21 +0,0 @@ -# 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. - -#----------------------------------------------------------------------- -# Use a protected word file to protect against the stemmer reducing two -# unrelated words to the same base word. - -# Some non-words that normally won't be encountered, -# just to test that they won't be stemmed. -dontstems -zwhacky - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/schema.xml b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/schema.xml deleted file mode 100644 index 11c1f763bcc..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/schema.xml +++ /dev/null @@ -1,927 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - id - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/solrconfig.xml b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/solrconfig.xml deleted file mode 100644 index 05de0b628c9..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/solrconfig.xml +++ /dev/null @@ -1,1426 +0,0 @@ - - - - - - - - - ${tests.luceneMatchVersion:LATEST} - - - - - - - - - - - - - - - - - - - - - - - - - - - - ${solr.data.dir:} - - - - - - - - - - - - - - - - - - - - - 128 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ${solr.ulog.dir:} - - - - - ${solr.autoCommit.maxTime:60000} - false - - - - - - ${solr.autoSoftCommit.maxTime:1000} - - - - - - - - - - - - - - - - - - - - 1024 - - - - - - - - - - - - - - - - - - - - - - true - - - - - - 20 - - - 200 - - - - - - - - - - - - static firstSearcher warming in solrconfig.xml - - - - - - false - - - 4 - - - - - - - - - - - - - - - - - - - - - - - explicit - 10 - text - - - - - - - - - - - - - - explicit - json - true - text - - - - - - - - - textSpell - - - - - - default - name - solr.DirectSolrSpellChecker - - internal - - 0.5 - - 2 - - 1 - - 5 - - 4 - - 0.01 - - - - - - wordbreak - solr.WordBreakSolrSpellChecker - name - true - true - 10 - - - - - - - - - - - - - - - - text - - default - wordbreak - on - true - 10 - 5 - 5 - true - true - 10 - 5 - - - spellcheck - - - - - - - - - - text - true - - - tvComponent - - - - - - - - - default - - - org.carrot2.clustering.lingo.LingoClusteringAlgorithm - - - 20 - - - clustering/carrot2 - - - ENGLISH - - - stc - org.carrot2.clustering.stc.STCClusteringAlgorithm - - - - - - - true - default - true - - name - id - - features - - true - - - - false - - edismax - - text^0.5 features^1.0 name^1.2 sku^1.5 id^10.0 manu^1.1 cat^1.4 - - *:* - 10 - *,score - - - clustering - - - - - - - - - - true - - - terms - - - - - - - - string - elevate.xml - - - - - - explicit - text - - - elevator - - - - - - - - - - - 100 - - - - - - - - 70 - - 0.5 - - [-\w ,/\n\"']{20,200} - - - - - - - ]]> - ]]> - - - - - - - - - - - - - - - - - - - - - - - - ,, - ,, - ,, - ,, - ,]]> - ]]> - - - - - - 10 - .,!? - - - - - - - WORD - - - en - US - - - - - - - - - - - - - - - - - - - - - - text/plain; charset=UTF-8 - - - - - - - - - 5 - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/stopwords.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/stopwords.txt deleted file mode 100644 index ae1e83eeb3d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/stopwords.txt +++ /dev/null @@ -1,14 +0,0 @@ -# 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. diff --git a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/synonyms.txt b/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/synonyms.txt deleted file mode 100644 index 7f72128303b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/collection1/conf/synonyms.txt +++ /dev/null @@ -1,29 +0,0 @@ -# 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. - -#----------------------------------------------------------------------- -#some test synonym mappings unlikely to appear in real input text -aaafoo => aaabar -bbbfoo => bbbfoo bbbbar -cccfoo => cccbar cccbaz -fooaaa,baraaa,bazaaa - -# Some synonym groups specific to this example -GB,gib,gigabyte,gigabytes -MB,mib,megabyte,megabytes -Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming -#after us won't split it into two words. - -# Synonym mappings can be used for spelling correction too -pixima => pixma - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/currency.xml b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/currency.xml deleted file mode 100644 index 3a9c58afee8..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/currency.xml +++ /dev/null @@ -1,67 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/elevate.xml b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/elevate.xml deleted file mode 100644 index 2c09ebed669..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/elevate.xml +++ /dev/null @@ -1,42 +0,0 @@ - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_ca.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_ca.txt deleted file mode 100644 index 307a85f913d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_ca.txt +++ /dev/null @@ -1,8 +0,0 @@ -# Set of Catalan contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -l -m -n -s -t diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_fr.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_fr.txt deleted file mode 100644 index 722db588333..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_fr.txt +++ /dev/null @@ -1,9 +0,0 @@ -# Set of French contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -l -m -t -qu -n -s -j diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_ga.txt deleted file mode 100644 index 9ebe7fa349a..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -m -b diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_it.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_it.txt deleted file mode 100644 index cac04095372..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/contractions_it.txt +++ /dev/null @@ -1,23 +0,0 @@ -# Set of Italian contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -c -l -all -dall -dell -nell -sull -coll -pell -gl -agl -dagl -degl -negl -sugl -un -m -t -s -v -d diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/hyphenations_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/hyphenations_ga.txt deleted file mode 100644 index 4d2642cc5a3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/hyphenations_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish hyphenations for StopFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -h -n -t diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stemdict_nl.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stemdict_nl.txt deleted file mode 100644 index 441072971d3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stemdict_nl.txt +++ /dev/null @@ -1,6 +0,0 @@ -# Set of overrides for the dutch stemmer -# TODO: load this as a resource from the analyzer and sync it in build.xml -fiets fiets -bromfiets bromfiets -ei eier -kind kinder diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stoptags_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stoptags_ja.txt deleted file mode 100644 index 71b750845e3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stoptags_ja.txt +++ /dev/null @@ -1,420 +0,0 @@ -# -# This file defines a Japanese stoptag set for JapanesePartOfSpeechStopFilter. -# -# Any token with a part-of-speech tag that exactly matches those defined in this -# file are removed from the token stream. -# -# Set your own stoptags by uncommenting the lines below. Note that comments are -# not allowed on the same line as a stoptag. See LUCENE-3745 for frequency lists, -# etc. that can be useful for building you own stoptag set. -# -# The entire possible tagset is provided below for convenience. -# -##### -# noun: unclassified nouns -#名詞 -# -# noun-common: Common nouns or nouns where the sub-classification is undefined -#名詞-一般 -# -# noun-proper: Proper nouns where the sub-classification is undefined -#名詞-固有名詞 -# -# noun-proper-misc: miscellaneous proper nouns -#名詞-固有名詞-一般 -# -# noun-proper-person: Personal names where the sub-classification is undefined -#名詞-固有名詞-人名 -# -# noun-proper-person-misc: names that cannot be divided into surname and -# given name; foreign names; names where the surname or given name is unknown. -# e.g. お市の方 -#名詞-固有名詞-人名-一般 -# -# noun-proper-person-surname: Mainly Japanese surnames. -# e.g. 山田 -#名詞-固有名詞-人名-姓 -# -# noun-proper-person-given_name: Mainly Japanese given names. -# e.g. 太郎 -#名詞-固有名詞-人名-名 -# -# noun-proper-organization: Names representing organizations. -# e.g. 通産省, NHK -#名詞-固有名詞-組織 -# -# noun-proper-place: Place names where the sub-classification is undefined -#名詞-固有名詞-地域 -# -# noun-proper-place-misc: Place names excluding countries. -# e.g. アジア, バルセロナ, 京都 -#名詞-固有名詞-地域-一般 -# -# noun-proper-place-country: Country names. -# e.g. 日本, オーストラリア -#名詞-固有名詞-地域-国 -# -# noun-pronoun: Pronouns where the sub-classification is undefined -#名詞-代名詞 -# -# noun-pronoun-misc: miscellaneous pronouns: -# e.g. それ, ここ, あいつ, あなた, あちこち, いくつ, どこか, なに, みなさん, みんな, わたくし, われわれ -#名詞-代名詞-一般 -# -# noun-pronoun-contraction: Spoken language contraction made by combining a -# pronoun and the particle 'wa'. -# e.g. ありゃ, こりゃ, こりゃあ, そりゃ, そりゃあ -#名詞-代名詞-縮約 -# -# noun-adverbial: Temporal nouns such as names of days or months that behave -# like adverbs. Nouns that represent amount or ratios and can be used adverbially, -# e.g. 金曜, 一月, 午後, 少量 -#名詞-副詞可能 -# -# noun-verbal: Nouns that take arguments with case and can appear followed by -# 'suru' and related verbs (する, できる, なさる, くださる) -# e.g. インプット, 愛着, 悪化, 悪戦苦闘, 一安心, 下取り -#名詞-サ変接続 -# -# noun-adjective-base: The base form of adjectives, words that appear before な ("na") -# e.g. 健康, 安易, 駄目, だめ -#名詞-形容動詞語幹 -# -# noun-numeric: Arabic numbers, Chinese numerals, and counters like 何 (回), 数. -# e.g. 0, 1, 2, 何, 数, 幾 -#名詞-数 -# -# noun-affix: noun affixes where the sub-classification is undefined -#名詞-非自立 -# -# noun-affix-misc: Of adnominalizers, the case-marker の ("no"), and words that -# attach to the base form of inflectional words, words that cannot be classified -# into any of the other categories below. This category includes indefinite nouns. -# e.g. あかつき, 暁, かい, 甲斐, 気, きらい, 嫌い, くせ, 癖, こと, 事, ごと, 毎, しだい, 次第, -# 順, せい, 所為, ついで, 序で, つもり, 積もり, 点, どころ, の, はず, 筈, はずみ, 弾み, -# 拍子, ふう, ふり, 振り, ほう, 方, 旨, もの, 物, 者, ゆえ, 故, ゆえん, 所以, わけ, 訳, -# わり, 割り, 割, ん-口語/, もん-口語/ -#名詞-非自立-一般 -# -# noun-affix-adverbial: noun affixes that that can behave as adverbs. -# e.g. あいだ, 間, あげく, 挙げ句, あと, 後, 余り, 以外, 以降, 以後, 以上, 以前, 一方, うえ, -# 上, うち, 内, おり, 折り, かぎり, 限り, きり, っきり, 結果, ころ, 頃, さい, 際, 最中, さなか, -# 最中, じたい, 自体, たび, 度, ため, 為, つど, 都度, とおり, 通り, とき, 時, ところ, 所, -# とたん, 途端, なか, 中, のち, 後, ばあい, 場合, 日, ぶん, 分, ほか, 他, まえ, 前, まま, -# 儘, 侭, みぎり, 矢先 -#名詞-非自立-副詞可能 -# -# noun-affix-aux: noun affixes treated as 助動詞 ("auxiliary verb") in school grammars -# with the stem よう(だ) ("you(da)"). -# e.g. よう, やう, 様 (よう) -#名詞-非自立-助動詞語幹 -# -# noun-affix-adjective-base: noun affixes that can connect to the indeclinable -# connection form な (aux "da"). -# e.g. みたい, ふう -#名詞-非自立-形容動詞語幹 -# -# noun-special: special nouns where the sub-classification is undefined. -#名詞-特殊 -# -# noun-special-aux: The そうだ ("souda") stem form that is used for reporting news, is -# treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the base -# form of inflectional words. -# e.g. そう -#名詞-特殊-助動詞語幹 -# -# noun-suffix: noun suffixes where the sub-classification is undefined. -#名詞-接尾 -# -# noun-suffix-misc: Of the nouns or stem forms of other parts of speech that connect -# to ガル or タイ and can combine into compound nouns, words that cannot be classified into -# any of the other categories below. In general, this category is more inclusive than -# 接尾語 ("suffix") and is usually the last element in a compound noun. -# e.g. おき, かた, 方, 甲斐 (がい), がかり, ぎみ, 気味, ぐるみ, (~した) さ, 次第, 済 (ず) み, -# よう, (でき)っこ, 感, 観, 性, 学, 類, 面, 用 -#名詞-接尾-一般 -# -# noun-suffix-person: Suffixes that form nouns and attach to person names more often -# than other nouns. -# e.g. 君, 様, 著 -#名詞-接尾-人名 -# -# noun-suffix-place: Suffixes that form nouns and attach to place names more often -# than other nouns. -# e.g. 町, 市, 県 -#名詞-接尾-地域 -# -# noun-suffix-verbal: Of the suffixes that attach to nouns and form nouns, those that -# can appear before スル ("suru"). -# e.g. 化, 視, 分け, 入り, 落ち, 買い -#名詞-接尾-サ変接続 -# -# noun-suffix-aux: The stem form of そうだ (様態) that is used to indicate conditions, -# is treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the -# conjunctive form of inflectional words. -# e.g. そう -#名詞-接尾-助動詞語幹 -# -# noun-suffix-adjective-base: Suffixes that attach to other nouns or the conjunctive -# form of inflectional words and appear before the copula だ ("da"). -# e.g. 的, げ, がち -#名詞-接尾-形容動詞語幹 -# -# noun-suffix-adverbial: Suffixes that attach to other nouns and can behave as adverbs. -# e.g. 後 (ご), 以後, 以降, 以前, 前後, 中, 末, 上, 時 (じ) -#名詞-接尾-副詞可能 -# -# noun-suffix-classifier: Suffixes that attach to numbers and form nouns. This category -# is more inclusive than 助数詞 ("classifier") and includes common nouns that attach -# to numbers. -# e.g. 個, つ, 本, 冊, パーセント, cm, kg, カ月, か国, 区画, 時間, 時半 -#名詞-接尾-助数詞 -# -# noun-suffix-special: Special suffixes that mainly attach to inflecting words. -# e.g. (楽し) さ, (考え) 方 -#名詞-接尾-特殊 -# -# noun-suffix-conjunctive: Nouns that behave like conjunctions and join two words -# together. -# e.g. (日本) 対 (アメリカ), 対 (アメリカ), (3) 対 (5), (女優) 兼 (主婦) -#名詞-接続詞的 -# -# noun-verbal_aux: Nouns that attach to the conjunctive particle て ("te") and are -# semantically verb-like. -# e.g. ごらん, ご覧, 御覧, 頂戴 -#名詞-動詞非自立的 -# -# noun-quotation: text that cannot be segmented into words, proverbs, Chinese poetry, -# dialects, English, etc. Currently, the only entry for 名詞 引用文字列 ("noun quotation") -# is いわく ("iwaku"). -#名詞-引用文字列 -# -# noun-nai_adjective: Words that appear before the auxiliary verb ない ("nai") and -# behave like an adjective. -# e.g. 申し訳, 仕方, とんでも, 違い -#名詞-ナイ形容詞語幹 -# -##### -# prefix: unclassified prefixes -#接頭詞 -# -# prefix-nominal: Prefixes that attach to nouns (including adjective stem forms) -# excluding numerical expressions. -# e.g. お (水), 某 (氏), 同 (社), 故 (~氏), 高 (品質), お (見事), ご (立派) -#接頭詞-名詞接続 -# -# prefix-verbal: Prefixes that attach to the imperative form of a verb or a verb -# in conjunctive form followed by なる/なさる/くださる. -# e.g. お (読みなさい), お (座り) -#接頭詞-動詞接続 -# -# prefix-adjectival: Prefixes that attach to adjectives. -# e.g. お (寒いですねえ), バカ (でかい) -#接頭詞-形容詞接続 -# -# prefix-numerical: Prefixes that attach to numerical expressions. -# e.g. 約, およそ, 毎時 -#接頭詞-数接続 -# -##### -# verb: unclassified verbs -#動詞 -# -# verb-main: -#動詞-自立 -# -# verb-auxiliary: -#動詞-非自立 -# -# verb-suffix: -#動詞-接尾 -# -##### -# adjective: unclassified adjectives -#形容詞 -# -# adjective-main: -#形容詞-自立 -# -# adjective-auxiliary: -#形容詞-非自立 -# -# adjective-suffix: -#形容詞-接尾 -# -##### -# adverb: unclassified adverbs -#副詞 -# -# adverb-misc: Words that can be segmented into one unit and where adnominal -# modification is not possible. -# e.g. あいかわらず, 多分 -#副詞-一般 -# -# adverb-particle_conjunction: Adverbs that can be followed by の, は, に, -# な, する, だ, etc. -# e.g. こんなに, そんなに, あんなに, なにか, なんでも -#副詞-助詞類接続 -# -##### -# adnominal: Words that only have noun-modifying forms. -# e.g. この, その, あの, どの, いわゆる, なんらかの, 何らかの, いろんな, こういう, そういう, ああいう, -# どういう, こんな, そんな, あんな, どんな, 大きな, 小さな, おかしな, ほんの, たいした, -# 「(, も) さる (ことながら)」, 微々たる, 堂々たる, 単なる, いかなる, 我が」「同じ, 亡き -#連体詞 -# -##### -# conjunction: Conjunctions that can occur independently. -# e.g. が, けれども, そして, じゃあ, それどころか -接続詞 -# -##### -# particle: unclassified particles. -助詞 -# -# particle-case: case particles where the subclassification is undefined. -助詞-格助詞 -# -# particle-case-misc: Case particles. -# e.g. から, が, で, と, に, へ, より, を, の, にて -助詞-格助詞-一般 -# -# particle-case-quote: the "to" that appears after nouns, a person’s speech, -# quotation marks, expressions of decisions from a meeting, reasons, judgements, -# conjectures, etc. -# e.g. ( だ) と (述べた.), ( である) と (して執行猶予...) -助詞-格助詞-引用 -# -# particle-case-compound: Compounds of particles and verbs that mainly behave -# like case particles. -# e.g. という, といった, とかいう, として, とともに, と共に, でもって, にあたって, に当たって, に当って, -# にあたり, に当たり, に当り, に当たる, にあたる, において, に於いて,に於て, における, に於ける, -# にかけ, にかけて, にかんし, に関し, にかんして, に関して, にかんする, に関する, に際し, -# に際して, にしたがい, に従い, に従う, にしたがって, に従って, にたいし, に対し, にたいして, -# に対して, にたいする, に対する, について, につき, につけ, につけて, につれ, につれて, にとって, -# にとり, にまつわる, によって, に依って, に因って, により, に依り, に因り, による, に依る, に因る, -# にわたって, にわたる, をもって, を以って, を通じ, を通じて, を通して, をめぐって, をめぐり, をめぐる, -# って-口語/, ちゅう-関西弁「という」/, (何) ていう (人)-口語/, っていう-口語/, といふ, とかいふ -助詞-格助詞-連語 -# -# particle-conjunctive: -# e.g. から, からには, が, けれど, けれども, けど, し, つつ, て, で, と, ところが, どころか, とも, ども, -# ながら, なり, ので, のに, ば, ものの, や ( した), やいなや, (ころん) じゃ(いけない)-口語/, -# (行っ) ちゃ(いけない)-口語/, (言っ) たって (しかたがない)-口語/, (それがなく)ったって (平気)-口語/ -助詞-接続助詞 -# -# particle-dependency: -# e.g. こそ, さえ, しか, すら, は, も, ぞ -助詞-係助詞 -# -# particle-adverbial: -# e.g. がてら, かも, くらい, 位, ぐらい, しも, (学校) じゃ(これが流行っている)-口語/, -# (それ)じゃあ (よくない)-口語/, ずつ, (私) なぞ, など, (私) なり (に), (先生) なんか (大嫌い)-口語/, -# (私) なんぞ, (先生) なんて (大嫌い)-口語/, のみ, だけ, (私) だって-口語/, だに, -# (彼)ったら-口語/, (お茶) でも (いかが), 等 (とう), (今後) とも, ばかり, ばっか-口語/, ばっかり-口語/, -# ほど, 程, まで, 迄, (誰) も (が)([助詞-格助詞] および [助詞-係助詞] の前に位置する「も」) -助詞-副助詞 -# -# particle-interjective: particles with interjective grammatical roles. -# e.g. (松島) や -助詞-間投助詞 -# -# particle-coordinate: -# e.g. と, たり, だの, だり, とか, なり, や, やら -助詞-並立助詞 -# -# particle-final: -# e.g. かい, かしら, さ, ぜ, (だ)っけ-口語/, (とまってる) で-方言/, な, ナ, なあ-口語/, ぞ, ね, ネ, -# ねぇ-口語/, ねえ-口語/, ねん-方言/, の, のう-口語/, や, よ, ヨ, よぉ-口語/, わ, わい-口語/ -助詞-終助詞 -# -# particle-adverbial/conjunctive/final: The particle "ka" when unknown whether it is -# adverbial, conjunctive, or sentence final. For example: -# (a) 「A か B か」. Ex:「(国内で運用する) か,(海外で運用する) か (.)」 -# (b) Inside an adverb phrase. Ex:「(幸いという) か (, 死者はいなかった.)」 -# 「(祈りが届いたせい) か (, 試験に合格した.)」 -# (c) 「かのように」. Ex:「(何もなかった) か (のように振る舞った.)」 -# e.g. か -助詞-副助詞/並立助詞/終助詞 -# -# particle-adnominalizer: The "no" that attaches to nouns and modifies -# non-inflectional words. -助詞-連体化 -# -# particle-adnominalizer: The "ni" and "to" that appear following nouns and adverbs -# that are giongo, giseigo, or gitaigo. -# e.g. に, と -助詞-副詞化 -# -# particle-special: A particle that does not fit into one of the above classifications. -# This includes particles that are used in Tanka, Haiku, and other poetry. -# e.g. かな, けむ, ( しただろう) に, (あんた) にゃ(わからん), (俺) ん (家) -助詞-特殊 -# -##### -# auxiliary-verb: -助動詞 -# -##### -# interjection: Greetings and other exclamations. -# e.g. おはよう, おはようございます, こんにちは, こんばんは, ありがとう, どうもありがとう, ありがとうございます, -# いただきます, ごちそうさま, さよなら, さようなら, はい, いいえ, ごめん, ごめんなさい -#感動詞 -# -##### -# symbol: unclassified Symbols. -記号 -# -# symbol-misc: A general symbol not in one of the categories below. -# e.g. [○◎@$〒→+] -記号-一般 -# -# symbol-comma: Commas -# e.g. [,、] -記号-読点 -# -# symbol-period: Periods and full stops. -# e.g. [..。] -記号-句点 -# -# symbol-space: Full-width whitespace. -記号-空白 -# -# symbol-open_bracket: -# e.g. [({‘“『【] -記号-括弧開 -# -# symbol-close_bracket: -# e.g. [)}’”』」】] -記号-括弧閉 -# -# symbol-alphabetic: -#記号-アルファベット -# -##### -# other: unclassified other -#その他 -# -# other-interjection: Words that are hard to classify as noun-suffixes or -# sentence-final particles. -# e.g. (だ)ァ -その他-間投 -# -##### -# filler: Aizuchi that occurs during a conversation or sounds inserted as filler. -# e.g. あの, うんと, えと -フィラー -# -##### -# non-verbal: non-verbal sound. -非言語音 -# -##### -# fragment: -#語断片 -# -##### -# unknown: unknown part of speech. -#未知語 -# -##### End of file diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ar.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ar.txt deleted file mode 100644 index 046829db6a2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ar.txt +++ /dev/null @@ -1,125 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Cleaned on October 11, 2009 (not normalized, so use before normalization) -# This means that when modifying this list, you might need to add some -# redundant entries, for example containing forms with both أ and ا -من -ومن -منها -منه -في -وفي -فيها -فيه -و -ف -ثم -او -أو -ب -بها -به -ا -أ -اى -اي -أي -أى -لا -ولا -الا -ألا -إلا -لكن -ما -وما -كما -فما -عن -مع -اذا -إذا -ان -أن -إن -انها -أنها -إنها -انه -أنه -إنه -بان -بأن -فان -فأن -وان -وأن -وإن -التى -التي -الذى -الذي -الذين -الى -الي -إلى -إلي -على -عليها -عليه -اما -أما -إما -ايضا -أيضا -كل -وكل -لم -ولم -لن -ولن -هى -هي -هو -وهى -وهي -وهو -فهى -فهي -فهو -انت -أنت -لك -لها -له -هذه -هذا -تلك -ذلك -هناك -كانت -كان -يكون -تكون -وكانت -وكان -غير -بعض -قد -نحو -بين -بينما -منذ -ضمن -حيث -الان -الآن -خلال -بعد -قبل -حتى -عند -عندما -لدى -جميع diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_bg.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_bg.txt deleted file mode 100644 index 1ae4ba2ae38..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_bg.txt +++ /dev/null @@ -1,193 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -а -аз -ако -ала -бе -без -беше -би -бил -била -били -било -близо -бъдат -бъде -бяха -в -вас -ваш -ваша -вероятно -вече -взема -ви -вие -винаги -все -всеки -всички -всичко -всяка -във -въпреки -върху -г -ги -главно -го -д -да -дали -до -докато -докога -дори -досега -доста -е -едва -един -ето -за -зад -заедно -заради -засега -затова -защо -защото -и -из -или -им -има -имат -иска -й -каза -как -каква -какво -както -какъв -като -кога -когато -което -които -кой -който -колко -която -къде -където -към -ли -м -ме -между -мен -ми -мнозина -мога -могат -може -моля -момента -му -н -на -над -назад -най -направи -напред -например -нас -не -него -нея -ни -ние -никой -нито -но -някои -някой -няма -обаче -около -освен -особено -от -отгоре -отново -още -пак -по -повече -повечето -под -поне -поради -после -почти -прави -пред -преди -през -при -пък -първо -с -са -само -се -сега -си -скоро -след -сме -според -сред -срещу -сте -съм -със -също -т -тази -така -такива -такъв -там -твой -те -тези -ти -тн -то -това -тогава -този -той -толкова -точно -трябва -тук -тъй -тя -тях -у -харесва -ч -че -често -чрез -ще -щом -я diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ca.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ca.txt deleted file mode 100644 index 3da65deafe1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ca.txt +++ /dev/null @@ -1,220 +0,0 @@ -# Catalan stopwords from http://github.com/vcl/cue.language (Apache 2 Licensed) -a -abans -ací -ah -així -això -al -als -aleshores -algun -alguna -algunes -alguns -alhora -allà -allí -allò -altra -altre -altres -amb -ambdós -ambdues -apa -aquell -aquella -aquelles -aquells -aquest -aquesta -aquestes -aquests -aquí -baix -cada -cadascú -cadascuna -cadascunes -cadascuns -com -contra -d'un -d'una -d'unes -d'uns -dalt -de -del -dels -des -després -dins -dintre -donat -doncs -durant -e -eh -el -els -em -en -encara -ens -entre -érem -eren -éreu -es -és -esta -està -estàvem -estaven -estàveu -esteu -et -etc -ets -fins -fora -gairebé -ha -han -has -havia -he -hem -heu -hi -ho -i -igual -iguals -ja -l'hi -la -les -li -li'n -llavors -m'he -ma -mal -malgrat -mateix -mateixa -mateixes -mateixos -me -mentre -més -meu -meus -meva -meves -molt -molta -moltes -molts -mon -mons -n'he -n'hi -ne -ni -no -nogensmenys -només -nosaltres -nostra -nostre -nostres -o -oh -oi -on -pas -pel -pels -per -però -perquè -poc -poca -pocs -poques -potser -propi -qual -quals -quan -quant -que -què -quelcom -qui -quin -quina -quines -quins -s'ha -s'han -sa -semblant -semblants -ses -seu -seus -seva -seva -seves -si -sobre -sobretot -sóc -solament -sols -son -són -sons -sota -sou -t'ha -t'han -t'he -ta -tal -també -tampoc -tan -tant -tanta -tantes -teu -teus -teva -teves -ton -tons -tot -tota -totes -tots -un -una -unes -uns -us -va -vaig -vam -van -vas -veu -vosaltres -vostra -vostre -vostres diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_cz.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_cz.txt deleted file mode 100644 index 53c6097dac7..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_cz.txt +++ /dev/null @@ -1,172 +0,0 @@ -a -s -k -o -i -u -v -z -dnes -cz -tímto -budeš -budem -byli -jseš -můj -svým -ta -tomto -tohle -tuto -tyto -jej -zda -proč -máte -tato -kam -tohoto -kdo -kteří -mi -nám -tom -tomuto -mít -nic -proto -kterou -byla -toho -protože -asi -ho -naši -napište -re -což -tím -takže -svých -její -svými -jste -aj -tu -tedy -teto -bylo -kde -ke -pravé -ji -nad -nejsou -či -pod -téma -mezi -přes -ty -pak -vám -ani -když -však -neg -jsem -tento -článku -články -aby -jsme -před -pta -jejich -byl -ještě -až -bez -také -pouze -první -vaše -která -nás -nový -tipy -pokud -může -strana -jeho -své -jiné -zprávy -nové -není -vás -jen -podle -zde -už -být -více -bude -již -než -který -by -které -co -nebo -ten -tak -má -při -od -po -jsou -jak -další -ale -si -se -ve -to -jako -za -zpět -ze -do -pro -je -na -atd -atp -jakmile -přičemž -já -on -ona -ono -oni -ony -my -vy -jí -ji -mě -mne -jemu -tomu -těm -těmu -němu -němuž -jehož -jíž -jelikož -jež -jakož -načež diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_da.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_da.txt deleted file mode 100644 index a3ff5fe122c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_da.txt +++ /dev/null @@ -1,108 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/danish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Danish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - -og | and -i | in -jeg | I -det | that (dem. pronoun)/it (pers. pronoun) -at | that (in front of a sentence)/to (with infinitive) -en | a/an -den | it (pers. pronoun)/that (dem. pronoun) -til | to/at/for/until/against/by/of/into, more -er | present tense of "to be" -som | who, as -på | on/upon/in/on/at/to/after/of/with/for, on -de | they -med | with/by/in, along -han | he -af | of/by/from/off/for/in/with/on, off -for | at/for/to/from/by/of/ago, in front/before, because -ikke | not -der | who/which, there/those -var | past tense of "to be" -mig | me/myself -sig | oneself/himself/herself/itself/themselves -men | but -et | a/an/one, one (number), someone/somebody/one -har | present tense of "to have" -om | round/about/for/in/a, about/around/down, if -vi | we -min | my -havde | past tense of "to have" -ham | him -hun | she -nu | now -over | over/above/across/by/beyond/past/on/about, over/past -da | then, when/as/since -fra | from/off/since, off, since -du | you -ud | out -sin | his/her/its/one's -dem | them -os | us/ourselves -op | up -man | you/one -hans | his -hvor | where -eller | or -hvad | what -skal | must/shall etc. -selv | myself/youself/herself/ourselves etc., even -her | here -alle | all/everyone/everybody etc. -vil | will (verb) -blev | past tense of "to stay/to remain/to get/to become" -kunne | could -ind | in -når | when -være | present tense of "to be" -dog | however/yet/after all -noget | something -ville | would -jo | you know/you see (adv), yes -deres | their/theirs -efter | after/behind/according to/for/by/from, later/afterwards -ned | down -skulle | should -denne | this -end | than -dette | this -mit | my/mine -også | also -under | under/beneath/below/during, below/underneath -have | have -dig | you -anden | other -hende | her -mine | my -alt | everything -meget | much/very, plenty of -sit | his, her, its, one's -sine | his, her, its, one's -vor | our -mod | against -disse | these -hvis | if -din | your/yours -nogle | some -hos | by/at -blive | be/become -mange | many -ad | by/through -bliver | present tense of "to be/to become" -hendes | her/hers -været | be -thi | for (conj) -jer | you -sådan | such, like this/like that diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_de.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_de.txt deleted file mode 100644 index f7703841887..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_de.txt +++ /dev/null @@ -1,292 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/german/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A German stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | The number of forms in this list is reduced significantly by passing it - | through the German stemmer. - - -aber | but - -alle | all -allem -allen -aller -alles - -als | than, as -also | so -am | an + dem -an | at - -ander | other -andere -anderem -anderen -anderer -anderes -anderm -andern -anderr -anders - -auch | also -auf | on -aus | out of -bei | by -bin | am -bis | until -bist | art -da | there -damit | with it -dann | then - -der | the -den -des -dem -die -das - -daß | that - -derselbe | the same -derselben -denselben -desselben -demselben -dieselbe -dieselben -dasselbe - -dazu | to that - -dein | thy -deine -deinem -deinen -deiner -deines - -denn | because - -derer | of those -dessen | of him - -dich | thee -dir | to thee -du | thou - -dies | this -diese -diesem -diesen -dieser -dieses - - -doch | (several meanings) -dort | (over) there - - -durch | through - -ein | a -eine -einem -einen -einer -eines - -einig | some -einige -einigem -einigen -einiger -einiges - -einmal | once - -er | he -ihn | him -ihm | to him - -es | it -etwas | something - -euer | your -eure -eurem -euren -eurer -eures - -für | for -gegen | towards -gewesen | p.p. of sein -hab | have -habe | have -haben | have -hat | has -hatte | had -hatten | had -hier | here -hin | there -hinter | behind - -ich | I -mich | me -mir | to me - - -ihr | you, to her -ihre -ihrem -ihren -ihrer -ihres -euch | to you - -im | in + dem -in | in -indem | while -ins | in + das -ist | is - -jede | each, every -jedem -jeden -jeder -jedes - -jene | that -jenem -jenen -jener -jenes - -jetzt | now -kann | can - -kein | no -keine -keinem -keinen -keiner -keines - -können | can -könnte | could -machen | do -man | one - -manche | some, many a -manchem -manchen -mancher -manches - -mein | my -meine -meinem -meinen -meiner -meines - -mit | with -muss | must -musste | had to -nach | to(wards) -nicht | not -nichts | nothing -noch | still, yet -nun | now -nur | only -ob | whether -oder | or -ohne | without -sehr | very - -sein | his -seine -seinem -seinen -seiner -seines - -selbst | self -sich | herself - -sie | they, she -ihnen | to them - -sind | are -so | so - -solche | such -solchem -solchen -solcher -solches - -soll | shall -sollte | should -sondern | but -sonst | else -über | over -um | about, around -und | and - -uns | us -unse -unsem -unsen -unser -unses - -unter | under -viel | much -vom | von + dem -von | from -vor | before -während | while -war | was -waren | were -warst | wast -was | what -weg | away, off -weil | because -weiter | further - -welche | which -welchem -welchen -welcher -welches - -wenn | when -werde | will -werden | will -wie | how -wieder | again -will | want -wir | we -wird | will -wirst | willst -wo | where -wollen | want -wollte | wanted -würde | would -würden | would -zu | to -zum | zu + dem -zur | zu + der -zwar | indeed -zwischen | between - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_el.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_el.txt deleted file mode 100644 index 232681f5bd6..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_el.txt +++ /dev/null @@ -1,78 +0,0 @@ -# Lucene Greek Stopwords list -# Note: by default this file is used after GreekLowerCaseFilter, -# so when modifying this file use 'σ' instead of 'ς' -ο -η -το -οι -τα -του -τησ -των -τον -την -και -κι -κ -ειμαι -εισαι -ειναι -ειμαστε -ειστε -στο -στον -στη -στην -μα -αλλα -απο -για -προσ -με -σε -ωσ -παρα -αντι -κατα -μετα -θα -να -δε -δεν -μη -μην -επι -ενω -εαν -αν -τοτε -που -πωσ -ποιοσ -ποια -ποιο -ποιοι -ποιεσ -ποιων -ποιουσ -αυτοσ -αυτη -αυτο -αυτοι -αυτων -αυτουσ -αυτεσ -αυτα -εκεινοσ -εκεινη -εκεινο -εκεινοι -εκεινεσ -εκεινα -εκεινων -εκεινουσ -οπωσ -ομωσ -ισωσ -οσο -οτι diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_en.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_en.txt deleted file mode 100644 index 2c164c0b2a1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_en.txt +++ /dev/null @@ -1,54 +0,0 @@ -# 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. - -# a couple of test stopwords to test that the words are really being -# configured from this file: -stopworda -stopwordb - -# Standard english stop words taken from Lucene's StopAnalyzer -a -an -and -are -as -at -be -but -by -for -if -in -into -is -it -no -not -of -on -or -such -that -the -their -then -there -these -they -this -to -was -will -with diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_es.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_es.txt deleted file mode 100644 index 2db14760075..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_es.txt +++ /dev/null @@ -1,354 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/spanish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Spanish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | from, of -la | the, her -que | who, that -el | the -en | in -y | and -a | to -los | the, them -del | de + el -se | himself, from him etc -las | the, them -por | for, by, etc -un | a -para | for -con | with -no | no -una | a -su | his, her -al | a + el - | es from SER -lo | him -como | how -más | more -pero | pero -sus | su plural -le | to him, her -ya | already -o | or - | fue from SER -este | this - | ha from HABER -sí | himself etc -porque | because -esta | this - | son from SER -entre | between - | está from ESTAR -cuando | when -muy | very -sin | without -sobre | on - | ser from SER - | tiene from TENER -también | also -me | me -hasta | until -hay | there is/are -donde | where - | han from HABER -quien | whom, that - | están from ESTAR - | estado from ESTAR -desde | from -todo | all -nos | us -durante | during - | estados from ESTAR -todos | all -uno | a -les | to them -ni | nor -contra | against -otros | other - | fueron from SER -ese | that -eso | that - | había from HABER -ante | before -ellos | they -e | and (variant of y) -esto | this -mí | me -antes | before -algunos | some -qué | what? -unos | a -yo | I -otro | other -otras | other -otra | other -él | he -tanto | so much, many -esa | that -estos | these -mucho | much, many -quienes | who -nada | nothing -muchos | many -cual | who - | sea from SER -poco | few -ella | she -estar | to be - | haber from HABER -estas | these - | estaba from ESTAR - | estamos from ESTAR -algunas | some -algo | something -nosotros | we - - | other forms - -mi | me -mis | mi plural -tú | thou -te | thee -ti | thee -tu | thy -tus | tu plural -ellas | they -nosotras | we -vosotros | you -vosotras | you -os | you -mío | mine -mía | -míos | -mías | -tuyo | thine -tuya | -tuyos | -tuyas | -suyo | his, hers, theirs -suya | -suyos | -suyas | -nuestro | ours -nuestra | -nuestros | -nuestras | -vuestro | yours -vuestra | -vuestros | -vuestras | -esos | those -esas | those - - | forms of estar, to be (not including the infinitive): -estoy -estás -está -estamos -estáis -están -esté -estés -estemos -estéis -estén -estaré -estarás -estará -estaremos -estaréis -estarán -estaría -estarías -estaríamos -estaríais -estarían -estaba -estabas -estábamos -estabais -estaban -estuve -estuviste -estuvo -estuvimos -estuvisteis -estuvieron -estuviera -estuvieras -estuviéramos -estuvierais -estuvieran -estuviese -estuvieses -estuviésemos -estuvieseis -estuviesen -estando -estado -estada -estados -estadas -estad - - | forms of haber, to have (not including the infinitive): -he -has -ha -hemos -habéis -han -haya -hayas -hayamos -hayáis -hayan -habré -habrás -habrá -habremos -habréis -habrán -habría -habrías -habríamos -habríais -habrían -había -habías -habíamos -habíais -habían -hube -hubiste -hubo -hubimos -hubisteis -hubieron -hubiera -hubieras -hubiéramos -hubierais -hubieran -hubiese -hubieses -hubiésemos -hubieseis -hubiesen -habiendo -habido -habida -habidos -habidas - - | forms of ser, to be (not including the infinitive): -soy -eres -es -somos -sois -son -sea -seas -seamos -seáis -sean -seré -serás -será -seremos -seréis -serán -sería -serías -seríamos -seríais -serían -era -eras -éramos -erais -eran -fui -fuiste -fue -fuimos -fuisteis -fueron -fuera -fueras -fuéramos -fuerais -fueran -fuese -fueses -fuésemos -fueseis -fuesen -siendo -sido - | sed also means 'thirst' - - | forms of tener, to have (not including the infinitive): -tengo -tienes -tiene -tenemos -tenéis -tienen -tenga -tengas -tengamos -tengáis -tengan -tendré -tendrás -tendrá -tendremos -tendréis -tendrán -tendría -tendrías -tendríamos -tendríais -tendrían -tenía -tenías -teníamos -teníais -tenían -tuve -tuviste -tuvo -tuvimos -tuvisteis -tuvieron -tuviera -tuvieras -tuviéramos -tuvierais -tuvieran -tuviese -tuvieses -tuviésemos -tuvieseis -tuviesen -teniendo -tenido -tenida -tenidos -tenidas -tened - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_eu.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_eu.txt deleted file mode 100644 index 25f1db93460..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_eu.txt +++ /dev/null @@ -1,99 +0,0 @@ -# example set of basque stopwords -al -anitz -arabera -asko -baina -bat -batean -batek -bati -batzuei -batzuek -batzuetan -batzuk -bera -beraiek -berau -berauek -bere -berori -beroriek -beste -bezala -da -dago -dira -ditu -du -dute -edo -egin -ere -eta -eurak -ez -gainera -gu -gutxi -guzti -haiei -haiek -haietan -hainbeste -hala -han -handik -hango -hara -hari -hark -hartan -hau -hauei -hauek -hauetan -hemen -hemendik -hemengo -hi -hona -honek -honela -honetan -honi -hor -hori -horiei -horiek -horietan -horko -horra -horrek -horrela -horretan -horri -hortik -hura -izan -ni -noiz -nola -non -nondik -nongo -nor -nora -ze -zein -zen -zenbait -zenbat -zer -zergatik -ziren -zituen -zu -zuek -zuen -zuten diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fa.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fa.txt deleted file mode 100644 index 723641c6da7..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fa.txt +++ /dev/null @@ -1,313 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Note: by default this file is used after normalization, so when adding entries -# to this file, use the arabic 'ي' instead of 'ی' -انان -نداشته -سراسر -خياه -ايشان -وي -تاكنون -بيشتري -دوم -پس -ناشي -وگو -يا -داشتند -سپس -هنگام -هرگز -پنج -نشان -امسال -ديگر -گروهي -شدند -چطور -ده -و -دو -نخستين -ولي -چرا -چه -وسط -ه -كدام -قابل -يك -رفت -هفت -همچنين -در -هزار -بله -بلي -شايد -اما -شناسي -گرفته -دهد -داشته -دانست -داشتن -خواهيم -ميليارد -وقتيكه -امد -خواهد -جز -اورده -شده -بلكه -خدمات -شدن -برخي -نبود -بسياري -جلوگيري -حق -كردند -نوعي -بعري -نكرده -نظير -نبايد -بوده -بودن -داد -اورد -هست -جايي -شود -دنبال -داده -بايد -سابق -هيچ -همان -انجا -كمتر -كجاست -گردد -كسي -تر -مردم -تان -دادن -بودند -سري -جدا -ندارند -مگر -يكديگر -دارد -دهند -بنابراين -هنگامي -سمت -جا -انچه -خود -دادند -زياد -دارند -اثر -بدون -بهترين -بيشتر -البته -به -براساس -بيرون -كرد -بعضي -گرفت -توي -اي -ميليون -او -جريان -تول -بر -مانند -برابر -باشيم -مدتي -گويند -اكنون -تا -تنها -جديد -چند -بي -نشده -كردن -كردم -گويد -كرده -كنيم -نمي -نزد -روي -قصد -فقط -بالاي -ديگران -اين -ديروز -توسط -سوم -ايم -دانند -سوي -استفاده -شما -كنار -داريم -ساخته -طور -امده -رفته -نخست -بيست -نزديك -طي -كنيد -از -انها -تمامي -داشت -يكي -طريق -اش -چيست -روب -نمايد -گفت -چندين -چيزي -تواند -ام -ايا -با -ان -ايد -ترين -اينكه -ديگري -راه -هايي -بروز -همچنان -پاعين -كس -حدود -مختلف -مقابل -چيز -گيرد -ندارد -ضد -همچون -سازي -شان -مورد -باره -مرسي -خويش -برخوردار -چون -خارج -شش -هنوز -تحت -ضمن -هستيم -گفته -فكر -بسيار -پيش -براي -روزهاي -انكه -نخواهد -بالا -كل -وقتي -كي -چنين -كه -گيري -نيست -است -كجا -كند -نيز -يابد -بندي -حتي -توانند -عقب -خواست -كنند -بين -تمام -همه -ما -باشند -مثل -شد -اري -باشد -اره -طبق -بعد -اگر -صورت -غير -جاي -بيش -ريزي -اند -زيرا -چگونه -بار -لطفا -مي -درباره -من -ديده -همين -گذاري -برداري -علت -گذاشته -هم -فوق -نه -ها -شوند -اباد -همواره -هر -اول -خواهند -چهار -نام -امروز -مان -هاي -قبل -كنم -سعي -تازه -را -هستند -زير -جلوي -عنوان -بود diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fi.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fi.txt deleted file mode 100644 index addad798c4b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fi.txt +++ /dev/null @@ -1,95 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/finnish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - -| forms of BE - -olla -olen -olet -on -olemme -olette -ovat -ole | negative form - -oli -olisi -olisit -olisin -olisimme -olisitte -olisivat -olit -olin -olimme -olitte -olivat -ollut -olleet - -en | negation -et -ei -emme -ette -eivät - -|Nom Gen Acc Part Iness Elat Illat Adess Ablat Allat Ess Trans -minä minun minut minua minussa minusta minuun minulla minulta minulle | I -sinä sinun sinut sinua sinussa sinusta sinuun sinulla sinulta sinulle | you -hän hänen hänet häntä hänessä hänestä häneen hänellä häneltä hänelle | he she -me meidän meidät meitä meissä meistä meihin meillä meiltä meille | we -te teidän teidät teitä teissä teistä teihin teillä teiltä teille | you -he heidän heidät heitä heissä heistä heihin heillä heiltä heille | they - -tämä tämän tätä tässä tästä tähän tallä tältä tälle tänä täksi | this -tuo tuon tuotä tuossa tuosta tuohon tuolla tuolta tuolle tuona tuoksi | that -se sen sitä siinä siitä siihen sillä siltä sille sinä siksi | it -nämä näiden näitä näissä näistä näihin näillä näiltä näille näinä näiksi | these -nuo noiden noita noissa noista noihin noilla noilta noille noina noiksi | those -ne niiden niitä niissä niistä niihin niillä niiltä niille niinä niiksi | they - -kuka kenen kenet ketä kenessä kenestä keneen kenellä keneltä kenelle kenenä keneksi| who -ketkä keiden ketkä keitä keissä keistä keihin keillä keiltä keille keinä keiksi | (pl) -mikä minkä minkä mitä missä mistä mihin millä miltä mille minä miksi | which what -mitkä | (pl) - -joka jonka jota jossa josta johon jolla jolta jolle jona joksi | who which -jotka joiden joita joissa joista joihin joilla joilta joille joina joiksi | (pl) - -| conjunctions - -että | that -ja | and -jos | if -koska | because -kuin | than -mutta | but -niin | so -sekä | and -sillä | for -tai | or -vaan | but -vai | or -vaikka | although - - -| prepositions - -kanssa | with -mukaan | according to -noin | about -poikki | across -yli | over, across - -| other - -kun | when -niin | so -nyt | now -itse | self - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fr.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fr.txt deleted file mode 100644 index c00837ea939..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_fr.txt +++ /dev/null @@ -1,183 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/french/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A French stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -au | a + le -aux | a + les -avec | with -ce | this -ces | these -dans | with -de | of -des | de + les -du | de + le -elle | she -en | `of them' etc -et | and -eux | them -il | he -je | I -la | the -le | the -leur | their -lui | him -ma | my (fem) -mais | but -me | me -même | same; as in moi-même (myself) etc -mes | me (pl) -moi | me -mon | my (masc) -ne | not -nos | our (pl) -notre | our -nous | we -on | one -ou | where -par | by -pas | not -pour | for -qu | que before vowel -que | that -qui | who -sa | his, her (fem) -se | oneself -ses | his (pl) -son | his, her (masc) -sur | on -ta | thy (fem) -te | thee -tes | thy (pl) -toi | thee -ton | thy (masc) -tu | thou -un | a -une | a -vos | your (pl) -votre | your -vous | you - - | single letter forms - -c | c' -d | d' -j | j' -l | l' -à | to, at -m | m' -n | n' -s | s' -t | t' -y | there - - | forms of être (not including the infinitive): -été -étée -étées -étés -étant -suis -es -est -sommes -êtes -sont -serai -seras -sera -serons -serez -seront -serais -serait -serions -seriez -seraient -étais -était -étions -étiez -étaient -fus -fut -fûmes -fûtes -furent -sois -soit -soyons -soyez -soient -fusse -fusses -fût -fussions -fussiez -fussent - - | forms of avoir (not including the infinitive): -ayant -eu -eue -eues -eus -ai -as -avons -avez -ont -aurai -auras -aura -aurons -aurez -auront -aurais -aurait -aurions -auriez -auraient -avais -avait -avions -aviez -avaient -eut -eûmes -eûtes -eurent -aie -aies -ait -ayons -ayez -aient -eusse -eusses -eût -eussions -eussiez -eussent - - | Later additions (from Jean-Christophe Deschamps) -ceci | this -celà  | that -cet | this -cette | this -ici | here -ils | they -les | the (pl) -leurs | their (pl) -quel | which -quels | which -quelle | which -quelles | which -sans | without -soi | oneself - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ga.txt deleted file mode 100644 index 9ff88d747e5..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ga.txt +++ /dev/null @@ -1,110 +0,0 @@ - -a -ach -ag -agus -an -aon -ar -arna -as -b' -ba -beirt -bhúr -caoga -ceathair -ceathrar -chomh -chtó -chuig -chun -cois -céad -cúig -cúigear -d' -daichead -dar -de -deich -deichniúr -den -dhá -do -don -dtí -dá -dár -dó -faoi -faoin -faoina -faoinár -fara -fiche -gach -gan -go -gur -haon -hocht -i -iad -idir -in -ina -ins -inár -is -le -leis -lena -lenár -m' -mar -mo -mé -na -nach -naoi -naonúr -ná -ní -níor -nó -nócha -ocht -ochtar -os -roimh -sa -seacht -seachtar -seachtó -seasca -seisear -siad -sibh -sinn -sna -sé -sí -tar -thar -thú -triúr -trí -trína -trínár -tríocha -tú -um -ár -é -éis -í -ó -ón -óna -ónár diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_gl.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_gl.txt deleted file mode 100644 index d8760b12c14..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_gl.txt +++ /dev/null @@ -1,161 +0,0 @@ -# galican stopwords -a -aínda -alí -aquel -aquela -aquelas -aqueles -aquilo -aquí -ao -aos -as -así -á -ben -cando -che -co -coa -comigo -con -connosco -contigo -convosco -coas -cos -cun -cuns -cunha -cunhas -da -dalgunha -dalgunhas -dalgún -dalgúns -das -de -del -dela -delas -deles -desde -deste -do -dos -dun -duns -dunha -dunhas -e -el -ela -elas -eles -en -era -eran -esa -esas -ese -eses -esta -estar -estaba -está -están -este -estes -estiven -estou -eu -é -facer -foi -foron -fun -había -hai -iso -isto -la -las -lle -lles -lo -los -mais -me -meu -meus -min -miña -miñas -moi -na -nas -neste -nin -no -non -nos -nosa -nosas -noso -nosos -nós -nun -nunha -nuns -nunhas -o -os -ou -ó -ós -para -pero -pode -pois -pola -polas -polo -polos -por -que -se -senón -ser -seu -seus -sexa -sido -sobre -súa -súas -tamén -tan -te -ten -teñen -teño -ter -teu -teus -ti -tido -tiña -tiven -túa -túas -un -unha -unhas -uns -vos -vosa -vosas -voso -vosos -vós diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hi.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hi.txt deleted file mode 100644 index 86286bb083b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hi.txt +++ /dev/null @@ -1,235 +0,0 @@ -# Also see http://www.opensource.org/licenses/bsd-license.html -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# This file was created by Jacques Savoy and is distributed under the BSD license. -# Note: by default this file also contains forms normalized by HindiNormalizer -# for spelling variation (see section below), such that it can be used whether or -# not you enable that feature. When adding additional entries to this list, -# please add the normalized form as well. -अंदर -अत -अपना -अपनी -अपने -अभी -आदि -आप -इत्यादि -इन -इनका -इन्हीं -इन्हें -इन्हों -इस -इसका -इसकी -इसके -इसमें -इसी -इसे -उन -उनका -उनकी -उनके -उनको -उन्हीं -उन्हें -उन्हों -उस -उसके -उसी -उसे -एक -एवं -एस -ऐसे -और -कई -कर -करता -करते -करना -करने -करें -कहते -कहा -का -काफ़ी -कि -कितना -किन्हें -किन्हों -किया -किर -किस -किसी -किसे -की -कुछ -कुल -के -को -कोई -कौन -कौनसा -गया -घर -जब -जहाँ -जा -जितना -जिन -जिन्हें -जिन्हों -जिस -जिसे -जीधर -जैसा -जैसे -जो -तक -तब -तरह -तिन -तिन्हें -तिन्हों -तिस -तिसे -तो -था -थी -थे -दबारा -दिया -दुसरा -दूसरे -दो -द्वारा -न -नहीं -ना -निहायत -नीचे -ने -पर -पर -पहले -पूरा -पे -फिर -बनी -बही -बहुत -बाद -बाला -बिलकुल -भी -भीतर -मगर -मानो -मे -में -यदि -यह -यहाँ -यही -या -यिह -ये -रखें -रहा -रहे -ऱ्वासा -लिए -लिये -लेकिन -व -वर्ग -वह -वह -वहाँ -वहीं -वाले -वुह -वे -वग़ैरह -संग -सकता -सकते -सबसे -सभी -साथ -साबुत -साभ -सारा -से -सो -ही -हुआ -हुई -हुए -है -हैं -हो -होता -होती -होते -होना -होने -# additional normalized forms of the above -अपनि -जेसे -होति -सभि -तिंहों -इंहों -दवारा -इसि -किंहें -थि -उंहों -ओर -जिंहें -वहिं -अभि -बनि -हि -उंहिं -उंहें -हें -वगेरह -एसे -रवासा -कोन -निचे -काफि -उसि -पुरा -भितर -हे -बहि -वहां -कोइ -यहां -जिंहों -तिंहें -किसि -कइ -यहि -इंहिं -जिधर -इंहें -अदि -इतयादि -हुइ -कोनसा -इसकि -दुसरे -जहां -अप -किंहों -उनकि -भि -वरग -हुअ -जेसा -नहिं diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hu.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hu.txt deleted file mode 100644 index 1a96f1db6f2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hu.txt +++ /dev/null @@ -1,209 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/hungarian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - -| Hungarian stop word list -| prepared by Anna Tordai - -a -ahogy -ahol -aki -akik -akkor -alatt -által -általában -amely -amelyek -amelyekben -amelyeket -amelyet -amelynek -ami -amit -amolyan -amíg -amikor -át -abban -ahhoz -annak -arra -arról -az -azok -azon -azt -azzal -azért -aztán -azután -azonban -bár -be -belül -benne -cikk -cikkek -cikkeket -csak -de -e -eddig -egész -egy -egyes -egyetlen -egyéb -egyik -egyre -ekkor -el -elég -ellen -elő -először -előtt -első -én -éppen -ebben -ehhez -emilyen -ennek -erre -ez -ezt -ezek -ezen -ezzel -ezért -és -fel -felé -hanem -hiszen -hogy -hogyan -igen -így -illetve -ill. -ill -ilyen -ilyenkor -ison -ismét -itt -jó -jól -jobban -kell -kellett -keresztül -keressünk -ki -kívül -között -közül -legalább -lehet -lehetett -legyen -lenne -lenni -lesz -lett -maga -magát -majd -majd -már -más -másik -meg -még -mellett -mert -mely -melyek -mi -mit -míg -miért -milyen -mikor -minden -mindent -mindenki -mindig -mint -mintha -mivel -most -nagy -nagyobb -nagyon -ne -néha -nekem -neki -nem -néhány -nélkül -nincs -olyan -ott -össze -ő -ők -őket -pedig -persze -rá -s -saját -sem -semmi -sok -sokat -sokkal -számára -szemben -szerint -szinte -talán -tehát -teljes -tovább -továbbá -több -úgy -ugyanis -új -újabb -újra -után -utána -utolsó -vagy -vagyis -valaki -valami -valamint -való -vagyok -van -vannak -volt -voltam -voltak -voltunk -vissza -vele -viszont -volna diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hy.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hy.txt deleted file mode 100644 index 60c1c50fbc8..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_hy.txt +++ /dev/null @@ -1,46 +0,0 @@ -# example set of Armenian stopwords. -այդ -այլ -այն -այս -դու -դուք -եմ -են -ենք -ես -եք -է -էի -էին -էինք -էիր -էիք -էր -ըստ -թ -ի -ին -իսկ -իր -կամ -համար -հետ -հետո -մենք -մեջ -մի -ն -նա -նաև -նրա -նրանք -որ -որը -որոնք -որպես -ու -ում -պիտի -վրա -և diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_id.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_id.txt deleted file mode 100644 index 4617f83a5c5..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_id.txt +++ /dev/null @@ -1,359 +0,0 @@ -# from appendix D of: A Study of Stemming Effects on Information -# Retrieval in Bahasa Indonesia -ada -adanya -adalah -adapun -agak -agaknya -agar -akan -akankah -akhirnya -aku -akulah -amat -amatlah -anda -andalah -antar -diantaranya -antara -antaranya -diantara -apa -apaan -mengapa -apabila -apakah -apalagi -apatah -atau -ataukah -ataupun -bagai -bagaikan -sebagai -sebagainya -bagaimana -bagaimanapun -sebagaimana -bagaimanakah -bagi -bahkan -bahwa -bahwasanya -sebaliknya -banyak -sebanyak -beberapa -seberapa -begini -beginian -beginikah -beginilah -sebegini -begitu -begitukah -begitulah -begitupun -sebegitu -belum -belumlah -sebelum -sebelumnya -sebenarnya -berapa -berapakah -berapalah -berapapun -betulkah -sebetulnya -biasa -biasanya -bila -bilakah -bisa -bisakah -sebisanya -boleh -bolehkah -bolehlah -buat -bukan -bukankah -bukanlah -bukannya -cuma -percuma -dahulu -dalam -dan -dapat -dari -daripada -dekat -demi -demikian -demikianlah -sedemikian -dengan -depan -di -dia -dialah -dini -diri -dirinya -terdiri -dong -dulu -enggak -enggaknya -entah -entahlah -terhadap -terhadapnya -hal -hampir -hanya -hanyalah -harus -haruslah -harusnya -seharusnya -hendak -hendaklah -hendaknya -hingga -sehingga -ia -ialah -ibarat -ingin -inginkah -inginkan -ini -inikah -inilah -itu -itukah -itulah -jangan -jangankan -janganlah -jika -jikalau -juga -justru -kala -kalau -kalaulah -kalaupun -kalian -kami -kamilah -kamu -kamulah -kan -kapan -kapankah -kapanpun -dikarenakan -karena -karenanya -ke -kecil -kemudian -kenapa -kepada -kepadanya -ketika -seketika -khususnya -kini -kinilah -kiranya -sekiranya -kita -kitalah -kok -lagi -lagian -selagi -lah -lain -lainnya -melainkan -selaku -lalu -melalui -terlalu -lama -lamanya -selama -selama -selamanya -lebih -terlebih -bermacam -macam -semacam -maka -makanya -makin -malah -malahan -mampu -mampukah -mana -manakala -manalagi -masih -masihkah -semasih -masing -mau -maupun -semaunya -memang -mereka -merekalah -meski -meskipun -semula -mungkin -mungkinkah -nah -namun -nanti -nantinya -nyaris -oleh -olehnya -seorang -seseorang -pada -padanya -padahal -paling -sepanjang -pantas -sepantasnya -sepantasnyalah -para -pasti -pastilah -per -pernah -pula -pun -merupakan -rupanya -serupa -saat -saatnya -sesaat -saja -sajalah -saling -bersama -sama -sesama -sambil -sampai -sana -sangat -sangatlah -saya -sayalah -se -sebab -sebabnya -sebuah -tersebut -tersebutlah -sedang -sedangkan -sedikit -sedikitnya -segala -segalanya -segera -sesegera -sejak -sejenak -sekali -sekalian -sekalipun -sesekali -sekaligus -sekarang -sekarang -sekitar -sekitarnya -sela -selain -selalu -seluruh -seluruhnya -semakin -sementara -sempat -semua -semuanya -sendiri -sendirinya -seolah -seperti -sepertinya -sering -seringnya -serta -siapa -siapakah -siapapun -disini -disinilah -sini -sinilah -sesuatu -sesuatunya -suatu -sesudah -sesudahnya -sudah -sudahkah -sudahlah -supaya -tadi -tadinya -tak -tanpa -setelah -telah -tentang -tentu -tentulah -tentunya -tertentu -seterusnya -tapi -tetapi -setiap -tiap -setidaknya -tidak -tidakkah -tidaklah -toh -waduh -wah -wahai -sewaktu -walau -walaupun -wong -yaitu -yakni -yang diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_it.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_it.txt deleted file mode 100644 index 4cb5b0891b1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_it.txt +++ /dev/null @@ -1,301 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/italian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | An Italian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -ad | a (to) before vowel -al | a + il -allo | a + lo -ai | a + i -agli | a + gli -all | a + l' -agl | a + gl' -alla | a + la -alle | a + le -con | with -col | con + il -coi | con + i (forms collo, cogli etc are now very rare) -da | from -dal | da + il -dallo | da + lo -dai | da + i -dagli | da + gli -dall | da + l' -dagl | da + gll' -dalla | da + la -dalle | da + le -di | of -del | di + il -dello | di + lo -dei | di + i -degli | di + gli -dell | di + l' -degl | di + gl' -della | di + la -delle | di + le -in | in -nel | in + el -nello | in + lo -nei | in + i -negli | in + gli -nell | in + l' -negl | in + gl' -nella | in + la -nelle | in + le -su | on -sul | su + il -sullo | su + lo -sui | su + i -sugli | su + gli -sull | su + l' -sugl | su + gl' -sulla | su + la -sulle | su + le -per | through, by -tra | among -contro | against -io | I -tu | thou -lui | he -lei | she -noi | we -voi | you -loro | they -mio | my -mia | -miei | -mie | -tuo | -tua | -tuoi | thy -tue | -suo | -sua | -suoi | his, her -sue | -nostro | our -nostra | -nostri | -nostre | -vostro | your -vostra | -vostri | -vostre | -mi | me -ti | thee -ci | us, there -vi | you, there -lo | him, the -la | her, the -li | them -le | them, the -gli | to him, the -ne | from there etc -il | the -un | a -uno | a -una | a -ma | but -ed | and -se | if -perché | why, because -anche | also -come | how -dov | where (as dov') -dove | where -che | who, that -chi | who -cui | whom -non | not -più | more -quale | who, that -quanto | how much -quanti | -quanta | -quante | -quello | that -quelli | -quella | -quelle | -questo | this -questi | -questa | -queste | -si | yes -tutto | all -tutti | all - - | single letter forms: - -a | at -c | as c' for ce or ci -e | and -i | the -l | as l' -o | or - - | forms of avere, to have (not including the infinitive): - -ho -hai -ha -abbiamo -avete -hanno -abbia -abbiate -abbiano -avrò -avrai -avrà -avremo -avrete -avranno -avrei -avresti -avrebbe -avremmo -avreste -avrebbero -avevo -avevi -aveva -avevamo -avevate -avevano -ebbi -avesti -ebbe -avemmo -aveste -ebbero -avessi -avesse -avessimo -avessero -avendo -avuto -avuta -avuti -avute - - | forms of essere, to be (not including the infinitive): -sono -sei -è -siamo -siete -sia -siate -siano -sarò -sarai -sarà -saremo -sarete -saranno -sarei -saresti -sarebbe -saremmo -sareste -sarebbero -ero -eri -era -eravamo -eravate -erano -fui -fosti -fu -fummo -foste -furono -fossi -fosse -fossimo -fossero -essendo - - | forms of fare, to do (not including the infinitive, fa, fat-): -faccio -fai -facciamo -fanno -faccia -facciate -facciano -farò -farai -farà -faremo -farete -faranno -farei -faresti -farebbe -faremmo -fareste -farebbero -facevo -facevi -faceva -facevamo -facevate -facevano -feci -facesti -fece -facemmo -faceste -fecero -facessi -facesse -facessimo -facessero -facendo - - | forms of stare, to be (not including the infinitive): -sto -stai -sta -stiamo -stanno -stia -stiate -stiano -starò -starai -starà -staremo -starete -staranno -starei -staresti -starebbe -staremmo -stareste -starebbero -stavo -stavi -stava -stavamo -stavate -stavano -stetti -stesti -stette -stemmo -steste -stettero -stessi -stesse -stessimo -stessero -stando diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ja.txt deleted file mode 100644 index d4321be6b16..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ja.txt +++ /dev/null @@ -1,127 +0,0 @@ -# -# This file defines a stopword set for Japanese. -# -# This set is made up of hand-picked frequent terms from segmented Japanese Wikipedia. -# Punctuation characters and frequent kanji have mostly been left out. See LUCENE-3745 -# for frequency lists, etc. that can be useful for making your own set (if desired) -# -# Note that there is an overlap between these stopwords and the terms stopped when used -# in combination with the JapanesePartOfSpeechStopFilter. When editing this file, note -# that comments are not allowed on the same line as stopwords. -# -# Also note that stopping is done in a case-insensitive manner. Change your StopFilter -# configuration if you need case-sensitive stopping. Lastly, note that stopping is done -# using the same character width as the entries in this file. Since this StopFilter is -# normally done after a CJKWidthFilter in your chain, you would usually want your romaji -# entries to be in half-width and your kana entries to be in full-width. -# -の -に -は -を -た -が -で -て -と -し -れ -さ -ある -いる -も -する -から -な -こと -として -い -や -れる -など -なっ -ない -この -ため -その -あっ -よう -また -もの -という -あり -まで -られ -なる -へ -か -だ -これ -によって -により -おり -より -による -ず -なり -られる -において -ば -なかっ -なく -しかし -について -せ -だっ -その後 -できる -それ -う -ので -なお -のみ -でき -き -つ -における -および -いう -さらに -でも -ら -たり -その他 -に関する -たち -ます -ん -なら -に対して -特に -せる -及び -これら -とき -では -にて -ほか -ながら -うち -そして -とともに -ただし -かつて -それぞれ -または -お -ほど -ものの -に対する -ほとんど -と共に -といった -です -とも -ところ -ここ -##### End of file diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_lv.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_lv.txt deleted file mode 100644 index e21a23c06c3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_lv.txt +++ /dev/null @@ -1,172 +0,0 @@ -# Set of Latvian stopwords from A Stemming Algorithm for Latvian, Karlis Kreslins -# the original list of over 800 forms was refined: -# pronouns, adverbs, interjections were removed -# -# prepositions -aiz -ap -ar -apakš -ārpus -augšpus -bez -caur -dēļ -gar -iekš -iz -kopš -labad -lejpus -līdz -no -otrpus -pa -par -pār -pēc -pie -pirms -pret -priekš -starp -šaipus -uz -viņpus -virs -virspus -zem -apakšpus -# Conjunctions -un -bet -jo -ja -ka -lai -tomēr -tikko -turpretī -arī -kaut -gan -tādēļ -tā -ne -tikvien -vien -kā -ir -te -vai -kamēr -# Particles -ar -diezin -droši -diemžēl -nebūt -ik -it -taču -nu -pat -tiklab -iekšpus -nedz -tik -nevis -turpretim -jeb -iekam -iekām -iekāms -kolīdz -līdzko -tiklīdz -jebšu -tālab -tāpēc -nekā -itin -jā -jau -jel -nē -nezin -tad -tikai -vis -tak -iekams -vien -# modal verbs -būt -biju -biji -bija -bijām -bijāt -esmu -esi -esam -esat -būšu -būsi -būs -būsim -būsiet -tikt -tiku -tiki -tika -tikām -tikāt -tieku -tiec -tiek -tiekam -tiekat -tikšu -tiks -tiksim -tiksiet -tapt -tapi -tapāt -topat -tapšu -tapsi -taps -tapsim -tapsiet -kļūt -kļuvu -kļuvi -kļuva -kļuvām -kļuvāt -kļūstu -kļūsti -kļūst -kļūstam -kļūstat -kļūšu -kļūsi -kļūs -kļūsim -kļūsiet -# verbs -varēt -varēju -varējām -varēšu -varēsim -var -varēji -varējāt -varēsi -varēsiet -varat -varēja -varēs diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_nl.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_nl.txt deleted file mode 100644 index f4d61f5092c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_nl.txt +++ /dev/null @@ -1,117 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/dutch/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Dutch stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large sample of Dutch text. - - | Dutch stop words frequently exhibit homonym clashes. These are indicated - | clearly below. - -de | the -en | and -van | of, from -ik | I, the ego -te | (1) chez, at etc, (2) to, (3) too -dat | that, which -die | that, those, who, which -in | in, inside -een | a, an, one -hij | he -het | the, it -niet | not, nothing, naught -zijn | (1) to be, being, (2) his, one's, its -is | is -was | (1) was, past tense of all persons sing. of 'zijn' (to be) (2) wax, (3) the washing, (4) rise of river -op | on, upon, at, in, up, used up -aan | on, upon, to (as dative) -met | with, by -als | like, such as, when -voor | (1) before, in front of, (2) furrow -had | had, past tense all persons sing. of 'hebben' (have) -er | there -maar | but, only -om | round, about, for etc -hem | him -dan | then -zou | should/would, past tense all persons sing. of 'zullen' -of | or, whether, if -wat | what, something, anything -mijn | possessive and noun 'mine' -men | people, 'one' -dit | this -zo | so, thus, in this way -door | through by -over | over, across -ze | she, her, they, them -zich | oneself -bij | (1) a bee, (2) by, near, at -ook | also, too -tot | till, until -je | you -mij | me -uit | out of, from -der | Old Dutch form of 'van der' still found in surnames -daar | (1) there, (2) because -haar | (1) her, their, them, (2) hair -naar | (1) unpleasant, unwell etc, (2) towards, (3) as -heb | present first person sing. of 'to have' -hoe | how, why -heeft | present third person sing. of 'to have' -hebben | 'to have' and various parts thereof -deze | this -u | you -want | (1) for, (2) mitten, (3) rigging -nog | yet, still -zal | 'shall', first and third person sing. of verb 'zullen' (will) -me | me -zij | she, they -nu | now -ge | 'thou', still used in Belgium and south Netherlands -geen | none -omdat | because -iets | something, somewhat -worden | to become, grow, get -toch | yet, still -al | all, every, each -waren | (1) 'were' (2) to wander, (3) wares, (3) -veel | much, many -meer | (1) more, (2) lake -doen | to do, to make -toen | then, when -moet | noun 'spot/mote' and present form of 'to must' -ben | (1) am, (2) 'are' in interrogative second person singular of 'to be' -zonder | without -kan | noun 'can' and present form of 'to be able' -hun | their, them -dus | so, consequently -alles | all, everything, anything -onder | under, beneath -ja | yes, of course -eens | once, one day -hier | here -wie | who -werd | imperfect third person sing. of 'become' -altijd | always -doch | yet, but etc -wordt | present third person sing. of 'become' -wezen | (1) to be, (2) 'been' as in 'been fishing', (3) orphans -kunnen | to be able -ons | us/our -zelf | self -tegen | against, towards, at -na | after, near -reeds | already -wil | (1) present tense of 'want', (2) 'will', noun, (3) fender -kon | could; past tense of 'to be able' -niets | nothing -uw | your -iemand | somebody -geweest | been; past participle of 'be' -andere | other diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_no.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_no.txt deleted file mode 100644 index e76f36e69ed..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_no.txt +++ /dev/null @@ -1,192 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/norwegian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Norwegian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This stop word list is for the dominant bokmål dialect. Words unique - | to nynorsk are marked *. - - | Revised by Jan Bruusgaard , Jan 2005 - -og | and -i | in -jeg | I -det | it/this/that -at | to (w. inf.) -en | a/an -et | a/an -den | it/this/that -til | to -er | is/am/are -som | who/that -på | on -de | they / you(formal) -med | with -han | he -av | of -ikke | not -ikkje | not * -der | there -så | so -var | was/were -meg | me -seg | you -men | but -ett | one -har | have -om | about -vi | we -min | my -mitt | my -ha | have -hadde | had -hun | she -nå | now -over | over -da | when/as -ved | by/know -fra | from -du | you -ut | out -sin | your -dem | them -oss | us -opp | up -man | you/one -kan | can -hans | his -hvor | where -eller | or -hva | what -skal | shall/must -selv | self (reflective) -sjøl | self (reflective) -her | here -alle | all -vil | will -bli | become -ble | became -blei | became * -blitt | have become -kunne | could -inn | in -når | when -være | be -kom | come -noen | some -noe | some -ville | would -dere | you -som | who/which/that -deres | their/theirs -kun | only/just -ja | yes -etter | after -ned | down -skulle | should -denne | this -for | for/because -deg | you -si | hers/his -sine | hers/his -sitt | hers/his -mot | against -å | to -meget | much -hvorfor | why -dette | this -disse | these/those -uten | without -hvordan | how -ingen | none -din | your -ditt | your -blir | become -samme | same -hvilken | which -hvilke | which (plural) -sånn | such a -inni | inside/within -mellom | between -vår | our -hver | each -hvem | who -vors | us/ours -hvis | whose -både | both -bare | only/just -enn | than -fordi | as/because -før | before -mange | many -også | also -slik | just -vært | been -være | to be -båe | both * -begge | both -siden | since -dykk | your * -dykkar | yours * -dei | they * -deira | them * -deires | theirs * -deim | them * -di | your (fem.) * -då | as/when * -eg | I * -ein | a/an * -eit | a/an * -eitt | a/an * -elles | or * -honom | he * -hjå | at * -ho | she * -hoe | she * -henne | her -hennar | her/hers -hennes | hers -hoss | how * -hossen | how * -ikkje | not * -ingi | noone * -inkje | noone * -korleis | how * -korso | how * -kva | what/which * -kvar | where * -kvarhelst | where * -kven | who/whom * -kvi | why * -kvifor | why * -me | we * -medan | while * -mi | my * -mine | my * -mykje | much * -no | now * -nokon | some (masc./neut.) * -noka | some (fem.) * -nokor | some * -noko | some * -nokre | some * -si | his/hers * -sia | since * -sidan | since * -so | so * -somt | some * -somme | some * -um | about* -upp | up * -vere | be * -vore | was * -verte | become * -vort | become * -varte | became * -vart | became * - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_pt.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_pt.txt deleted file mode 100644 index 276c1b446f2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_pt.txt +++ /dev/null @@ -1,251 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/portuguese/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Portuguese stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | of, from -a | the; to, at; her -o | the; him -que | who, that -e | and -do | de + o -da | de + a -em | in -um | a -para | for - | é from SER -com | with -não | not, no -uma | a -os | the; them -no | em + o -se | himself etc -na | em + a -por | for -mais | more -as | the; them -dos | de + os -como | as, like -mas | but - | foi from SER -ao | a + o -ele | he -das | de + as - | tem from TER -à | a + a -seu | his -sua | her -ou | or - | ser from SER -quando | when -muito | much - | há from HAV -nos | em + os; us -já | already, now - | está from EST -eu | I -também | also -só | only, just -pelo | per + o -pela | per + a -até | up to -isso | that -ela | he -entre | between - | era from SER -depois | after -sem | without -mesmo | same -aos | a + os - | ter from TER -seus | his -quem | whom -nas | em + as -me | me -esse | that -eles | they - | estão from EST -você | you - | tinha from TER - | foram from SER -essa | that -num | em + um -nem | nor -suas | her -meu | my -às | a + as -minha | my - | têm from TER -numa | em + uma -pelos | per + os -elas | they - | havia from HAV - | seja from SER -qual | which - | será from SER -nós | we - | tenho from TER -lhe | to him, her -deles | of them -essas | those -esses | those -pelas | per + as -este | this - | fosse from SER -dele | of him - - | other words. There are many contractions such as naquele = em+aquele, - | mo = me+o, but they are rare. - | Indefinite article plural forms are also rare. - -tu | thou -te | thee -vocês | you (plural) -vos | you -lhes | to them -meus | my -minhas -teu | thy -tua -teus -tuas -nosso | our -nossa -nossos -nossas - -dela | of her -delas | of them - -esta | this -estes | these -estas | these -aquele | that -aquela | that -aqueles | those -aquelas | those -isto | this -aquilo | that - - | forms of estar, to be (not including the infinitive): -estou -está -estamos -estão -estive -esteve -estivemos -estiveram -estava -estávamos -estavam -estivera -estivéramos -esteja -estejamos -estejam -estivesse -estivéssemos -estivessem -estiver -estivermos -estiverem - - | forms of haver, to have (not including the infinitive): -hei -há -havemos -hão -houve -houvemos -houveram -houvera -houvéramos -haja -hajamos -hajam -houvesse -houvéssemos -houvessem -houver -houvermos -houverem -houverei -houverá -houveremos -houverão -houveria -houveríamos -houveriam - - | forms of ser, to be (not including the infinitive): -sou -somos -são -era -éramos -eram -fui -foi -fomos -foram -fora -fôramos -seja -sejamos -sejam -fosse -fôssemos -fossem -for -formos -forem -serei -será -seremos -serão -seria -seríamos -seriam - - | forms of ter, to have (not including the infinitive): -tenho -tem -temos -tém -tinha -tínhamos -tinham -tive -teve -tivemos -tiveram -tivera -tivéramos -tenha -tenhamos -tenham -tivesse -tivéssemos -tivessem -tiver -tivermos -tiverem -terei -terá -teremos -terão -teria -teríamos -teriam diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ro.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ro.txt deleted file mode 100644 index 4fdee90a5ba..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ro.txt +++ /dev/null @@ -1,233 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -acea -aceasta -această -aceea -acei -aceia -acel -acela -acele -acelea -acest -acesta -aceste -acestea -aceşti -aceştia -acolo -acum -ai -aia -aibă -aici -al -ăla -ale -alea -ălea -altceva -altcineva -am -ar -are -aş -aşadar -asemenea -asta -ăsta -astăzi -astea -ăstea -ăştia -asupra -aţi -au -avea -avem -aveţi -azi -bine -bucur -bună -ca -că -căci -când -care -cărei -căror -cărui -cât -câte -câţi -către -câtva -ce -cel -ceva -chiar -cînd -cine -cineva -cît -cîte -cîţi -cîtva -contra -cu -cum -cumva -curând -curînd -da -dă -dacă -dar -datorită -de -deci -deja -deoarece -departe -deşi -din -dinaintea -dintr -dintre -drept -după -ea -ei -el -ele -eram -este -eşti -eu -face -fără -fi -fie -fiecare -fii -fim -fiţi -iar -ieri -îi -îl -îmi -împotriva -în -înainte -înaintea -încât -încît -încotro -între -întrucât -întrucît -îţi -la -lângă -le -li -lîngă -lor -lui -mă -mâine -mea -mei -mele -mereu -meu -mi -mine -mult -multă -mulţi -ne -nicăieri -nici -nimeni -nişte -noastră -noastre -noi -noştri -nostru -nu -ori -oricând -oricare -oricât -orice -oricînd -oricine -oricît -oricum -oriunde -până -pe -pentru -peste -pînă -poate -pot -prea -prima -primul -prin -printr -sa -să -săi -sale -sau -său -se -şi -sînt -sîntem -sînteţi -spre -sub -sunt -suntem -sunteţi -ta -tăi -tale -tău -te -ţi -ţie -tine -toată -toate -tot -toţi -totuşi -tu -un -una -unde -undeva -unei -unele -uneori -unor -vă -vi -voastră -voastre -voi -voştri -vostru -vouă -vreo -vreun diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ru.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ru.txt deleted file mode 100644 index 64307693457..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_ru.txt +++ /dev/null @@ -1,241 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/russian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | a russian stop word list. comments begin with vertical bar. each stop - | word is at the start of a line. - - | this is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | letter `ё' is translated to `е'. - -и | and -в | in/into -во | alternative form -не | not -что | what/that -он | he -на | on/onto -я | i -с | from -со | alternative form -как | how -а | milder form of `no' (but) -то | conjunction and form of `that' -все | all -она | she -так | so, thus -его | him -но | but -да | yes/and -ты | thou -к | towards, by -у | around, chez -же | intensifier particle -вы | you -за | beyond, behind -бы | conditional/subj. particle -по | up to, along -только | only -ее | her -мне | to me -было | it was -вот | here is/are, particle -от | away from -меня | me -еще | still, yet, more -нет | no, there isnt/arent -о | about -из | out of -ему | to him -теперь | now -когда | when -даже | even -ну | so, well -вдруг | suddenly -ли | interrogative particle -если | if -уже | already, but homonym of `narrower' -или | or -ни | neither -быть | to be -был | he was -него | prepositional form of его -до | up to -вас | you accusative -нибудь | indef. suffix preceded by hyphen -опять | again -уж | already, but homonym of `adder' -вам | to you -сказал | he said -ведь | particle `after all' -там | there -потом | then -себя | oneself -ничего | nothing -ей | to her -может | usually with `быть' as `maybe' -они | they -тут | here -где | where -есть | there is/are -надо | got to, must -ней | prepositional form of ей -для | for -мы | we -тебя | thee -их | them, their -чем | than -была | she was -сам | self -чтоб | in order to -без | without -будто | as if -человек | man, person, one -чего | genitive form of `what' -раз | once -тоже | also -себе | to oneself -под | beneath -жизнь | life -будет | will be -ж | short form of intensifer particle `же' -тогда | then -кто | who -этот | this -говорил | was saying -того | genitive form of `that' -потому | for that reason -этого | genitive form of `this' -какой | which -совсем | altogether -ним | prepositional form of `его', `они' -здесь | here -этом | prepositional form of `этот' -один | one -почти | almost -мой | my -тем | instrumental/dative plural of `тот', `то' -чтобы | full form of `in order that' -нее | her (acc.) -кажется | it seems -сейчас | now -были | they were -куда | where to -зачем | why -сказать | to say -всех | all (acc., gen. preposn. plural) -никогда | never -сегодня | today -можно | possible, one can -при | by -наконец | finally -два | two -об | alternative form of `о', about -другой | another -хоть | even -после | after -над | above -больше | more -тот | that one (masc.) -через | across, in -эти | these -нас | us -про | about -всего | in all, only, of all -них | prepositional form of `они' (they) -какая | which, feminine -много | lots -разве | interrogative particle -сказала | she said -три | three -эту | this, acc. fem. sing. -моя | my, feminine -впрочем | moreover, besides -хорошо | good -свою | ones own, acc. fem. sing. -этой | oblique form of `эта', fem. `this' -перед | in front of -иногда | sometimes -лучше | better -чуть | a little -том | preposn. form of `that one' -нельзя | one must not -такой | such a one -им | to them -более | more -всегда | always -конечно | of course -всю | acc. fem. sing of `all' -между | between - - - | b: some paradigms - | - | personal pronouns - | - | я меня мне мной [мною] - | ты тебя тебе тобой [тобою] - | он его ему им [него, нему, ним] - | она ее эи ею [нее, нэи, нею] - | оно его ему им [него, нему, ним] - | - | мы нас нам нами - | вы вас вам вами - | они их им ими [них, ним, ними] - | - | себя себе собой [собою] - | - | demonstrative pronouns: этот (this), тот (that) - | - | этот эта это эти - | этого эты это эти - | этого этой этого этих - | этому этой этому этим - | этим этой этим [этою] этими - | этом этой этом этих - | - | тот та то те - | того ту то те - | того той того тех - | тому той тому тем - | тем той тем [тою] теми - | том той том тех - | - | determinative pronouns - | - | (a) весь (all) - | - | весь вся все все - | всего всю все все - | всего всей всего всех - | всему всей всему всем - | всем всей всем [всею] всеми - | всем всей всем всех - | - | (b) сам (himself etc) - | - | сам сама само сами - | самого саму само самих - | самого самой самого самих - | самому самой самому самим - | самим самой самим [самою] самими - | самом самой самом самих - | - | stems of verbs `to be', `to have', `to do' and modal - | - | быть бы буд быв есть суть - | име - | дел - | мог мож мочь - | уме - | хоч хот - | долж - | можн - | нужн - | нельзя - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_sv.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_sv.txt deleted file mode 100644 index 22bddfd8cb3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_sv.txt +++ /dev/null @@ -1,131 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/swedish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Swedish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | Swedish stop words occasionally exhibit homonym clashes. For example - | så = so, but also seed. These are indicated clearly below. - -och | and -det | it, this/that -att | to (with infinitive) -i | in, at -en | a -jag | I -hon | she -som | who, that -han | he -på | on -den | it, this/that -med | with -var | where, each -sig | him(self) etc -för | for -så | so (also: seed) -till | to -är | is -men | but -ett | a -om | if; around, about -hade | had -de | they, these/those -av | of -icke | not, no -mig | me -du | you -henne | her -då | then, when -sin | his -nu | now -har | have -inte | inte någon = no one -hans | his -honom | him -skulle | 'sake' -hennes | her -där | there -min | my -man | one (pronoun) -ej | nor -vid | at, by, on (also: vast) -kunde | could -något | some etc -från | from, off -ut | out -när | when -efter | after, behind -upp | up -vi | we -dem | them -vara | be -vad | what -över | over -än | than -dig | you -kan | can -sina | his -här | here -ha | have -mot | towards -alla | all -under | under (also: wonder) -någon | some etc -eller | or (else) -allt | all -mycket | much -sedan | since -ju | why -denna | this/that -själv | myself, yourself etc -detta | this/that -åt | to -utan | without -varit | was -hur | how -ingen | no -mitt | my -ni | you -bli | to be, become -blev | from bli -oss | us -din | thy -dessa | these/those -några | some etc -deras | their -blir | from bli -mina | my -samma | (the) same -vilken | who, that -er | you, your -sådan | such a -vår | our -blivit | from bli -dess | its -inom | within -mellan | between -sådant | such a -varför | why -varje | each -vilka | who, that -ditt | thy -vem | who -vilket | who, that -sitta | his -sådana | such a -vart | each -dina | thy -vars | whose -vårt | our -våra | our -ert | your -era | your -vilkas | whose - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_th.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_th.txt deleted file mode 100644 index 07f0fabe692..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_th.txt +++ /dev/null @@ -1,119 +0,0 @@ -# Thai stopwords from: -# "Opinion Detection in Thai Political News Columns -# Based on Subjectivity Analysis" -# Khampol Sukhum, Supot Nitsuwat, and Choochart Haruechaiyasak -ไว้ -ไม่ -ไป -ได้ -ให้ -ใน -โดย -แห่ง -แล้ว -และ -แรก -แบบ -แต่ -เอง -เห็น -เลย -เริ่ม -เรา -เมื่อ -เพื่อ -เพราะ -เป็นการ -เป็น -เปิดเผย -เปิด -เนื่องจาก -เดียวกัน -เดียว -เช่น -เฉพาะ -เคย -เข้า -เขา -อีก -อาจ -อะไร -ออก -อย่าง -อยู่ -อยาก -หาก -หลาย -หลังจาก -หลัง -หรือ -หนึ่ง -ส่วน -ส่ง -สุด -สําหรับ -ว่า -วัน -ลง -ร่วม -ราย -รับ -ระหว่าง -รวม -ยัง -มี -มาก -มา -พร้อม -พบ -ผ่าน -ผล -บาง -น่า -นี้ -นํา -นั้น -นัก -นอกจาก -ทุก -ที่สุด -ที่ -ทําให้ -ทํา -ทาง -ทั้งนี้ -ทั้ง -ถ้า -ถูก -ถึง -ต้อง -ต่างๆ -ต่าง -ต่อ -ตาม -ตั้งแต่ -ตั้ง -ด้าน -ด้วย -ดัง -ซึ่ง -ช่วง -จึง -จาก -จัด -จะ -คือ -ความ -ครั้ง -คง -ขึ้น -ของ -ขอ -ขณะ -ก่อน -ก็ -การ -กับ -กัน -กว่า -กล่าว diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_tr.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_tr.txt deleted file mode 100644 index 84d9408d4ea..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/stopwords_tr.txt +++ /dev/null @@ -1,212 +0,0 @@ -# Turkish stopwords from LUCENE-559 -# merged with the list from "Information Retrieval on Turkish Texts" -# (http://www.users.muohio.edu/canf/papers/JASIST2008offPrint.pdf) -acaba -altmış -altı -ama -ancak -arada -aslında -ayrıca -bana -bazı -belki -ben -benden -beni -benim -beri -beş -bile -bin -bir -birçok -biri -birkaç -birkez -birşey -birşeyi -biz -bize -bizden -bizi -bizim -böyle -böylece -bu -buna -bunda -bundan -bunlar -bunları -bunların -bunu -bunun -burada -çok -çünkü -da -daha -dahi -de -defa -değil -diğer -diye -doksan -dokuz -dolayı -dolayısıyla -dört -edecek -eden -ederek -edilecek -ediliyor -edilmesi -ediyor -eğer -elli -en -etmesi -etti -ettiği -ettiğini -gibi -göre -halen -hangi -hatta -hem -henüz -hep -hepsi -her -herhangi -herkesin -hiç -hiçbir -için -iki -ile -ilgili -ise -işte -itibaren -itibariyle -kadar -karşın -katrilyon -kendi -kendilerine -kendini -kendisi -kendisine -kendisini -kez -ki -kim -kimden -kime -kimi -kimse -kırk -milyar -milyon -mu -mü -mı -nasıl -ne -neden -nedenle -nerde -nerede -nereye -niye -niçin -o -olan -olarak -oldu -olduğu -olduğunu -olduklarını -olmadı -olmadığı -olmak -olması -olmayan -olmaz -olsa -olsun -olup -olur -olursa -oluyor -on -ona -ondan -onlar -onlardan -onları -onların -onu -onun -otuz -oysa -öyle -pek -rağmen -sadece -sanki -sekiz -seksen -sen -senden -seni -senin -siz -sizden -sizi -sizin -şey -şeyden -şeyi -şeyler -şöyle -şu -şuna -şunda -şundan -şunları -şunu -tarafından -trilyon -tüm -üç -üzere -var -vardı -ve -veya -ya -yani -yapacak -yapılan -yapılması -yapıyor -yapmak -yaptı -yaptığı -yaptığını -yaptıkları -yedi -yerine -yetmiş -yine -yirmi -yoksa -yüz -zaten diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/userdict_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/userdict_ja.txt deleted file mode 100644 index 6f0368e4d81..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/lang/userdict_ja.txt +++ /dev/null @@ -1,29 +0,0 @@ -# -# This is a sample user dictionary for Kuromoji (JapaneseTokenizer) -# -# Add entries to this file in order to override the statistical model in terms -# of segmentation, readings and part-of-speech tags. Notice that entries do -# not have weights since they are always used when found. This is by-design -# in order to maximize ease-of-use. -# -# Entries are defined using the following CSV format: -# , ... , ... , -# -# Notice that a single half-width space separates tokens and readings, and -# that the number tokens and readings must match exactly. -# -# Also notice that multiple entries with the same is undefined. -# -# Whitespace only lines are ignored. Comments are not allowed on entry lines. -# - -# Custom segmentation for kanji compounds -日本経済新聞,日本 経済 新聞,ニホン ケイザイ シンブン,カスタム名詞 -関西国際空港,関西 国際 空港,カンサイ コクサイ クウコウ,カスタム名詞 - -# Custom segmentation for compound katakana -トートバッグ,トート バッグ,トート バッグ,かずカナ名詞 -ショルダーバッグ,ショルダー バッグ,ショルダー バッグ,かずカナ名詞 - -# Custom reading for former sumo wrestler -朝青龍,朝青龍,アサショウリュウ,カスタム人名 diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/protwords.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/protwords.txt deleted file mode 100644 index 1dfc0abecbf..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/protwords.txt +++ /dev/null @@ -1,21 +0,0 @@ -# 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. - -#----------------------------------------------------------------------- -# Use a protected word file to protect against the stemmer reducing two -# unrelated words to the same base word. - -# Some non-words that normally won't be encountered, -# just to test that they won't be stemmed. -dontstems -zwhacky - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/schema.xml b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/schema.xml deleted file mode 100644 index 1dee871aca7..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/schema.xml +++ /dev/null @@ -1,941 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - id - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/solrconfig.xml b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/solrconfig.xml deleted file mode 100644 index 9d298d43d12..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/solrconfig.xml +++ /dev/null @@ -1,1446 +0,0 @@ - - - - - - - - - ${tests.luceneMatchVersion:LATEST} - - - - - - - - - - - - - - - - - - - - - - - - ${solr.data.dir:} - - - - - ${solr.hdfs.home:} - ${solr.hdfs.confdir:} - ${solr.hdfs.blockcache.enabled:true} - ${solr.hdfs.blockcache.slab.count:1} - ${solr.hdfs.blockcache.direct.memory.allocation:true} - ${solr.hdfs.blockcache.blocksperbank:16384} - ${solr.hdfs.blockcache.read.enabled:true} - ${solr.hdfs.nrtcachingdirectory.enable:true} - ${solr.hdfs.nrtcachingdirectory.maxmergesizemb:16} - ${solr.hdfs.nrtcachingdirectory.maxcachedmb:192} - ${solr.hdfs.blockcache.global:false} - - - - - - - - - - - - - - - - - - 128 - - - - - - - - - - ${solr.lock.type:hdfs} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ${solr.ulog.dir:} - - - - - ${solr.autoCommit.maxTime:60000} - false - - - - - ${solr.autoSoftCommit.maxTime:1000} - - - - - - - - - - - - - - - - - - - 1024 - - - - - - - - - - - - - - - - - - - - - - true - - - - - - 20 - - - 200 - - - - - - - - - - - - static firstSearcher warming in solrconfig.xml - - - - - - false - - - 4 - - - - - - - - - - - - - - - - - - - - - - - explicit - 10 - text - - - - - - - - - - - - - - explicit - json - true - text - - - - - - - - - text_general - - - - - - default - text - solr.DirectSolrSpellChecker - - internal - - 0.5 - - 2 - - 1 - - 5 - - 4 - - 0.01 - - - - - - wordbreak - solr.WordBreakSolrSpellChecker - name - true - true - 10 - - - - - - - - - - - - - - - - text - - default - wordbreak - on - true - 10 - 5 - 5 - true - true - 10 - 5 - - - spellcheck - - - - - - - - - - text - true - - - tvComponent - - - - - - - - - default - - - org.carrot2.clustering.lingo.LingoClusteringAlgorithm - - - 20 - - - clustering/carrot2 - - - ENGLISH - - - stc - org.carrot2.clustering.stc.STCClusteringAlgorithm - - - - - - - true - default - true - - name - id - - features - - true - - - - false - - edismax - - text^0.5 features^1.0 name^1.2 sku^1.5 id^10.0 manu^1.1 cat^1.4 - - *:* - 10 - *,score - - - clustering - - - - - - - - - - true - false - - - terms - - - - - - - - string - elevate.xml - - - - - - explicit - text - - - elevator - - - - - - - - - - - 100 - - - - - - - - 70 - - 0.5 - - [-\w ,/\n\"']{20,200} - - - - - - - ]]> - ]]> - - - - - - - - - - - - - - - - - - - - - - - - ,, - ,, - ,, - ,, - ,]]> - ]]> - - - - - - 10 - .,!? - - - - - - - WORD - - - en - US - - - - - - - - - - - - - - - - - - - - - - text/plain; charset=UTF-8 - - - - - - - - - 5 - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/stopwords.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/stopwords.txt deleted file mode 100644 index ae1e83eeb3d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/stopwords.txt +++ /dev/null @@ -1,14 +0,0 @@ -# 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. diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/synonyms.txt b/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/synonyms.txt deleted file mode 100644 index 7f72128303b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/synonyms.txt +++ /dev/null @@ -1,29 +0,0 @@ -# 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. - -#----------------------------------------------------------------------- -#some test synonym mappings unlikely to appear in real input text -aaafoo => aaabar -bbbfoo => bbbfoo bbbbar -cccfoo => cccbar cccbaz -fooaaa,baraaa,bazaaa - -# Some synonym groups specific to this example -GB,gib,gigabyte,gigabytes -MB,mib,megabyte,megabytes -Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming -#after us won't split it into two words. - -# Synonym mappings can be used for spelling correction too -pixima => pixma - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/minimr/solr.xml b/solr/contrib/morphlines-core/src/test-files/solr/minimr/solr.xml deleted file mode 100644 index 2fa9a8a3cde..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/minimr/solr.xml +++ /dev/null @@ -1,46 +0,0 @@ - - - - - - - ${shareSchema:false} - - - 127.0.0.1 - ${hostPort:8983} - ${hostContext:solr} - ${genericCoreNodeNames:true} - 0 - ${distribUpdateSoTimeout:120000} - ${distribUpdateConnTimeout:15000} - ${solr.zkclienttimeout:30000} - - - - ${urlScheme:} - ${socketTimeout:120000} - ${connTimeout:15000} - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/currency.xml b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/currency.xml deleted file mode 100644 index 3a9c58afee8..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/currency.xml +++ /dev/null @@ -1,67 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/elevate.xml b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/elevate.xml deleted file mode 100644 index 2c09ebed669..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/elevate.xml +++ /dev/null @@ -1,42 +0,0 @@ - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_ca.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_ca.txt deleted file mode 100644 index 307a85f913d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_ca.txt +++ /dev/null @@ -1,8 +0,0 @@ -# Set of Catalan contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -l -m -n -s -t diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_fr.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_fr.txt deleted file mode 100644 index 722db588333..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_fr.txt +++ /dev/null @@ -1,9 +0,0 @@ -# Set of French contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -l -m -t -qu -n -s -j diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_ga.txt deleted file mode 100644 index 9ebe7fa349a..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -m -b diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_it.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_it.txt deleted file mode 100644 index cac04095372..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/contractions_it.txt +++ /dev/null @@ -1,23 +0,0 @@ -# Set of Italian contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -c -l -all -dall -dell -nell -sull -coll -pell -gl -agl -dagl -degl -negl -sugl -un -m -t -s -v -d diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/hyphenations_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/hyphenations_ga.txt deleted file mode 100644 index 4d2642cc5a3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/hyphenations_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish hyphenations for StopFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -h -n -t diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stemdict_nl.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stemdict_nl.txt deleted file mode 100644 index 441072971d3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stemdict_nl.txt +++ /dev/null @@ -1,6 +0,0 @@ -# Set of overrides for the dutch stemmer -# TODO: load this as a resource from the analyzer and sync it in build.xml -fiets fiets -bromfiets bromfiets -ei eier -kind kinder diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stoptags_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stoptags_ja.txt deleted file mode 100644 index 71b750845e3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stoptags_ja.txt +++ /dev/null @@ -1,420 +0,0 @@ -# -# This file defines a Japanese stoptag set for JapanesePartOfSpeechStopFilter. -# -# Any token with a part-of-speech tag that exactly matches those defined in this -# file are removed from the token stream. -# -# Set your own stoptags by uncommenting the lines below. Note that comments are -# not allowed on the same line as a stoptag. See LUCENE-3745 for frequency lists, -# etc. that can be useful for building you own stoptag set. -# -# The entire possible tagset is provided below for convenience. -# -##### -# noun: unclassified nouns -#名詞 -# -# noun-common: Common nouns or nouns where the sub-classification is undefined -#名詞-一般 -# -# noun-proper: Proper nouns where the sub-classification is undefined -#名詞-固有名詞 -# -# noun-proper-misc: miscellaneous proper nouns -#名詞-固有名詞-一般 -# -# noun-proper-person: Personal names where the sub-classification is undefined -#名詞-固有名詞-人名 -# -# noun-proper-person-misc: names that cannot be divided into surname and -# given name; foreign names; names where the surname or given name is unknown. -# e.g. お市の方 -#名詞-固有名詞-人名-一般 -# -# noun-proper-person-surname: Mainly Japanese surnames. -# e.g. 山田 -#名詞-固有名詞-人名-姓 -# -# noun-proper-person-given_name: Mainly Japanese given names. -# e.g. 太郎 -#名詞-固有名詞-人名-名 -# -# noun-proper-organization: Names representing organizations. -# e.g. 通産省, NHK -#名詞-固有名詞-組織 -# -# noun-proper-place: Place names where the sub-classification is undefined -#名詞-固有名詞-地域 -# -# noun-proper-place-misc: Place names excluding countries. -# e.g. アジア, バルセロナ, 京都 -#名詞-固有名詞-地域-一般 -# -# noun-proper-place-country: Country names. -# e.g. 日本, オーストラリア -#名詞-固有名詞-地域-国 -# -# noun-pronoun: Pronouns where the sub-classification is undefined -#名詞-代名詞 -# -# noun-pronoun-misc: miscellaneous pronouns: -# e.g. それ, ここ, あいつ, あなた, あちこち, いくつ, どこか, なに, みなさん, みんな, わたくし, われわれ -#名詞-代名詞-一般 -# -# noun-pronoun-contraction: Spoken language contraction made by combining a -# pronoun and the particle 'wa'. -# e.g. ありゃ, こりゃ, こりゃあ, そりゃ, そりゃあ -#名詞-代名詞-縮約 -# -# noun-adverbial: Temporal nouns such as names of days or months that behave -# like adverbs. Nouns that represent amount or ratios and can be used adverbially, -# e.g. 金曜, 一月, 午後, 少量 -#名詞-副詞可能 -# -# noun-verbal: Nouns that take arguments with case and can appear followed by -# 'suru' and related verbs (する, できる, なさる, くださる) -# e.g. インプット, 愛着, 悪化, 悪戦苦闘, 一安心, 下取り -#名詞-サ変接続 -# -# noun-adjective-base: The base form of adjectives, words that appear before な ("na") -# e.g. 健康, 安易, 駄目, だめ -#名詞-形容動詞語幹 -# -# noun-numeric: Arabic numbers, Chinese numerals, and counters like 何 (回), 数. -# e.g. 0, 1, 2, 何, 数, 幾 -#名詞-数 -# -# noun-affix: noun affixes where the sub-classification is undefined -#名詞-非自立 -# -# noun-affix-misc: Of adnominalizers, the case-marker の ("no"), and words that -# attach to the base form of inflectional words, words that cannot be classified -# into any of the other categories below. This category includes indefinite nouns. -# e.g. あかつき, 暁, かい, 甲斐, 気, きらい, 嫌い, くせ, 癖, こと, 事, ごと, 毎, しだい, 次第, -# 順, せい, 所為, ついで, 序で, つもり, 積もり, 点, どころ, の, はず, 筈, はずみ, 弾み, -# 拍子, ふう, ふり, 振り, ほう, 方, 旨, もの, 物, 者, ゆえ, 故, ゆえん, 所以, わけ, 訳, -# わり, 割り, 割, ん-口語/, もん-口語/ -#名詞-非自立-一般 -# -# noun-affix-adverbial: noun affixes that that can behave as adverbs. -# e.g. あいだ, 間, あげく, 挙げ句, あと, 後, 余り, 以外, 以降, 以後, 以上, 以前, 一方, うえ, -# 上, うち, 内, おり, 折り, かぎり, 限り, きり, っきり, 結果, ころ, 頃, さい, 際, 最中, さなか, -# 最中, じたい, 自体, たび, 度, ため, 為, つど, 都度, とおり, 通り, とき, 時, ところ, 所, -# とたん, 途端, なか, 中, のち, 後, ばあい, 場合, 日, ぶん, 分, ほか, 他, まえ, 前, まま, -# 儘, 侭, みぎり, 矢先 -#名詞-非自立-副詞可能 -# -# noun-affix-aux: noun affixes treated as 助動詞 ("auxiliary verb") in school grammars -# with the stem よう(だ) ("you(da)"). -# e.g. よう, やう, 様 (よう) -#名詞-非自立-助動詞語幹 -# -# noun-affix-adjective-base: noun affixes that can connect to the indeclinable -# connection form な (aux "da"). -# e.g. みたい, ふう -#名詞-非自立-形容動詞語幹 -# -# noun-special: special nouns where the sub-classification is undefined. -#名詞-特殊 -# -# noun-special-aux: The そうだ ("souda") stem form that is used for reporting news, is -# treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the base -# form of inflectional words. -# e.g. そう -#名詞-特殊-助動詞語幹 -# -# noun-suffix: noun suffixes where the sub-classification is undefined. -#名詞-接尾 -# -# noun-suffix-misc: Of the nouns or stem forms of other parts of speech that connect -# to ガル or タイ and can combine into compound nouns, words that cannot be classified into -# any of the other categories below. In general, this category is more inclusive than -# 接尾語 ("suffix") and is usually the last element in a compound noun. -# e.g. おき, かた, 方, 甲斐 (がい), がかり, ぎみ, 気味, ぐるみ, (~した) さ, 次第, 済 (ず) み, -# よう, (でき)っこ, 感, 観, 性, 学, 類, 面, 用 -#名詞-接尾-一般 -# -# noun-suffix-person: Suffixes that form nouns and attach to person names more often -# than other nouns. -# e.g. 君, 様, 著 -#名詞-接尾-人名 -# -# noun-suffix-place: Suffixes that form nouns and attach to place names more often -# than other nouns. -# e.g. 町, 市, 県 -#名詞-接尾-地域 -# -# noun-suffix-verbal: Of the suffixes that attach to nouns and form nouns, those that -# can appear before スル ("suru"). -# e.g. 化, 視, 分け, 入り, 落ち, 買い -#名詞-接尾-サ変接続 -# -# noun-suffix-aux: The stem form of そうだ (様態) that is used to indicate conditions, -# is treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the -# conjunctive form of inflectional words. -# e.g. そう -#名詞-接尾-助動詞語幹 -# -# noun-suffix-adjective-base: Suffixes that attach to other nouns or the conjunctive -# form of inflectional words and appear before the copula だ ("da"). -# e.g. 的, げ, がち -#名詞-接尾-形容動詞語幹 -# -# noun-suffix-adverbial: Suffixes that attach to other nouns and can behave as adverbs. -# e.g. 後 (ご), 以後, 以降, 以前, 前後, 中, 末, 上, 時 (じ) -#名詞-接尾-副詞可能 -# -# noun-suffix-classifier: Suffixes that attach to numbers and form nouns. This category -# is more inclusive than 助数詞 ("classifier") and includes common nouns that attach -# to numbers. -# e.g. 個, つ, 本, 冊, パーセント, cm, kg, カ月, か国, 区画, 時間, 時半 -#名詞-接尾-助数詞 -# -# noun-suffix-special: Special suffixes that mainly attach to inflecting words. -# e.g. (楽し) さ, (考え) 方 -#名詞-接尾-特殊 -# -# noun-suffix-conjunctive: Nouns that behave like conjunctions and join two words -# together. -# e.g. (日本) 対 (アメリカ), 対 (アメリカ), (3) 対 (5), (女優) 兼 (主婦) -#名詞-接続詞的 -# -# noun-verbal_aux: Nouns that attach to the conjunctive particle て ("te") and are -# semantically verb-like. -# e.g. ごらん, ご覧, 御覧, 頂戴 -#名詞-動詞非自立的 -# -# noun-quotation: text that cannot be segmented into words, proverbs, Chinese poetry, -# dialects, English, etc. Currently, the only entry for 名詞 引用文字列 ("noun quotation") -# is いわく ("iwaku"). -#名詞-引用文字列 -# -# noun-nai_adjective: Words that appear before the auxiliary verb ない ("nai") and -# behave like an adjective. -# e.g. 申し訳, 仕方, とんでも, 違い -#名詞-ナイ形容詞語幹 -# -##### -# prefix: unclassified prefixes -#接頭詞 -# -# prefix-nominal: Prefixes that attach to nouns (including adjective stem forms) -# excluding numerical expressions. -# e.g. お (水), 某 (氏), 同 (社), 故 (~氏), 高 (品質), お (見事), ご (立派) -#接頭詞-名詞接続 -# -# prefix-verbal: Prefixes that attach to the imperative form of a verb or a verb -# in conjunctive form followed by なる/なさる/くださる. -# e.g. お (読みなさい), お (座り) -#接頭詞-動詞接続 -# -# prefix-adjectival: Prefixes that attach to adjectives. -# e.g. お (寒いですねえ), バカ (でかい) -#接頭詞-形容詞接続 -# -# prefix-numerical: Prefixes that attach to numerical expressions. -# e.g. 約, およそ, 毎時 -#接頭詞-数接続 -# -##### -# verb: unclassified verbs -#動詞 -# -# verb-main: -#動詞-自立 -# -# verb-auxiliary: -#動詞-非自立 -# -# verb-suffix: -#動詞-接尾 -# -##### -# adjective: unclassified adjectives -#形容詞 -# -# adjective-main: -#形容詞-自立 -# -# adjective-auxiliary: -#形容詞-非自立 -# -# adjective-suffix: -#形容詞-接尾 -# -##### -# adverb: unclassified adverbs -#副詞 -# -# adverb-misc: Words that can be segmented into one unit and where adnominal -# modification is not possible. -# e.g. あいかわらず, 多分 -#副詞-一般 -# -# adverb-particle_conjunction: Adverbs that can be followed by の, は, に, -# な, する, だ, etc. -# e.g. こんなに, そんなに, あんなに, なにか, なんでも -#副詞-助詞類接続 -# -##### -# adnominal: Words that only have noun-modifying forms. -# e.g. この, その, あの, どの, いわゆる, なんらかの, 何らかの, いろんな, こういう, そういう, ああいう, -# どういう, こんな, そんな, あんな, どんな, 大きな, 小さな, おかしな, ほんの, たいした, -# 「(, も) さる (ことながら)」, 微々たる, 堂々たる, 単なる, いかなる, 我が」「同じ, 亡き -#連体詞 -# -##### -# conjunction: Conjunctions that can occur independently. -# e.g. が, けれども, そして, じゃあ, それどころか -接続詞 -# -##### -# particle: unclassified particles. -助詞 -# -# particle-case: case particles where the subclassification is undefined. -助詞-格助詞 -# -# particle-case-misc: Case particles. -# e.g. から, が, で, と, に, へ, より, を, の, にて -助詞-格助詞-一般 -# -# particle-case-quote: the "to" that appears after nouns, a person’s speech, -# quotation marks, expressions of decisions from a meeting, reasons, judgements, -# conjectures, etc. -# e.g. ( だ) と (述べた.), ( である) と (して執行猶予...) -助詞-格助詞-引用 -# -# particle-case-compound: Compounds of particles and verbs that mainly behave -# like case particles. -# e.g. という, といった, とかいう, として, とともに, と共に, でもって, にあたって, に当たって, に当って, -# にあたり, に当たり, に当り, に当たる, にあたる, において, に於いて,に於て, における, に於ける, -# にかけ, にかけて, にかんし, に関し, にかんして, に関して, にかんする, に関する, に際し, -# に際して, にしたがい, に従い, に従う, にしたがって, に従って, にたいし, に対し, にたいして, -# に対して, にたいする, に対する, について, につき, につけ, につけて, につれ, につれて, にとって, -# にとり, にまつわる, によって, に依って, に因って, により, に依り, に因り, による, に依る, に因る, -# にわたって, にわたる, をもって, を以って, を通じ, を通じて, を通して, をめぐって, をめぐり, をめぐる, -# って-口語/, ちゅう-関西弁「という」/, (何) ていう (人)-口語/, っていう-口語/, といふ, とかいふ -助詞-格助詞-連語 -# -# particle-conjunctive: -# e.g. から, からには, が, けれど, けれども, けど, し, つつ, て, で, と, ところが, どころか, とも, ども, -# ながら, なり, ので, のに, ば, ものの, や ( した), やいなや, (ころん) じゃ(いけない)-口語/, -# (行っ) ちゃ(いけない)-口語/, (言っ) たって (しかたがない)-口語/, (それがなく)ったって (平気)-口語/ -助詞-接続助詞 -# -# particle-dependency: -# e.g. こそ, さえ, しか, すら, は, も, ぞ -助詞-係助詞 -# -# particle-adverbial: -# e.g. がてら, かも, くらい, 位, ぐらい, しも, (学校) じゃ(これが流行っている)-口語/, -# (それ)じゃあ (よくない)-口語/, ずつ, (私) なぞ, など, (私) なり (に), (先生) なんか (大嫌い)-口語/, -# (私) なんぞ, (先生) なんて (大嫌い)-口語/, のみ, だけ, (私) だって-口語/, だに, -# (彼)ったら-口語/, (お茶) でも (いかが), 等 (とう), (今後) とも, ばかり, ばっか-口語/, ばっかり-口語/, -# ほど, 程, まで, 迄, (誰) も (が)([助詞-格助詞] および [助詞-係助詞] の前に位置する「も」) -助詞-副助詞 -# -# particle-interjective: particles with interjective grammatical roles. -# e.g. (松島) や -助詞-間投助詞 -# -# particle-coordinate: -# e.g. と, たり, だの, だり, とか, なり, や, やら -助詞-並立助詞 -# -# particle-final: -# e.g. かい, かしら, さ, ぜ, (だ)っけ-口語/, (とまってる) で-方言/, な, ナ, なあ-口語/, ぞ, ね, ネ, -# ねぇ-口語/, ねえ-口語/, ねん-方言/, の, のう-口語/, や, よ, ヨ, よぉ-口語/, わ, わい-口語/ -助詞-終助詞 -# -# particle-adverbial/conjunctive/final: The particle "ka" when unknown whether it is -# adverbial, conjunctive, or sentence final. For example: -# (a) 「A か B か」. Ex:「(国内で運用する) か,(海外で運用する) か (.)」 -# (b) Inside an adverb phrase. Ex:「(幸いという) か (, 死者はいなかった.)」 -# 「(祈りが届いたせい) か (, 試験に合格した.)」 -# (c) 「かのように」. Ex:「(何もなかった) か (のように振る舞った.)」 -# e.g. か -助詞-副助詞/並立助詞/終助詞 -# -# particle-adnominalizer: The "no" that attaches to nouns and modifies -# non-inflectional words. -助詞-連体化 -# -# particle-adnominalizer: The "ni" and "to" that appear following nouns and adverbs -# that are giongo, giseigo, or gitaigo. -# e.g. に, と -助詞-副詞化 -# -# particle-special: A particle that does not fit into one of the above classifications. -# This includes particles that are used in Tanka, Haiku, and other poetry. -# e.g. かな, けむ, ( しただろう) に, (あんた) にゃ(わからん), (俺) ん (家) -助詞-特殊 -# -##### -# auxiliary-verb: -助動詞 -# -##### -# interjection: Greetings and other exclamations. -# e.g. おはよう, おはようございます, こんにちは, こんばんは, ありがとう, どうもありがとう, ありがとうございます, -# いただきます, ごちそうさま, さよなら, さようなら, はい, いいえ, ごめん, ごめんなさい -#感動詞 -# -##### -# symbol: unclassified Symbols. -記号 -# -# symbol-misc: A general symbol not in one of the categories below. -# e.g. [○◎@$〒→+] -記号-一般 -# -# symbol-comma: Commas -# e.g. [,、] -記号-読点 -# -# symbol-period: Periods and full stops. -# e.g. [..。] -記号-句点 -# -# symbol-space: Full-width whitespace. -記号-空白 -# -# symbol-open_bracket: -# e.g. [({‘“『【] -記号-括弧開 -# -# symbol-close_bracket: -# e.g. [)}’”』」】] -記号-括弧閉 -# -# symbol-alphabetic: -#記号-アルファベット -# -##### -# other: unclassified other -#その他 -# -# other-interjection: Words that are hard to classify as noun-suffixes or -# sentence-final particles. -# e.g. (だ)ァ -その他-間投 -# -##### -# filler: Aizuchi that occurs during a conversation or sounds inserted as filler. -# e.g. あの, うんと, えと -フィラー -# -##### -# non-verbal: non-verbal sound. -非言語音 -# -##### -# fragment: -#語断片 -# -##### -# unknown: unknown part of speech. -#未知語 -# -##### End of file diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ar.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ar.txt deleted file mode 100644 index 046829db6a2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ar.txt +++ /dev/null @@ -1,125 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Cleaned on October 11, 2009 (not normalized, so use before normalization) -# This means that when modifying this list, you might need to add some -# redundant entries, for example containing forms with both أ and ا -من -ومن -منها -منه -في -وفي -فيها -فيه -و -ف -ثم -او -أو -ب -بها -به -ا -أ -اى -اي -أي -أى -لا -ولا -الا -ألا -إلا -لكن -ما -وما -كما -فما -عن -مع -اذا -إذا -ان -أن -إن -انها -أنها -إنها -انه -أنه -إنه -بان -بأن -فان -فأن -وان -وأن -وإن -التى -التي -الذى -الذي -الذين -الى -الي -إلى -إلي -على -عليها -عليه -اما -أما -إما -ايضا -أيضا -كل -وكل -لم -ولم -لن -ولن -هى -هي -هو -وهى -وهي -وهو -فهى -فهي -فهو -انت -أنت -لك -لها -له -هذه -هذا -تلك -ذلك -هناك -كانت -كان -يكون -تكون -وكانت -وكان -غير -بعض -قد -نحو -بين -بينما -منذ -ضمن -حيث -الان -الآن -خلال -بعد -قبل -حتى -عند -عندما -لدى -جميع diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_bg.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_bg.txt deleted file mode 100644 index 1ae4ba2ae38..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_bg.txt +++ /dev/null @@ -1,193 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -а -аз -ако -ала -бе -без -беше -би -бил -била -били -било -близо -бъдат -бъде -бяха -в -вас -ваш -ваша -вероятно -вече -взема -ви -вие -винаги -все -всеки -всички -всичко -всяка -във -въпреки -върху -г -ги -главно -го -д -да -дали -до -докато -докога -дори -досега -доста -е -едва -един -ето -за -зад -заедно -заради -засега -затова -защо -защото -и -из -или -им -има -имат -иска -й -каза -как -каква -какво -както -какъв -като -кога -когато -което -които -кой -който -колко -която -къде -където -към -ли -м -ме -между -мен -ми -мнозина -мога -могат -може -моля -момента -му -н -на -над -назад -най -направи -напред -например -нас -не -него -нея -ни -ние -никой -нито -но -някои -някой -няма -обаче -около -освен -особено -от -отгоре -отново -още -пак -по -повече -повечето -под -поне -поради -после -почти -прави -пред -преди -през -при -пък -първо -с -са -само -се -сега -си -скоро -след -сме -според -сред -срещу -сте -съм -със -също -т -тази -така -такива -такъв -там -твой -те -тези -ти -тн -то -това -тогава -този -той -толкова -точно -трябва -тук -тъй -тя -тях -у -харесва -ч -че -често -чрез -ще -щом -я diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ca.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ca.txt deleted file mode 100644 index 3da65deafe1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ca.txt +++ /dev/null @@ -1,220 +0,0 @@ -# Catalan stopwords from http://github.com/vcl/cue.language (Apache 2 Licensed) -a -abans -ací -ah -així -això -al -als -aleshores -algun -alguna -algunes -alguns -alhora -allà -allí -allò -altra -altre -altres -amb -ambdós -ambdues -apa -aquell -aquella -aquelles -aquells -aquest -aquesta -aquestes -aquests -aquí -baix -cada -cadascú -cadascuna -cadascunes -cadascuns -com -contra -d'un -d'una -d'unes -d'uns -dalt -de -del -dels -des -després -dins -dintre -donat -doncs -durant -e -eh -el -els -em -en -encara -ens -entre -érem -eren -éreu -es -és -esta -està -estàvem -estaven -estàveu -esteu -et -etc -ets -fins -fora -gairebé -ha -han -has -havia -he -hem -heu -hi -ho -i -igual -iguals -ja -l'hi -la -les -li -li'n -llavors -m'he -ma -mal -malgrat -mateix -mateixa -mateixes -mateixos -me -mentre -més -meu -meus -meva -meves -molt -molta -moltes -molts -mon -mons -n'he -n'hi -ne -ni -no -nogensmenys -només -nosaltres -nostra -nostre -nostres -o -oh -oi -on -pas -pel -pels -per -però -perquè -poc -poca -pocs -poques -potser -propi -qual -quals -quan -quant -que -què -quelcom -qui -quin -quina -quines -quins -s'ha -s'han -sa -semblant -semblants -ses -seu -seus -seva -seva -seves -si -sobre -sobretot -sóc -solament -sols -son -són -sons -sota -sou -t'ha -t'han -t'he -ta -tal -també -tampoc -tan -tant -tanta -tantes -teu -teus -teva -teves -ton -tons -tot -tota -totes -tots -un -una -unes -uns -us -va -vaig -vam -van -vas -veu -vosaltres -vostra -vostre -vostres diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_cz.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_cz.txt deleted file mode 100644 index 53c6097dac7..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_cz.txt +++ /dev/null @@ -1,172 +0,0 @@ -a -s -k -o -i -u -v -z -dnes -cz -tímto -budeš -budem -byli -jseš -můj -svým -ta -tomto -tohle -tuto -tyto -jej -zda -proč -máte -tato -kam -tohoto -kdo -kteří -mi -nám -tom -tomuto -mít -nic -proto -kterou -byla -toho -protože -asi -ho -naši -napište -re -což -tím -takže -svých -její -svými -jste -aj -tu -tedy -teto -bylo -kde -ke -pravé -ji -nad -nejsou -či -pod -téma -mezi -přes -ty -pak -vám -ani -když -však -neg -jsem -tento -článku -články -aby -jsme -před -pta -jejich -byl -ještě -až -bez -také -pouze -první -vaše -která -nás -nový -tipy -pokud -může -strana -jeho -své -jiné -zprávy -nové -není -vás -jen -podle -zde -už -být -více -bude -již -než -který -by -které -co -nebo -ten -tak -má -při -od -po -jsou -jak -další -ale -si -se -ve -to -jako -za -zpět -ze -do -pro -je -na -atd -atp -jakmile -přičemž -já -on -ona -ono -oni -ony -my -vy -jí -ji -mě -mne -jemu -tomu -těm -těmu -němu -němuž -jehož -jíž -jelikož -jež -jakož -načež diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_da.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_da.txt deleted file mode 100644 index a3ff5fe122c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_da.txt +++ /dev/null @@ -1,108 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/danish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Danish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - -og | and -i | in -jeg | I -det | that (dem. pronoun)/it (pers. pronoun) -at | that (in front of a sentence)/to (with infinitive) -en | a/an -den | it (pers. pronoun)/that (dem. pronoun) -til | to/at/for/until/against/by/of/into, more -er | present tense of "to be" -som | who, as -på | on/upon/in/on/at/to/after/of/with/for, on -de | they -med | with/by/in, along -han | he -af | of/by/from/off/for/in/with/on, off -for | at/for/to/from/by/of/ago, in front/before, because -ikke | not -der | who/which, there/those -var | past tense of "to be" -mig | me/myself -sig | oneself/himself/herself/itself/themselves -men | but -et | a/an/one, one (number), someone/somebody/one -har | present tense of "to have" -om | round/about/for/in/a, about/around/down, if -vi | we -min | my -havde | past tense of "to have" -ham | him -hun | she -nu | now -over | over/above/across/by/beyond/past/on/about, over/past -da | then, when/as/since -fra | from/off/since, off, since -du | you -ud | out -sin | his/her/its/one's -dem | them -os | us/ourselves -op | up -man | you/one -hans | his -hvor | where -eller | or -hvad | what -skal | must/shall etc. -selv | myself/youself/herself/ourselves etc., even -her | here -alle | all/everyone/everybody etc. -vil | will (verb) -blev | past tense of "to stay/to remain/to get/to become" -kunne | could -ind | in -når | when -være | present tense of "to be" -dog | however/yet/after all -noget | something -ville | would -jo | you know/you see (adv), yes -deres | their/theirs -efter | after/behind/according to/for/by/from, later/afterwards -ned | down -skulle | should -denne | this -end | than -dette | this -mit | my/mine -også | also -under | under/beneath/below/during, below/underneath -have | have -dig | you -anden | other -hende | her -mine | my -alt | everything -meget | much/very, plenty of -sit | his, her, its, one's -sine | his, her, its, one's -vor | our -mod | against -disse | these -hvis | if -din | your/yours -nogle | some -hos | by/at -blive | be/become -mange | many -ad | by/through -bliver | present tense of "to be/to become" -hendes | her/hers -været | be -thi | for (conj) -jer | you -sådan | such, like this/like that diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_de.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_de.txt deleted file mode 100644 index f7703841887..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_de.txt +++ /dev/null @@ -1,292 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/german/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A German stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | The number of forms in this list is reduced significantly by passing it - | through the German stemmer. - - -aber | but - -alle | all -allem -allen -aller -alles - -als | than, as -also | so -am | an + dem -an | at - -ander | other -andere -anderem -anderen -anderer -anderes -anderm -andern -anderr -anders - -auch | also -auf | on -aus | out of -bei | by -bin | am -bis | until -bist | art -da | there -damit | with it -dann | then - -der | the -den -des -dem -die -das - -daß | that - -derselbe | the same -derselben -denselben -desselben -demselben -dieselbe -dieselben -dasselbe - -dazu | to that - -dein | thy -deine -deinem -deinen -deiner -deines - -denn | because - -derer | of those -dessen | of him - -dich | thee -dir | to thee -du | thou - -dies | this -diese -diesem -diesen -dieser -dieses - - -doch | (several meanings) -dort | (over) there - - -durch | through - -ein | a -eine -einem -einen -einer -eines - -einig | some -einige -einigem -einigen -einiger -einiges - -einmal | once - -er | he -ihn | him -ihm | to him - -es | it -etwas | something - -euer | your -eure -eurem -euren -eurer -eures - -für | for -gegen | towards -gewesen | p.p. of sein -hab | have -habe | have -haben | have -hat | has -hatte | had -hatten | had -hier | here -hin | there -hinter | behind - -ich | I -mich | me -mir | to me - - -ihr | you, to her -ihre -ihrem -ihren -ihrer -ihres -euch | to you - -im | in + dem -in | in -indem | while -ins | in + das -ist | is - -jede | each, every -jedem -jeden -jeder -jedes - -jene | that -jenem -jenen -jener -jenes - -jetzt | now -kann | can - -kein | no -keine -keinem -keinen -keiner -keines - -können | can -könnte | could -machen | do -man | one - -manche | some, many a -manchem -manchen -mancher -manches - -mein | my -meine -meinem -meinen -meiner -meines - -mit | with -muss | must -musste | had to -nach | to(wards) -nicht | not -nichts | nothing -noch | still, yet -nun | now -nur | only -ob | whether -oder | or -ohne | without -sehr | very - -sein | his -seine -seinem -seinen -seiner -seines - -selbst | self -sich | herself - -sie | they, she -ihnen | to them - -sind | are -so | so - -solche | such -solchem -solchen -solcher -solches - -soll | shall -sollte | should -sondern | but -sonst | else -über | over -um | about, around -und | and - -uns | us -unse -unsem -unsen -unser -unses - -unter | under -viel | much -vom | von + dem -von | from -vor | before -während | while -war | was -waren | were -warst | wast -was | what -weg | away, off -weil | because -weiter | further - -welche | which -welchem -welchen -welcher -welches - -wenn | when -werde | will -werden | will -wie | how -wieder | again -will | want -wir | we -wird | will -wirst | willst -wo | where -wollen | want -wollte | wanted -würde | would -würden | would -zu | to -zum | zu + dem -zur | zu + der -zwar | indeed -zwischen | between - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_el.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_el.txt deleted file mode 100644 index 232681f5bd6..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_el.txt +++ /dev/null @@ -1,78 +0,0 @@ -# Lucene Greek Stopwords list -# Note: by default this file is used after GreekLowerCaseFilter, -# so when modifying this file use 'σ' instead of 'ς' -ο -η -το -οι -τα -του -τησ -των -τον -την -και -κι -κ -ειμαι -εισαι -ειναι -ειμαστε -ειστε -στο -στον -στη -στην -μα -αλλα -απο -για -προσ -με -σε -ωσ -παρα -αντι -κατα -μετα -θα -να -δε -δεν -μη -μην -επι -ενω -εαν -αν -τοτε -που -πωσ -ποιοσ -ποια -ποιο -ποιοι -ποιεσ -ποιων -ποιουσ -αυτοσ -αυτη -αυτο -αυτοι -αυτων -αυτουσ -αυτεσ -αυτα -εκεινοσ -εκεινη -εκεινο -εκεινοι -εκεινεσ -εκεινα -εκεινων -εκεινουσ -οπωσ -ομωσ -ισωσ -οσο -οτι diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_en.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_en.txt deleted file mode 100644 index 2c164c0b2a1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_en.txt +++ /dev/null @@ -1,54 +0,0 @@ -# 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. - -# a couple of test stopwords to test that the words are really being -# configured from this file: -stopworda -stopwordb - -# Standard english stop words taken from Lucene's StopAnalyzer -a -an -and -are -as -at -be -but -by -for -if -in -into -is -it -no -not -of -on -or -such -that -the -their -then -there -these -they -this -to -was -will -with diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_es.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_es.txt deleted file mode 100644 index 2db14760075..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_es.txt +++ /dev/null @@ -1,354 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/spanish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Spanish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | from, of -la | the, her -que | who, that -el | the -en | in -y | and -a | to -los | the, them -del | de + el -se | himself, from him etc -las | the, them -por | for, by, etc -un | a -para | for -con | with -no | no -una | a -su | his, her -al | a + el - | es from SER -lo | him -como | how -más | more -pero | pero -sus | su plural -le | to him, her -ya | already -o | or - | fue from SER -este | this - | ha from HABER -sí | himself etc -porque | because -esta | this - | son from SER -entre | between - | está from ESTAR -cuando | when -muy | very -sin | without -sobre | on - | ser from SER - | tiene from TENER -también | also -me | me -hasta | until -hay | there is/are -donde | where - | han from HABER -quien | whom, that - | están from ESTAR - | estado from ESTAR -desde | from -todo | all -nos | us -durante | during - | estados from ESTAR -todos | all -uno | a -les | to them -ni | nor -contra | against -otros | other - | fueron from SER -ese | that -eso | that - | había from HABER -ante | before -ellos | they -e | and (variant of y) -esto | this -mí | me -antes | before -algunos | some -qué | what? -unos | a -yo | I -otro | other -otras | other -otra | other -él | he -tanto | so much, many -esa | that -estos | these -mucho | much, many -quienes | who -nada | nothing -muchos | many -cual | who - | sea from SER -poco | few -ella | she -estar | to be - | haber from HABER -estas | these - | estaba from ESTAR - | estamos from ESTAR -algunas | some -algo | something -nosotros | we - - | other forms - -mi | me -mis | mi plural -tú | thou -te | thee -ti | thee -tu | thy -tus | tu plural -ellas | they -nosotras | we -vosotros | you -vosotras | you -os | you -mío | mine -mía | -míos | -mías | -tuyo | thine -tuya | -tuyos | -tuyas | -suyo | his, hers, theirs -suya | -suyos | -suyas | -nuestro | ours -nuestra | -nuestros | -nuestras | -vuestro | yours -vuestra | -vuestros | -vuestras | -esos | those -esas | those - - | forms of estar, to be (not including the infinitive): -estoy -estás -está -estamos -estáis -están -esté -estés -estemos -estéis -estén -estaré -estarás -estará -estaremos -estaréis -estarán -estaría -estarías -estaríamos -estaríais -estarían -estaba -estabas -estábamos -estabais -estaban -estuve -estuviste -estuvo -estuvimos -estuvisteis -estuvieron -estuviera -estuvieras -estuviéramos -estuvierais -estuvieran -estuviese -estuvieses -estuviésemos -estuvieseis -estuviesen -estando -estado -estada -estados -estadas -estad - - | forms of haber, to have (not including the infinitive): -he -has -ha -hemos -habéis -han -haya -hayas -hayamos -hayáis -hayan -habré -habrás -habrá -habremos -habréis -habrán -habría -habrías -habríamos -habríais -habrían -había -habías -habíamos -habíais -habían -hube -hubiste -hubo -hubimos -hubisteis -hubieron -hubiera -hubieras -hubiéramos -hubierais -hubieran -hubiese -hubieses -hubiésemos -hubieseis -hubiesen -habiendo -habido -habida -habidos -habidas - - | forms of ser, to be (not including the infinitive): -soy -eres -es -somos -sois -son -sea -seas -seamos -seáis -sean -seré -serás -será -seremos -seréis -serán -sería -serías -seríamos -seríais -serían -era -eras -éramos -erais -eran -fui -fuiste -fue -fuimos -fuisteis -fueron -fuera -fueras -fuéramos -fuerais -fueran -fuese -fueses -fuésemos -fueseis -fuesen -siendo -sido - | sed also means 'thirst' - - | forms of tener, to have (not including the infinitive): -tengo -tienes -tiene -tenemos -tenéis -tienen -tenga -tengas -tengamos -tengáis -tengan -tendré -tendrás -tendrá -tendremos -tendréis -tendrán -tendría -tendrías -tendríamos -tendríais -tendrían -tenía -tenías -teníamos -teníais -tenían -tuve -tuviste -tuvo -tuvimos -tuvisteis -tuvieron -tuviera -tuvieras -tuviéramos -tuvierais -tuvieran -tuviese -tuvieses -tuviésemos -tuvieseis -tuviesen -teniendo -tenido -tenida -tenidos -tenidas -tened - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_eu.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_eu.txt deleted file mode 100644 index 25f1db93460..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_eu.txt +++ /dev/null @@ -1,99 +0,0 @@ -# example set of basque stopwords -al -anitz -arabera -asko -baina -bat -batean -batek -bati -batzuei -batzuek -batzuetan -batzuk -bera -beraiek -berau -berauek -bere -berori -beroriek -beste -bezala -da -dago -dira -ditu -du -dute -edo -egin -ere -eta -eurak -ez -gainera -gu -gutxi -guzti -haiei -haiek -haietan -hainbeste -hala -han -handik -hango -hara -hari -hark -hartan -hau -hauei -hauek -hauetan -hemen -hemendik -hemengo -hi -hona -honek -honela -honetan -honi -hor -hori -horiei -horiek -horietan -horko -horra -horrek -horrela -horretan -horri -hortik -hura -izan -ni -noiz -nola -non -nondik -nongo -nor -nora -ze -zein -zen -zenbait -zenbat -zer -zergatik -ziren -zituen -zu -zuek -zuen -zuten diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fa.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fa.txt deleted file mode 100644 index 723641c6da7..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fa.txt +++ /dev/null @@ -1,313 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Note: by default this file is used after normalization, so when adding entries -# to this file, use the arabic 'ي' instead of 'ی' -انان -نداشته -سراسر -خياه -ايشان -وي -تاكنون -بيشتري -دوم -پس -ناشي -وگو -يا -داشتند -سپس -هنگام -هرگز -پنج -نشان -امسال -ديگر -گروهي -شدند -چطور -ده -و -دو -نخستين -ولي -چرا -چه -وسط -ه -كدام -قابل -يك -رفت -هفت -همچنين -در -هزار -بله -بلي -شايد -اما -شناسي -گرفته -دهد -داشته -دانست -داشتن -خواهيم -ميليارد -وقتيكه -امد -خواهد -جز -اورده -شده -بلكه -خدمات -شدن -برخي -نبود -بسياري -جلوگيري -حق -كردند -نوعي -بعري -نكرده -نظير -نبايد -بوده -بودن -داد -اورد -هست -جايي -شود -دنبال -داده -بايد -سابق -هيچ -همان -انجا -كمتر -كجاست -گردد -كسي -تر -مردم -تان -دادن -بودند -سري -جدا -ندارند -مگر -يكديگر -دارد -دهند -بنابراين -هنگامي -سمت -جا -انچه -خود -دادند -زياد -دارند -اثر -بدون -بهترين -بيشتر -البته -به -براساس -بيرون -كرد -بعضي -گرفت -توي -اي -ميليون -او -جريان -تول -بر -مانند -برابر -باشيم -مدتي -گويند -اكنون -تا -تنها -جديد -چند -بي -نشده -كردن -كردم -گويد -كرده -كنيم -نمي -نزد -روي -قصد -فقط -بالاي -ديگران -اين -ديروز -توسط -سوم -ايم -دانند -سوي -استفاده -شما -كنار -داريم -ساخته -طور -امده -رفته -نخست -بيست -نزديك -طي -كنيد -از -انها -تمامي -داشت -يكي -طريق -اش -چيست -روب -نمايد -گفت -چندين -چيزي -تواند -ام -ايا -با -ان -ايد -ترين -اينكه -ديگري -راه -هايي -بروز -همچنان -پاعين -كس -حدود -مختلف -مقابل -چيز -گيرد -ندارد -ضد -همچون -سازي -شان -مورد -باره -مرسي -خويش -برخوردار -چون -خارج -شش -هنوز -تحت -ضمن -هستيم -گفته -فكر -بسيار -پيش -براي -روزهاي -انكه -نخواهد -بالا -كل -وقتي -كي -چنين -كه -گيري -نيست -است -كجا -كند -نيز -يابد -بندي -حتي -توانند -عقب -خواست -كنند -بين -تمام -همه -ما -باشند -مثل -شد -اري -باشد -اره -طبق -بعد -اگر -صورت -غير -جاي -بيش -ريزي -اند -زيرا -چگونه -بار -لطفا -مي -درباره -من -ديده -همين -گذاري -برداري -علت -گذاشته -هم -فوق -نه -ها -شوند -اباد -همواره -هر -اول -خواهند -چهار -نام -امروز -مان -هاي -قبل -كنم -سعي -تازه -را -هستند -زير -جلوي -عنوان -بود diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fi.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fi.txt deleted file mode 100644 index addad798c4b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fi.txt +++ /dev/null @@ -1,95 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/finnish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - -| forms of BE - -olla -olen -olet -on -olemme -olette -ovat -ole | negative form - -oli -olisi -olisit -olisin -olisimme -olisitte -olisivat -olit -olin -olimme -olitte -olivat -ollut -olleet - -en | negation -et -ei -emme -ette -eivät - -|Nom Gen Acc Part Iness Elat Illat Adess Ablat Allat Ess Trans -minä minun minut minua minussa minusta minuun minulla minulta minulle | I -sinä sinun sinut sinua sinussa sinusta sinuun sinulla sinulta sinulle | you -hän hänen hänet häntä hänessä hänestä häneen hänellä häneltä hänelle | he she -me meidän meidät meitä meissä meistä meihin meillä meiltä meille | we -te teidän teidät teitä teissä teistä teihin teillä teiltä teille | you -he heidän heidät heitä heissä heistä heihin heillä heiltä heille | they - -tämä tämän tätä tässä tästä tähän tallä tältä tälle tänä täksi | this -tuo tuon tuotä tuossa tuosta tuohon tuolla tuolta tuolle tuona tuoksi | that -se sen sitä siinä siitä siihen sillä siltä sille sinä siksi | it -nämä näiden näitä näissä näistä näihin näillä näiltä näille näinä näiksi | these -nuo noiden noita noissa noista noihin noilla noilta noille noina noiksi | those -ne niiden niitä niissä niistä niihin niillä niiltä niille niinä niiksi | they - -kuka kenen kenet ketä kenessä kenestä keneen kenellä keneltä kenelle kenenä keneksi| who -ketkä keiden ketkä keitä keissä keistä keihin keillä keiltä keille keinä keiksi | (pl) -mikä minkä minkä mitä missä mistä mihin millä miltä mille minä miksi | which what -mitkä | (pl) - -joka jonka jota jossa josta johon jolla jolta jolle jona joksi | who which -jotka joiden joita joissa joista joihin joilla joilta joille joina joiksi | (pl) - -| conjunctions - -että | that -ja | and -jos | if -koska | because -kuin | than -mutta | but -niin | so -sekä | and -sillä | for -tai | or -vaan | but -vai | or -vaikka | although - - -| prepositions - -kanssa | with -mukaan | according to -noin | about -poikki | across -yli | over, across - -| other - -kun | when -niin | so -nyt | now -itse | self - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fr.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fr.txt deleted file mode 100644 index c00837ea939..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_fr.txt +++ /dev/null @@ -1,183 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/french/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A French stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -au | a + le -aux | a + les -avec | with -ce | this -ces | these -dans | with -de | of -des | de + les -du | de + le -elle | she -en | `of them' etc -et | and -eux | them -il | he -je | I -la | the -le | the -leur | their -lui | him -ma | my (fem) -mais | but -me | me -même | same; as in moi-même (myself) etc -mes | me (pl) -moi | me -mon | my (masc) -ne | not -nos | our (pl) -notre | our -nous | we -on | one -ou | where -par | by -pas | not -pour | for -qu | que before vowel -que | that -qui | who -sa | his, her (fem) -se | oneself -ses | his (pl) -son | his, her (masc) -sur | on -ta | thy (fem) -te | thee -tes | thy (pl) -toi | thee -ton | thy (masc) -tu | thou -un | a -une | a -vos | your (pl) -votre | your -vous | you - - | single letter forms - -c | c' -d | d' -j | j' -l | l' -à | to, at -m | m' -n | n' -s | s' -t | t' -y | there - - | forms of être (not including the infinitive): -été -étée -étées -étés -étant -suis -es -est -sommes -êtes -sont -serai -seras -sera -serons -serez -seront -serais -serait -serions -seriez -seraient -étais -était -étions -étiez -étaient -fus -fut -fûmes -fûtes -furent -sois -soit -soyons -soyez -soient -fusse -fusses -fût -fussions -fussiez -fussent - - | forms of avoir (not including the infinitive): -ayant -eu -eue -eues -eus -ai -as -avons -avez -ont -aurai -auras -aura -aurons -aurez -auront -aurais -aurait -aurions -auriez -auraient -avais -avait -avions -aviez -avaient -eut -eûmes -eûtes -eurent -aie -aies -ait -ayons -ayez -aient -eusse -eusses -eût -eussions -eussiez -eussent - - | Later additions (from Jean-Christophe Deschamps) -ceci | this -celà  | that -cet | this -cette | this -ici | here -ils | they -les | the (pl) -leurs | their (pl) -quel | which -quels | which -quelle | which -quelles | which -sans | without -soi | oneself - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ga.txt deleted file mode 100644 index 9ff88d747e5..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ga.txt +++ /dev/null @@ -1,110 +0,0 @@ - -a -ach -ag -agus -an -aon -ar -arna -as -b' -ba -beirt -bhúr -caoga -ceathair -ceathrar -chomh -chtó -chuig -chun -cois -céad -cúig -cúigear -d' -daichead -dar -de -deich -deichniúr -den -dhá -do -don -dtí -dá -dár -dó -faoi -faoin -faoina -faoinár -fara -fiche -gach -gan -go -gur -haon -hocht -i -iad -idir -in -ina -ins -inár -is -le -leis -lena -lenár -m' -mar -mo -mé -na -nach -naoi -naonúr -ná -ní -níor -nó -nócha -ocht -ochtar -os -roimh -sa -seacht -seachtar -seachtó -seasca -seisear -siad -sibh -sinn -sna -sé -sí -tar -thar -thú -triúr -trí -trína -trínár -tríocha -tú -um -ár -é -éis -í -ó -ón -óna -ónár diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_gl.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_gl.txt deleted file mode 100644 index d8760b12c14..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_gl.txt +++ /dev/null @@ -1,161 +0,0 @@ -# galican stopwords -a -aínda -alí -aquel -aquela -aquelas -aqueles -aquilo -aquí -ao -aos -as -así -á -ben -cando -che -co -coa -comigo -con -connosco -contigo -convosco -coas -cos -cun -cuns -cunha -cunhas -da -dalgunha -dalgunhas -dalgún -dalgúns -das -de -del -dela -delas -deles -desde -deste -do -dos -dun -duns -dunha -dunhas -e -el -ela -elas -eles -en -era -eran -esa -esas -ese -eses -esta -estar -estaba -está -están -este -estes -estiven -estou -eu -é -facer -foi -foron -fun -había -hai -iso -isto -la -las -lle -lles -lo -los -mais -me -meu -meus -min -miña -miñas -moi -na -nas -neste -nin -no -non -nos -nosa -nosas -noso -nosos -nós -nun -nunha -nuns -nunhas -o -os -ou -ó -ós -para -pero -pode -pois -pola -polas -polo -polos -por -que -se -senón -ser -seu -seus -sexa -sido -sobre -súa -súas -tamén -tan -te -ten -teñen -teño -ter -teu -teus -ti -tido -tiña -tiven -túa -túas -un -unha -unhas -uns -vos -vosa -vosas -voso -vosos -vós diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hi.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hi.txt deleted file mode 100644 index 86286bb083b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hi.txt +++ /dev/null @@ -1,235 +0,0 @@ -# Also see http://www.opensource.org/licenses/bsd-license.html -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# This file was created by Jacques Savoy and is distributed under the BSD license. -# Note: by default this file also contains forms normalized by HindiNormalizer -# for spelling variation (see section below), such that it can be used whether or -# not you enable that feature. When adding additional entries to this list, -# please add the normalized form as well. -अंदर -अत -अपना -अपनी -अपने -अभी -आदि -आप -इत्यादि -इन -इनका -इन्हीं -इन्हें -इन्हों -इस -इसका -इसकी -इसके -इसमें -इसी -इसे -उन -उनका -उनकी -उनके -उनको -उन्हीं -उन्हें -उन्हों -उस -उसके -उसी -उसे -एक -एवं -एस -ऐसे -और -कई -कर -करता -करते -करना -करने -करें -कहते -कहा -का -काफ़ी -कि -कितना -किन्हें -किन्हों -किया -किर -किस -किसी -किसे -की -कुछ -कुल -के -को -कोई -कौन -कौनसा -गया -घर -जब -जहाँ -जा -जितना -जिन -जिन्हें -जिन्हों -जिस -जिसे -जीधर -जैसा -जैसे -जो -तक -तब -तरह -तिन -तिन्हें -तिन्हों -तिस -तिसे -तो -था -थी -थे -दबारा -दिया -दुसरा -दूसरे -दो -द्वारा -न -नहीं -ना -निहायत -नीचे -ने -पर -पर -पहले -पूरा -पे -फिर -बनी -बही -बहुत -बाद -बाला -बिलकुल -भी -भीतर -मगर -मानो -मे -में -यदि -यह -यहाँ -यही -या -यिह -ये -रखें -रहा -रहे -ऱ्वासा -लिए -लिये -लेकिन -व -वर्ग -वह -वह -वहाँ -वहीं -वाले -वुह -वे -वग़ैरह -संग -सकता -सकते -सबसे -सभी -साथ -साबुत -साभ -सारा -से -सो -ही -हुआ -हुई -हुए -है -हैं -हो -होता -होती -होते -होना -होने -# additional normalized forms of the above -अपनि -जेसे -होति -सभि -तिंहों -इंहों -दवारा -इसि -किंहें -थि -उंहों -ओर -जिंहें -वहिं -अभि -बनि -हि -उंहिं -उंहें -हें -वगेरह -एसे -रवासा -कोन -निचे -काफि -उसि -पुरा -भितर -हे -बहि -वहां -कोइ -यहां -जिंहों -तिंहें -किसि -कइ -यहि -इंहिं -जिधर -इंहें -अदि -इतयादि -हुइ -कोनसा -इसकि -दुसरे -जहां -अप -किंहों -उनकि -भि -वरग -हुअ -जेसा -नहिं diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hu.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hu.txt deleted file mode 100644 index 1a96f1db6f2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hu.txt +++ /dev/null @@ -1,209 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/hungarian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - -| Hungarian stop word list -| prepared by Anna Tordai - -a -ahogy -ahol -aki -akik -akkor -alatt -által -általában -amely -amelyek -amelyekben -amelyeket -amelyet -amelynek -ami -amit -amolyan -amíg -amikor -át -abban -ahhoz -annak -arra -arról -az -azok -azon -azt -azzal -azért -aztán -azután -azonban -bár -be -belül -benne -cikk -cikkek -cikkeket -csak -de -e -eddig -egész -egy -egyes -egyetlen -egyéb -egyik -egyre -ekkor -el -elég -ellen -elő -először -előtt -első -én -éppen -ebben -ehhez -emilyen -ennek -erre -ez -ezt -ezek -ezen -ezzel -ezért -és -fel -felé -hanem -hiszen -hogy -hogyan -igen -így -illetve -ill. -ill -ilyen -ilyenkor -ison -ismét -itt -jó -jól -jobban -kell -kellett -keresztül -keressünk -ki -kívül -között -közül -legalább -lehet -lehetett -legyen -lenne -lenni -lesz -lett -maga -magát -majd -majd -már -más -másik -meg -még -mellett -mert -mely -melyek -mi -mit -míg -miért -milyen -mikor -minden -mindent -mindenki -mindig -mint -mintha -mivel -most -nagy -nagyobb -nagyon -ne -néha -nekem -neki -nem -néhány -nélkül -nincs -olyan -ott -össze -ő -ők -őket -pedig -persze -rá -s -saját -sem -semmi -sok -sokat -sokkal -számára -szemben -szerint -szinte -talán -tehát -teljes -tovább -továbbá -több -úgy -ugyanis -új -újabb -újra -után -utána -utolsó -vagy -vagyis -valaki -valami -valamint -való -vagyok -van -vannak -volt -voltam -voltak -voltunk -vissza -vele -viszont -volna diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hy.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hy.txt deleted file mode 100644 index 60c1c50fbc8..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_hy.txt +++ /dev/null @@ -1,46 +0,0 @@ -# example set of Armenian stopwords. -այդ -այլ -այն -այս -դու -դուք -եմ -են -ենք -ես -եք -է -էի -էին -էինք -էիր -էիք -էր -ըստ -թ -ի -ին -իսկ -իր -կամ -համար -հետ -հետո -մենք -մեջ -մի -ն -նա -նաև -նրա -նրանք -որ -որը -որոնք -որպես -ու -ում -պիտի -վրա -և diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_id.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_id.txt deleted file mode 100644 index 4617f83a5c5..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_id.txt +++ /dev/null @@ -1,359 +0,0 @@ -# from appendix D of: A Study of Stemming Effects on Information -# Retrieval in Bahasa Indonesia -ada -adanya -adalah -adapun -agak -agaknya -agar -akan -akankah -akhirnya -aku -akulah -amat -amatlah -anda -andalah -antar -diantaranya -antara -antaranya -diantara -apa -apaan -mengapa -apabila -apakah -apalagi -apatah -atau -ataukah -ataupun -bagai -bagaikan -sebagai -sebagainya -bagaimana -bagaimanapun -sebagaimana -bagaimanakah -bagi -bahkan -bahwa -bahwasanya -sebaliknya -banyak -sebanyak -beberapa -seberapa -begini -beginian -beginikah -beginilah -sebegini -begitu -begitukah -begitulah -begitupun -sebegitu -belum -belumlah -sebelum -sebelumnya -sebenarnya -berapa -berapakah -berapalah -berapapun -betulkah -sebetulnya -biasa -biasanya -bila -bilakah -bisa -bisakah -sebisanya -boleh -bolehkah -bolehlah -buat -bukan -bukankah -bukanlah -bukannya -cuma -percuma -dahulu -dalam -dan -dapat -dari -daripada -dekat -demi -demikian -demikianlah -sedemikian -dengan -depan -di -dia -dialah -dini -diri -dirinya -terdiri -dong -dulu -enggak -enggaknya -entah -entahlah -terhadap -terhadapnya -hal -hampir -hanya -hanyalah -harus -haruslah -harusnya -seharusnya -hendak -hendaklah -hendaknya -hingga -sehingga -ia -ialah -ibarat -ingin -inginkah -inginkan -ini -inikah -inilah -itu -itukah -itulah -jangan -jangankan -janganlah -jika -jikalau -juga -justru -kala -kalau -kalaulah -kalaupun -kalian -kami -kamilah -kamu -kamulah -kan -kapan -kapankah -kapanpun -dikarenakan -karena -karenanya -ke -kecil -kemudian -kenapa -kepada -kepadanya -ketika -seketika -khususnya -kini -kinilah -kiranya -sekiranya -kita -kitalah -kok -lagi -lagian -selagi -lah -lain -lainnya -melainkan -selaku -lalu -melalui -terlalu -lama -lamanya -selama -selama -selamanya -lebih -terlebih -bermacam -macam -semacam -maka -makanya -makin -malah -malahan -mampu -mampukah -mana -manakala -manalagi -masih -masihkah -semasih -masing -mau -maupun -semaunya -memang -mereka -merekalah -meski -meskipun -semula -mungkin -mungkinkah -nah -namun -nanti -nantinya -nyaris -oleh -olehnya -seorang -seseorang -pada -padanya -padahal -paling -sepanjang -pantas -sepantasnya -sepantasnyalah -para -pasti -pastilah -per -pernah -pula -pun -merupakan -rupanya -serupa -saat -saatnya -sesaat -saja -sajalah -saling -bersama -sama -sesama -sambil -sampai -sana -sangat -sangatlah -saya -sayalah -se -sebab -sebabnya -sebuah -tersebut -tersebutlah -sedang -sedangkan -sedikit -sedikitnya -segala -segalanya -segera -sesegera -sejak -sejenak -sekali -sekalian -sekalipun -sesekali -sekaligus -sekarang -sekarang -sekitar -sekitarnya -sela -selain -selalu -seluruh -seluruhnya -semakin -sementara -sempat -semua -semuanya -sendiri -sendirinya -seolah -seperti -sepertinya -sering -seringnya -serta -siapa -siapakah -siapapun -disini -disinilah -sini -sinilah -sesuatu -sesuatunya -suatu -sesudah -sesudahnya -sudah -sudahkah -sudahlah -supaya -tadi -tadinya -tak -tanpa -setelah -telah -tentang -tentu -tentulah -tentunya -tertentu -seterusnya -tapi -tetapi -setiap -tiap -setidaknya -tidak -tidakkah -tidaklah -toh -waduh -wah -wahai -sewaktu -walau -walaupun -wong -yaitu -yakni -yang diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_it.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_it.txt deleted file mode 100644 index 4cb5b0891b1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_it.txt +++ /dev/null @@ -1,301 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/italian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | An Italian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -ad | a (to) before vowel -al | a + il -allo | a + lo -ai | a + i -agli | a + gli -all | a + l' -agl | a + gl' -alla | a + la -alle | a + le -con | with -col | con + il -coi | con + i (forms collo, cogli etc are now very rare) -da | from -dal | da + il -dallo | da + lo -dai | da + i -dagli | da + gli -dall | da + l' -dagl | da + gll' -dalla | da + la -dalle | da + le -di | of -del | di + il -dello | di + lo -dei | di + i -degli | di + gli -dell | di + l' -degl | di + gl' -della | di + la -delle | di + le -in | in -nel | in + el -nello | in + lo -nei | in + i -negli | in + gli -nell | in + l' -negl | in + gl' -nella | in + la -nelle | in + le -su | on -sul | su + il -sullo | su + lo -sui | su + i -sugli | su + gli -sull | su + l' -sugl | su + gl' -sulla | su + la -sulle | su + le -per | through, by -tra | among -contro | against -io | I -tu | thou -lui | he -lei | she -noi | we -voi | you -loro | they -mio | my -mia | -miei | -mie | -tuo | -tua | -tuoi | thy -tue | -suo | -sua | -suoi | his, her -sue | -nostro | our -nostra | -nostri | -nostre | -vostro | your -vostra | -vostri | -vostre | -mi | me -ti | thee -ci | us, there -vi | you, there -lo | him, the -la | her, the -li | them -le | them, the -gli | to him, the -ne | from there etc -il | the -un | a -uno | a -una | a -ma | but -ed | and -se | if -perché | why, because -anche | also -come | how -dov | where (as dov') -dove | where -che | who, that -chi | who -cui | whom -non | not -più | more -quale | who, that -quanto | how much -quanti | -quanta | -quante | -quello | that -quelli | -quella | -quelle | -questo | this -questi | -questa | -queste | -si | yes -tutto | all -tutti | all - - | single letter forms: - -a | at -c | as c' for ce or ci -e | and -i | the -l | as l' -o | or - - | forms of avere, to have (not including the infinitive): - -ho -hai -ha -abbiamo -avete -hanno -abbia -abbiate -abbiano -avrò -avrai -avrà -avremo -avrete -avranno -avrei -avresti -avrebbe -avremmo -avreste -avrebbero -avevo -avevi -aveva -avevamo -avevate -avevano -ebbi -avesti -ebbe -avemmo -aveste -ebbero -avessi -avesse -avessimo -avessero -avendo -avuto -avuta -avuti -avute - - | forms of essere, to be (not including the infinitive): -sono -sei -è -siamo -siete -sia -siate -siano -sarò -sarai -sarà -saremo -sarete -saranno -sarei -saresti -sarebbe -saremmo -sareste -sarebbero -ero -eri -era -eravamo -eravate -erano -fui -fosti -fu -fummo -foste -furono -fossi -fosse -fossimo -fossero -essendo - - | forms of fare, to do (not including the infinitive, fa, fat-): -faccio -fai -facciamo -fanno -faccia -facciate -facciano -farò -farai -farà -faremo -farete -faranno -farei -faresti -farebbe -faremmo -fareste -farebbero -facevo -facevi -faceva -facevamo -facevate -facevano -feci -facesti -fece -facemmo -faceste -fecero -facessi -facesse -facessimo -facessero -facendo - - | forms of stare, to be (not including the infinitive): -sto -stai -sta -stiamo -stanno -stia -stiate -stiano -starò -starai -starà -staremo -starete -staranno -starei -staresti -starebbe -staremmo -stareste -starebbero -stavo -stavi -stava -stavamo -stavate -stavano -stetti -stesti -stette -stemmo -steste -stettero -stessi -stesse -stessimo -stessero -stando diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ja.txt deleted file mode 100644 index d4321be6b16..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ja.txt +++ /dev/null @@ -1,127 +0,0 @@ -# -# This file defines a stopword set for Japanese. -# -# This set is made up of hand-picked frequent terms from segmented Japanese Wikipedia. -# Punctuation characters and frequent kanji have mostly been left out. See LUCENE-3745 -# for frequency lists, etc. that can be useful for making your own set (if desired) -# -# Note that there is an overlap between these stopwords and the terms stopped when used -# in combination with the JapanesePartOfSpeechStopFilter. When editing this file, note -# that comments are not allowed on the same line as stopwords. -# -# Also note that stopping is done in a case-insensitive manner. Change your StopFilter -# configuration if you need case-sensitive stopping. Lastly, note that stopping is done -# using the same character width as the entries in this file. Since this StopFilter is -# normally done after a CJKWidthFilter in your chain, you would usually want your romaji -# entries to be in half-width and your kana entries to be in full-width. -# -の -に -は -を -た -が -で -て -と -し -れ -さ -ある -いる -も -する -から -な -こと -として -い -や -れる -など -なっ -ない -この -ため -その -あっ -よう -また -もの -という -あり -まで -られ -なる -へ -か -だ -これ -によって -により -おり -より -による -ず -なり -られる -において -ば -なかっ -なく -しかし -について -せ -だっ -その後 -できる -それ -う -ので -なお -のみ -でき -き -つ -における -および -いう -さらに -でも -ら -たり -その他 -に関する -たち -ます -ん -なら -に対して -特に -せる -及び -これら -とき -では -にて -ほか -ながら -うち -そして -とともに -ただし -かつて -それぞれ -または -お -ほど -ものの -に対する -ほとんど -と共に -といった -です -とも -ところ -ここ -##### End of file diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_lv.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_lv.txt deleted file mode 100644 index e21a23c06c3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_lv.txt +++ /dev/null @@ -1,172 +0,0 @@ -# Set of Latvian stopwords from A Stemming Algorithm for Latvian, Karlis Kreslins -# the original list of over 800 forms was refined: -# pronouns, adverbs, interjections were removed -# -# prepositions -aiz -ap -ar -apakš -ārpus -augšpus -bez -caur -dēļ -gar -iekš -iz -kopš -labad -lejpus -līdz -no -otrpus -pa -par -pār -pēc -pie -pirms -pret -priekš -starp -šaipus -uz -viņpus -virs -virspus -zem -apakšpus -# Conjunctions -un -bet -jo -ja -ka -lai -tomēr -tikko -turpretī -arī -kaut -gan -tādēļ -tā -ne -tikvien -vien -kā -ir -te -vai -kamēr -# Particles -ar -diezin -droši -diemžēl -nebūt -ik -it -taču -nu -pat -tiklab -iekšpus -nedz -tik -nevis -turpretim -jeb -iekam -iekām -iekāms -kolīdz -līdzko -tiklīdz -jebšu -tālab -tāpēc -nekā -itin -jā -jau -jel -nē -nezin -tad -tikai -vis -tak -iekams -vien -# modal verbs -būt -biju -biji -bija -bijām -bijāt -esmu -esi -esam -esat -būšu -būsi -būs -būsim -būsiet -tikt -tiku -tiki -tika -tikām -tikāt -tieku -tiec -tiek -tiekam -tiekat -tikšu -tiks -tiksim -tiksiet -tapt -tapi -tapāt -topat -tapšu -tapsi -taps -tapsim -tapsiet -kļūt -kļuvu -kļuvi -kļuva -kļuvām -kļuvāt -kļūstu -kļūsti -kļūst -kļūstam -kļūstat -kļūšu -kļūsi -kļūs -kļūsim -kļūsiet -# verbs -varēt -varēju -varējām -varēšu -varēsim -var -varēji -varējāt -varēsi -varēsiet -varat -varēja -varēs diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_nl.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_nl.txt deleted file mode 100644 index f4d61f5092c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_nl.txt +++ /dev/null @@ -1,117 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/dutch/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Dutch stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large sample of Dutch text. - - | Dutch stop words frequently exhibit homonym clashes. These are indicated - | clearly below. - -de | the -en | and -van | of, from -ik | I, the ego -te | (1) chez, at etc, (2) to, (3) too -dat | that, which -die | that, those, who, which -in | in, inside -een | a, an, one -hij | he -het | the, it -niet | not, nothing, naught -zijn | (1) to be, being, (2) his, one's, its -is | is -was | (1) was, past tense of all persons sing. of 'zijn' (to be) (2) wax, (3) the washing, (4) rise of river -op | on, upon, at, in, up, used up -aan | on, upon, to (as dative) -met | with, by -als | like, such as, when -voor | (1) before, in front of, (2) furrow -had | had, past tense all persons sing. of 'hebben' (have) -er | there -maar | but, only -om | round, about, for etc -hem | him -dan | then -zou | should/would, past tense all persons sing. of 'zullen' -of | or, whether, if -wat | what, something, anything -mijn | possessive and noun 'mine' -men | people, 'one' -dit | this -zo | so, thus, in this way -door | through by -over | over, across -ze | she, her, they, them -zich | oneself -bij | (1) a bee, (2) by, near, at -ook | also, too -tot | till, until -je | you -mij | me -uit | out of, from -der | Old Dutch form of 'van der' still found in surnames -daar | (1) there, (2) because -haar | (1) her, their, them, (2) hair -naar | (1) unpleasant, unwell etc, (2) towards, (3) as -heb | present first person sing. of 'to have' -hoe | how, why -heeft | present third person sing. of 'to have' -hebben | 'to have' and various parts thereof -deze | this -u | you -want | (1) for, (2) mitten, (3) rigging -nog | yet, still -zal | 'shall', first and third person sing. of verb 'zullen' (will) -me | me -zij | she, they -nu | now -ge | 'thou', still used in Belgium and south Netherlands -geen | none -omdat | because -iets | something, somewhat -worden | to become, grow, get -toch | yet, still -al | all, every, each -waren | (1) 'were' (2) to wander, (3) wares, (3) -veel | much, many -meer | (1) more, (2) lake -doen | to do, to make -toen | then, when -moet | noun 'spot/mote' and present form of 'to must' -ben | (1) am, (2) 'are' in interrogative second person singular of 'to be' -zonder | without -kan | noun 'can' and present form of 'to be able' -hun | their, them -dus | so, consequently -alles | all, everything, anything -onder | under, beneath -ja | yes, of course -eens | once, one day -hier | here -wie | who -werd | imperfect third person sing. of 'become' -altijd | always -doch | yet, but etc -wordt | present third person sing. of 'become' -wezen | (1) to be, (2) 'been' as in 'been fishing', (3) orphans -kunnen | to be able -ons | us/our -zelf | self -tegen | against, towards, at -na | after, near -reeds | already -wil | (1) present tense of 'want', (2) 'will', noun, (3) fender -kon | could; past tense of 'to be able' -niets | nothing -uw | your -iemand | somebody -geweest | been; past participle of 'be' -andere | other diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_no.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_no.txt deleted file mode 100644 index e76f36e69ed..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_no.txt +++ /dev/null @@ -1,192 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/norwegian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Norwegian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This stop word list is for the dominant bokmål dialect. Words unique - | to nynorsk are marked *. - - | Revised by Jan Bruusgaard , Jan 2005 - -og | and -i | in -jeg | I -det | it/this/that -at | to (w. inf.) -en | a/an -et | a/an -den | it/this/that -til | to -er | is/am/are -som | who/that -på | on -de | they / you(formal) -med | with -han | he -av | of -ikke | not -ikkje | not * -der | there -så | so -var | was/were -meg | me -seg | you -men | but -ett | one -har | have -om | about -vi | we -min | my -mitt | my -ha | have -hadde | had -hun | she -nå | now -over | over -da | when/as -ved | by/know -fra | from -du | you -ut | out -sin | your -dem | them -oss | us -opp | up -man | you/one -kan | can -hans | his -hvor | where -eller | or -hva | what -skal | shall/must -selv | self (reflective) -sjøl | self (reflective) -her | here -alle | all -vil | will -bli | become -ble | became -blei | became * -blitt | have become -kunne | could -inn | in -når | when -være | be -kom | come -noen | some -noe | some -ville | would -dere | you -som | who/which/that -deres | their/theirs -kun | only/just -ja | yes -etter | after -ned | down -skulle | should -denne | this -for | for/because -deg | you -si | hers/his -sine | hers/his -sitt | hers/his -mot | against -å | to -meget | much -hvorfor | why -dette | this -disse | these/those -uten | without -hvordan | how -ingen | none -din | your -ditt | your -blir | become -samme | same -hvilken | which -hvilke | which (plural) -sånn | such a -inni | inside/within -mellom | between -vår | our -hver | each -hvem | who -vors | us/ours -hvis | whose -både | both -bare | only/just -enn | than -fordi | as/because -før | before -mange | many -også | also -slik | just -vært | been -være | to be -båe | both * -begge | both -siden | since -dykk | your * -dykkar | yours * -dei | they * -deira | them * -deires | theirs * -deim | them * -di | your (fem.) * -då | as/when * -eg | I * -ein | a/an * -eit | a/an * -eitt | a/an * -elles | or * -honom | he * -hjå | at * -ho | she * -hoe | she * -henne | her -hennar | her/hers -hennes | hers -hoss | how * -hossen | how * -ikkje | not * -ingi | noone * -inkje | noone * -korleis | how * -korso | how * -kva | what/which * -kvar | where * -kvarhelst | where * -kven | who/whom * -kvi | why * -kvifor | why * -me | we * -medan | while * -mi | my * -mine | my * -mykje | much * -no | now * -nokon | some (masc./neut.) * -noka | some (fem.) * -nokor | some * -noko | some * -nokre | some * -si | his/hers * -sia | since * -sidan | since * -so | so * -somt | some * -somme | some * -um | about* -upp | up * -vere | be * -vore | was * -verte | become * -vort | become * -varte | became * -vart | became * - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_pt.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_pt.txt deleted file mode 100644 index 276c1b446f2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_pt.txt +++ /dev/null @@ -1,251 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/portuguese/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Portuguese stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | of, from -a | the; to, at; her -o | the; him -que | who, that -e | and -do | de + o -da | de + a -em | in -um | a -para | for - | é from SER -com | with -não | not, no -uma | a -os | the; them -no | em + o -se | himself etc -na | em + a -por | for -mais | more -as | the; them -dos | de + os -como | as, like -mas | but - | foi from SER -ao | a + o -ele | he -das | de + as - | tem from TER -à | a + a -seu | his -sua | her -ou | or - | ser from SER -quando | when -muito | much - | há from HAV -nos | em + os; us -já | already, now - | está from EST -eu | I -também | also -só | only, just -pelo | per + o -pela | per + a -até | up to -isso | that -ela | he -entre | between - | era from SER -depois | after -sem | without -mesmo | same -aos | a + os - | ter from TER -seus | his -quem | whom -nas | em + as -me | me -esse | that -eles | they - | estão from EST -você | you - | tinha from TER - | foram from SER -essa | that -num | em + um -nem | nor -suas | her -meu | my -às | a + as -minha | my - | têm from TER -numa | em + uma -pelos | per + os -elas | they - | havia from HAV - | seja from SER -qual | which - | será from SER -nós | we - | tenho from TER -lhe | to him, her -deles | of them -essas | those -esses | those -pelas | per + as -este | this - | fosse from SER -dele | of him - - | other words. There are many contractions such as naquele = em+aquele, - | mo = me+o, but they are rare. - | Indefinite article plural forms are also rare. - -tu | thou -te | thee -vocês | you (plural) -vos | you -lhes | to them -meus | my -minhas -teu | thy -tua -teus -tuas -nosso | our -nossa -nossos -nossas - -dela | of her -delas | of them - -esta | this -estes | these -estas | these -aquele | that -aquela | that -aqueles | those -aquelas | those -isto | this -aquilo | that - - | forms of estar, to be (not including the infinitive): -estou -está -estamos -estão -estive -esteve -estivemos -estiveram -estava -estávamos -estavam -estivera -estivéramos -esteja -estejamos -estejam -estivesse -estivéssemos -estivessem -estiver -estivermos -estiverem - - | forms of haver, to have (not including the infinitive): -hei -há -havemos -hão -houve -houvemos -houveram -houvera -houvéramos -haja -hajamos -hajam -houvesse -houvéssemos -houvessem -houver -houvermos -houverem -houverei -houverá -houveremos -houverão -houveria -houveríamos -houveriam - - | forms of ser, to be (not including the infinitive): -sou -somos -são -era -éramos -eram -fui -foi -fomos -foram -fora -fôramos -seja -sejamos -sejam -fosse -fôssemos -fossem -for -formos -forem -serei -será -seremos -serão -seria -seríamos -seriam - - | forms of ter, to have (not including the infinitive): -tenho -tem -temos -tém -tinha -tínhamos -tinham -tive -teve -tivemos -tiveram -tivera -tivéramos -tenha -tenhamos -tenham -tivesse -tivéssemos -tivessem -tiver -tivermos -tiverem -terei -terá -teremos -terão -teria -teríamos -teriam diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ro.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ro.txt deleted file mode 100644 index 4fdee90a5ba..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ro.txt +++ /dev/null @@ -1,233 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -acea -aceasta -această -aceea -acei -aceia -acel -acela -acele -acelea -acest -acesta -aceste -acestea -aceşti -aceştia -acolo -acum -ai -aia -aibă -aici -al -ăla -ale -alea -ălea -altceva -altcineva -am -ar -are -aş -aşadar -asemenea -asta -ăsta -astăzi -astea -ăstea -ăştia -asupra -aţi -au -avea -avem -aveţi -azi -bine -bucur -bună -ca -că -căci -când -care -cărei -căror -cărui -cât -câte -câţi -către -câtva -ce -cel -ceva -chiar -cînd -cine -cineva -cît -cîte -cîţi -cîtva -contra -cu -cum -cumva -curând -curînd -da -dă -dacă -dar -datorită -de -deci -deja -deoarece -departe -deşi -din -dinaintea -dintr -dintre -drept -după -ea -ei -el -ele -eram -este -eşti -eu -face -fără -fi -fie -fiecare -fii -fim -fiţi -iar -ieri -îi -îl -îmi -împotriva -în -înainte -înaintea -încât -încît -încotro -între -întrucât -întrucît -îţi -la -lângă -le -li -lîngă -lor -lui -mă -mâine -mea -mei -mele -mereu -meu -mi -mine -mult -multă -mulţi -ne -nicăieri -nici -nimeni -nişte -noastră -noastre -noi -noştri -nostru -nu -ori -oricând -oricare -oricât -orice -oricînd -oricine -oricît -oricum -oriunde -până -pe -pentru -peste -pînă -poate -pot -prea -prima -primul -prin -printr -sa -să -săi -sale -sau -său -se -şi -sînt -sîntem -sînteţi -spre -sub -sunt -suntem -sunteţi -ta -tăi -tale -tău -te -ţi -ţie -tine -toată -toate -tot -toţi -totuşi -tu -un -una -unde -undeva -unei -unele -uneori -unor -vă -vi -voastră -voastre -voi -voştri -vostru -vouă -vreo -vreun diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ru.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ru.txt deleted file mode 100644 index 64307693457..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_ru.txt +++ /dev/null @@ -1,241 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/russian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | a russian stop word list. comments begin with vertical bar. each stop - | word is at the start of a line. - - | this is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | letter `ё' is translated to `е'. - -и | and -в | in/into -во | alternative form -не | not -что | what/that -он | he -на | on/onto -я | i -с | from -со | alternative form -как | how -а | milder form of `no' (but) -то | conjunction and form of `that' -все | all -она | she -так | so, thus -его | him -но | but -да | yes/and -ты | thou -к | towards, by -у | around, chez -же | intensifier particle -вы | you -за | beyond, behind -бы | conditional/subj. particle -по | up to, along -только | only -ее | her -мне | to me -было | it was -вот | here is/are, particle -от | away from -меня | me -еще | still, yet, more -нет | no, there isnt/arent -о | about -из | out of -ему | to him -теперь | now -когда | when -даже | even -ну | so, well -вдруг | suddenly -ли | interrogative particle -если | if -уже | already, but homonym of `narrower' -или | or -ни | neither -быть | to be -был | he was -него | prepositional form of его -до | up to -вас | you accusative -нибудь | indef. suffix preceded by hyphen -опять | again -уж | already, but homonym of `adder' -вам | to you -сказал | he said -ведь | particle `after all' -там | there -потом | then -себя | oneself -ничего | nothing -ей | to her -может | usually with `быть' as `maybe' -они | they -тут | here -где | where -есть | there is/are -надо | got to, must -ней | prepositional form of ей -для | for -мы | we -тебя | thee -их | them, their -чем | than -была | she was -сам | self -чтоб | in order to -без | without -будто | as if -человек | man, person, one -чего | genitive form of `what' -раз | once -тоже | also -себе | to oneself -под | beneath -жизнь | life -будет | will be -ж | short form of intensifer particle `же' -тогда | then -кто | who -этот | this -говорил | was saying -того | genitive form of `that' -потому | for that reason -этого | genitive form of `this' -какой | which -совсем | altogether -ним | prepositional form of `его', `они' -здесь | here -этом | prepositional form of `этот' -один | one -почти | almost -мой | my -тем | instrumental/dative plural of `тот', `то' -чтобы | full form of `in order that' -нее | her (acc.) -кажется | it seems -сейчас | now -были | they were -куда | where to -зачем | why -сказать | to say -всех | all (acc., gen. preposn. plural) -никогда | never -сегодня | today -можно | possible, one can -при | by -наконец | finally -два | two -об | alternative form of `о', about -другой | another -хоть | even -после | after -над | above -больше | more -тот | that one (masc.) -через | across, in -эти | these -нас | us -про | about -всего | in all, only, of all -них | prepositional form of `они' (they) -какая | which, feminine -много | lots -разве | interrogative particle -сказала | she said -три | three -эту | this, acc. fem. sing. -моя | my, feminine -впрочем | moreover, besides -хорошо | good -свою | ones own, acc. fem. sing. -этой | oblique form of `эта', fem. `this' -перед | in front of -иногда | sometimes -лучше | better -чуть | a little -том | preposn. form of `that one' -нельзя | one must not -такой | such a one -им | to them -более | more -всегда | always -конечно | of course -всю | acc. fem. sing of `all' -между | between - - - | b: some paradigms - | - | personal pronouns - | - | я меня мне мной [мною] - | ты тебя тебе тобой [тобою] - | он его ему им [него, нему, ним] - | она ее эи ею [нее, нэи, нею] - | оно его ему им [него, нему, ним] - | - | мы нас нам нами - | вы вас вам вами - | они их им ими [них, ним, ними] - | - | себя себе собой [собою] - | - | demonstrative pronouns: этот (this), тот (that) - | - | этот эта это эти - | этого эты это эти - | этого этой этого этих - | этому этой этому этим - | этим этой этим [этою] этими - | этом этой этом этих - | - | тот та то те - | того ту то те - | того той того тех - | тому той тому тем - | тем той тем [тою] теми - | том той том тех - | - | determinative pronouns - | - | (a) весь (all) - | - | весь вся все все - | всего всю все все - | всего всей всего всех - | всему всей всему всем - | всем всей всем [всею] всеми - | всем всей всем всех - | - | (b) сам (himself etc) - | - | сам сама само сами - | самого саму само самих - | самого самой самого самих - | самому самой самому самим - | самим самой самим [самою] самими - | самом самой самом самих - | - | stems of verbs `to be', `to have', `to do' and modal - | - | быть бы буд быв есть суть - | име - | дел - | мог мож мочь - | уме - | хоч хот - | долж - | можн - | нужн - | нельзя - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_sv.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_sv.txt deleted file mode 100644 index 22bddfd8cb3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_sv.txt +++ /dev/null @@ -1,131 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/swedish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Swedish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | Swedish stop words occasionally exhibit homonym clashes. For example - | så = so, but also seed. These are indicated clearly below. - -och | and -det | it, this/that -att | to (with infinitive) -i | in, at -en | a -jag | I -hon | she -som | who, that -han | he -på | on -den | it, this/that -med | with -var | where, each -sig | him(self) etc -för | for -så | so (also: seed) -till | to -är | is -men | but -ett | a -om | if; around, about -hade | had -de | they, these/those -av | of -icke | not, no -mig | me -du | you -henne | her -då | then, when -sin | his -nu | now -har | have -inte | inte någon = no one -hans | his -honom | him -skulle | 'sake' -hennes | her -där | there -min | my -man | one (pronoun) -ej | nor -vid | at, by, on (also: vast) -kunde | could -något | some etc -från | from, off -ut | out -när | when -efter | after, behind -upp | up -vi | we -dem | them -vara | be -vad | what -över | over -än | than -dig | you -kan | can -sina | his -här | here -ha | have -mot | towards -alla | all -under | under (also: wonder) -någon | some etc -eller | or (else) -allt | all -mycket | much -sedan | since -ju | why -denna | this/that -själv | myself, yourself etc -detta | this/that -åt | to -utan | without -varit | was -hur | how -ingen | no -mitt | my -ni | you -bli | to be, become -blev | from bli -oss | us -din | thy -dessa | these/those -några | some etc -deras | their -blir | from bli -mina | my -samma | (the) same -vilken | who, that -er | you, your -sådan | such a -vår | our -blivit | from bli -dess | its -inom | within -mellan | between -sådant | such a -varför | why -varje | each -vilka | who, that -ditt | thy -vem | who -vilket | who, that -sitta | his -sådana | such a -vart | each -dina | thy -vars | whose -vårt | our -våra | our -ert | your -era | your -vilkas | whose - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_th.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_th.txt deleted file mode 100644 index 07f0fabe692..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_th.txt +++ /dev/null @@ -1,119 +0,0 @@ -# Thai stopwords from: -# "Opinion Detection in Thai Political News Columns -# Based on Subjectivity Analysis" -# Khampol Sukhum, Supot Nitsuwat, and Choochart Haruechaiyasak -ไว้ -ไม่ -ไป -ได้ -ให้ -ใน -โดย -แห่ง -แล้ว -และ -แรก -แบบ -แต่ -เอง -เห็น -เลย -เริ่ม -เรา -เมื่อ -เพื่อ -เพราะ -เป็นการ -เป็น -เปิดเผย -เปิด -เนื่องจาก -เดียวกัน -เดียว -เช่น -เฉพาะ -เคย -เข้า -เขา -อีก -อาจ -อะไร -ออก -อย่าง -อยู่ -อยาก -หาก -หลาย -หลังจาก -หลัง -หรือ -หนึ่ง -ส่วน -ส่ง -สุด -สําหรับ -ว่า -วัน -ลง -ร่วม -ราย -รับ -ระหว่าง -รวม -ยัง -มี -มาก -มา -พร้อม -พบ -ผ่าน -ผล -บาง -น่า -นี้ -นํา -นั้น -นัก -นอกจาก -ทุก -ที่สุด -ที่ -ทําให้ -ทํา -ทาง -ทั้งนี้ -ทั้ง -ถ้า -ถูก -ถึง -ต้อง -ต่างๆ -ต่าง -ต่อ -ตาม -ตั้งแต่ -ตั้ง -ด้าน -ด้วย -ดัง -ซึ่ง -ช่วง -จึง -จาก -จัด -จะ -คือ -ความ -ครั้ง -คง -ขึ้น -ของ -ขอ -ขณะ -ก่อน -ก็ -การ -กับ -กัน -กว่า -กล่าว diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_tr.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_tr.txt deleted file mode 100644 index 84d9408d4ea..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/stopwords_tr.txt +++ /dev/null @@ -1,212 +0,0 @@ -# Turkish stopwords from LUCENE-559 -# merged with the list from "Information Retrieval on Turkish Texts" -# (http://www.users.muohio.edu/canf/papers/JASIST2008offPrint.pdf) -acaba -altmış -altı -ama -ancak -arada -aslında -ayrıca -bana -bazı -belki -ben -benden -beni -benim -beri -beş -bile -bin -bir -birçok -biri -birkaç -birkez -birşey -birşeyi -biz -bize -bizden -bizi -bizim -böyle -böylece -bu -buna -bunda -bundan -bunlar -bunları -bunların -bunu -bunun -burada -çok -çünkü -da -daha -dahi -de -defa -değil -diğer -diye -doksan -dokuz -dolayı -dolayısıyla -dört -edecek -eden -ederek -edilecek -ediliyor -edilmesi -ediyor -eğer -elli -en -etmesi -etti -ettiği -ettiğini -gibi -göre -halen -hangi -hatta -hem -henüz -hep -hepsi -her -herhangi -herkesin -hiç -hiçbir -için -iki -ile -ilgili -ise -işte -itibaren -itibariyle -kadar -karşın -katrilyon -kendi -kendilerine -kendini -kendisi -kendisine -kendisini -kez -ki -kim -kimden -kime -kimi -kimse -kırk -milyar -milyon -mu -mü -mı -nasıl -ne -neden -nedenle -nerde -nerede -nereye -niye -niçin -o -olan -olarak -oldu -olduğu -olduğunu -olduklarını -olmadı -olmadığı -olmak -olması -olmayan -olmaz -olsa -olsun -olup -olur -olursa -oluyor -on -ona -ondan -onlar -onlardan -onları -onların -onu -onun -otuz -oysa -öyle -pek -rağmen -sadece -sanki -sekiz -seksen -sen -senden -seni -senin -siz -sizden -sizi -sizin -şey -şeyden -şeyi -şeyler -şöyle -şu -şuna -şunda -şundan -şunları -şunu -tarafından -trilyon -tüm -üç -üzere -var -vardı -ve -veya -ya -yani -yapacak -yapılan -yapılması -yapıyor -yapmak -yaptı -yaptığı -yaptığını -yaptıkları -yedi -yerine -yetmiş -yine -yirmi -yoksa -yüz -zaten diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/userdict_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/userdict_ja.txt deleted file mode 100644 index 6f0368e4d81..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/lang/userdict_ja.txt +++ /dev/null @@ -1,29 +0,0 @@ -# -# This is a sample user dictionary for Kuromoji (JapaneseTokenizer) -# -# Add entries to this file in order to override the statistical model in terms -# of segmentation, readings and part-of-speech tags. Notice that entries do -# not have weights since they are always used when found. This is by-design -# in order to maximize ease-of-use. -# -# Entries are defined using the following CSV format: -# , ... , ... , -# -# Notice that a single half-width space separates tokens and readings, and -# that the number tokens and readings must match exactly. -# -# Also notice that multiple entries with the same is undefined. -# -# Whitespace only lines are ignored. Comments are not allowed on entry lines. -# - -# Custom segmentation for kanji compounds -日本経済新聞,日本 経済 新聞,ニホン ケイザイ シンブン,カスタム名詞 -関西国際空港,関西 国際 空港,カンサイ コクサイ クウコウ,カスタム名詞 - -# Custom segmentation for compound katakana -トートバッグ,トート バッグ,トート バッグ,かずカナ名詞 -ショルダーバッグ,ショルダー バッグ,ショルダー バッグ,かずカナ名詞 - -# Custom reading for former sumo wrestler -朝青龍,朝青龍,アサショウリュウ,カスタム人名 diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/protwords.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/protwords.txt deleted file mode 100644 index 1dfc0abecbf..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/protwords.txt +++ /dev/null @@ -1,21 +0,0 @@ -# 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. - -#----------------------------------------------------------------------- -# Use a protected word file to protect against the stemmer reducing two -# unrelated words to the same base word. - -# Some non-words that normally won't be encountered, -# just to test that they won't be stemmed. -dontstems -zwhacky - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/schema.xml b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/schema.xml deleted file mode 100644 index bb2f803ba8d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/schema.xml +++ /dev/null @@ -1,940 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - id - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/solrconfig.xml b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/solrconfig.xml deleted file mode 100644 index 72e2add8547..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/solrconfig.xml +++ /dev/null @@ -1,1449 +0,0 @@ - - - - - - - - - ${tests.luceneMatchVersion:LATEST} - - - - - - - - - - - - - - - - - - - - - - - - ${solr.data.dir:} - - - - - ${solr.hdfs.home:} - ${solr.hdfs.confdir:} - ${solr.hdfs.security.kerberos.enabled:false} - ${solr.hdfs.security.kerberos.keytabfile:} - ${solr.hdfs.security.kerberos.principal:} - ${solr.hdfs.blockcache.enabled:true} - ${solr.hdfs.blockcache.slab.count:1} - ${solr.hdfs.blockcache.direct.memory.allocation:true} - ${solr.hdfs.blockcache.blocksperbank:16384} - ${solr.hdfs.blockcache.read.enabled:true} - ${solr.hdfs.nrtcachingdirectory.enable:true} - ${solr.hdfs.nrtcachingdirectory.maxmergesizemb:16} - ${solr.hdfs.nrtcachingdirectory.maxcachedmb:192} - - - - - - - - - - - - - - - - - - 128 - - - - - - - - - - ${solr.lock.type:hdfs} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ${solr.autoCommit.maxTime:60000} - false - - - - - ${solr.autoSoftCommit.maxTime:1000} - - - - - - - - - - - - - - - - - - - 1024 - - - - - - - - - - - - - - - - - - - - - - true - - - - - - 20 - - - 200 - - - - - - - - - - - - static firstSearcher warming in solrconfig.xml - - - - - - false - - - 4 - - - - - - - - - - - - - - - - - - - - - - - explicit - 10 - text - - - - - - - - - - - - - - explicit - json - true - text - - - - - - - - - text_general - - - - - - default - text - solr.DirectSolrSpellChecker - - internal - - 0.5 - - 2 - - 1 - - 5 - - 4 - - 0.01 - - - - - - wordbreak - solr.WordBreakSolrSpellChecker - name - true - true - 10 - - - - - - - - - - - - - - - - text - - default - wordbreak - on - true - 10 - 5 - 5 - true - true - 10 - 5 - - - spellcheck - - - - - - - - - - text - true - - - tvComponent - - - - - - - - - default - - - org.carrot2.clustering.lingo.LingoClusteringAlgorithm - - - 20 - - - clustering/carrot2 - - - ENGLISH - - - stc - org.carrot2.clustering.stc.STCClusteringAlgorithm - - - - - - - true - default - true - - name - id - - features - - true - - - - false - - edismax - - text^0.5 features^1.0 name^1.2 sku^1.5 id^10.0 manu^1.1 cat^1.4 - - *:* - 10 - *,score - - - clustering - - - - - - - - - - true - false - - - terms - - - - - - - - string - elevate.xml - - - - - - explicit - text - - - elevator - - - - - - - - - - - 100 - - - - - - - - 70 - - 0.5 - - [-\w ,/\n\"']{20,200} - - - - - - - ]]> - ]]> - - - - - - - - - - - - - - - - - - - - - - - - ,, - ,, - ,, - ,, - ,]]> - ]]> - - - - - - 10 - .,!? - - - - - - - WORD - - - en - US - - - - - - - - - - - - - - - - - - - - - - text/plain; charset=UTF-8 - - - - - - - - - 5 - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/stopwords.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/stopwords.txt deleted file mode 100644 index ae1e83eeb3d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/stopwords.txt +++ /dev/null @@ -1,14 +0,0 @@ -# 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. diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/synonyms.txt b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/synonyms.txt deleted file mode 100644 index 7f72128303b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/conf/synonyms.txt +++ /dev/null @@ -1,29 +0,0 @@ -# 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. - -#----------------------------------------------------------------------- -#some test synonym mappings unlikely to appear in real input text -aaafoo => aaabar -bbbfoo => bbbfoo bbbbar -cccfoo => cccbar cccbaz -fooaaa,baraaa,bazaaa - -# Some synonym groups specific to this example -GB,gib,gigabyte,gigabytes -MB,mib,megabyte,megabytes -Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming -#after us won't split it into two words. - -# Synonym mappings can be used for spelling correction too -pixima => pixma - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/solr.xml b/solr/contrib/morphlines-core/src/test-files/solr/mrunit/solr.xml deleted file mode 100644 index 8c1509aabc4..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/mrunit/solr.xml +++ /dev/null @@ -1,45 +0,0 @@ - - - - - - - ${shareSchema:false} - - - ${hostPort:8983} - ${hostPort:8983} - ${solr.zkclienttimeout:30000} - ${genericCoreNodeNames:true} - 0 - ${distribUpdateConnTimeout:15000} - ${distribUpdateSoTimeout:120000} - - - - ${urlScheme:} - ${socketTimeout:120000} - ${connTimeout:15000} - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solr.xml b/solr/contrib/morphlines-core/src/test-files/solr/solr.xml deleted file mode 100644 index e67eb1d4054..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solr.xml +++ /dev/null @@ -1,34 +0,0 @@ - - - - - - - ${hostPort:8983} - ${hostContext:solr} - ${solr.zkclienttimeout:30000} - 127.0.0.1 - - - - ${urlScheme:} - ${socketTimeout:120000} - ${connTimeout:15000} - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/currency.xml b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/currency.xml deleted file mode 100644 index 3a9c58afee8..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/currency.xml +++ /dev/null @@ -1,67 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/elevate.xml b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/elevate.xml deleted file mode 100644 index 2c09ebed669..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/elevate.xml +++ /dev/null @@ -1,42 +0,0 @@ - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_ca.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_ca.txt deleted file mode 100644 index 307a85f913d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_ca.txt +++ /dev/null @@ -1,8 +0,0 @@ -# Set of Catalan contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -l -m -n -s -t diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_fr.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_fr.txt deleted file mode 100644 index 722db588333..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_fr.txt +++ /dev/null @@ -1,9 +0,0 @@ -# Set of French contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -l -m -t -qu -n -s -j diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_ga.txt deleted file mode 100644 index 9ebe7fa349a..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -m -b diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_it.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_it.txt deleted file mode 100644 index cac04095372..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/contractions_it.txt +++ /dev/null @@ -1,23 +0,0 @@ -# Set of Italian contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -c -l -all -dall -dell -nell -sull -coll -pell -gl -agl -dagl -degl -negl -sugl -un -m -t -s -v -d diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/hyphenations_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/hyphenations_ga.txt deleted file mode 100644 index 4d2642cc5a3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/hyphenations_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish hyphenations for StopFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -h -n -t diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stemdict_nl.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stemdict_nl.txt deleted file mode 100644 index 441072971d3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stemdict_nl.txt +++ /dev/null @@ -1,6 +0,0 @@ -# Set of overrides for the dutch stemmer -# TODO: load this as a resource from the analyzer and sync it in build.xml -fiets fiets -bromfiets bromfiets -ei eier -kind kinder diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stoptags_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stoptags_ja.txt deleted file mode 100644 index 71b750845e3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stoptags_ja.txt +++ /dev/null @@ -1,420 +0,0 @@ -# -# This file defines a Japanese stoptag set for JapanesePartOfSpeechStopFilter. -# -# Any token with a part-of-speech tag that exactly matches those defined in this -# file are removed from the token stream. -# -# Set your own stoptags by uncommenting the lines below. Note that comments are -# not allowed on the same line as a stoptag. See LUCENE-3745 for frequency lists, -# etc. that can be useful for building you own stoptag set. -# -# The entire possible tagset is provided below for convenience. -# -##### -# noun: unclassified nouns -#名詞 -# -# noun-common: Common nouns or nouns where the sub-classification is undefined -#名詞-一般 -# -# noun-proper: Proper nouns where the sub-classification is undefined -#名詞-固有名詞 -# -# noun-proper-misc: miscellaneous proper nouns -#名詞-固有名詞-一般 -# -# noun-proper-person: Personal names where the sub-classification is undefined -#名詞-固有名詞-人名 -# -# noun-proper-person-misc: names that cannot be divided into surname and -# given name; foreign names; names where the surname or given name is unknown. -# e.g. お市の方 -#名詞-固有名詞-人名-一般 -# -# noun-proper-person-surname: Mainly Japanese surnames. -# e.g. 山田 -#名詞-固有名詞-人名-姓 -# -# noun-proper-person-given_name: Mainly Japanese given names. -# e.g. 太郎 -#名詞-固有名詞-人名-名 -# -# noun-proper-organization: Names representing organizations. -# e.g. 通産省, NHK -#名詞-固有名詞-組織 -# -# noun-proper-place: Place names where the sub-classification is undefined -#名詞-固有名詞-地域 -# -# noun-proper-place-misc: Place names excluding countries. -# e.g. アジア, バルセロナ, 京都 -#名詞-固有名詞-地域-一般 -# -# noun-proper-place-country: Country names. -# e.g. 日本, オーストラリア -#名詞-固有名詞-地域-国 -# -# noun-pronoun: Pronouns where the sub-classification is undefined -#名詞-代名詞 -# -# noun-pronoun-misc: miscellaneous pronouns: -# e.g. それ, ここ, あいつ, あなた, あちこち, いくつ, どこか, なに, みなさん, みんな, わたくし, われわれ -#名詞-代名詞-一般 -# -# noun-pronoun-contraction: Spoken language contraction made by combining a -# pronoun and the particle 'wa'. -# e.g. ありゃ, こりゃ, こりゃあ, そりゃ, そりゃあ -#名詞-代名詞-縮約 -# -# noun-adverbial: Temporal nouns such as names of days or months that behave -# like adverbs. Nouns that represent amount or ratios and can be used adverbially, -# e.g. 金曜, 一月, 午後, 少量 -#名詞-副詞可能 -# -# noun-verbal: Nouns that take arguments with case and can appear followed by -# 'suru' and related verbs (する, できる, なさる, くださる) -# e.g. インプット, 愛着, 悪化, 悪戦苦闘, 一安心, 下取り -#名詞-サ変接続 -# -# noun-adjective-base: The base form of adjectives, words that appear before な ("na") -# e.g. 健康, 安易, 駄目, だめ -#名詞-形容動詞語幹 -# -# noun-numeric: Arabic numbers, Chinese numerals, and counters like 何 (回), 数. -# e.g. 0, 1, 2, 何, 数, 幾 -#名詞-数 -# -# noun-affix: noun affixes where the sub-classification is undefined -#名詞-非自立 -# -# noun-affix-misc: Of adnominalizers, the case-marker の ("no"), and words that -# attach to the base form of inflectional words, words that cannot be classified -# into any of the other categories below. This category includes indefinite nouns. -# e.g. あかつき, 暁, かい, 甲斐, 気, きらい, 嫌い, くせ, 癖, こと, 事, ごと, 毎, しだい, 次第, -# 順, せい, 所為, ついで, 序で, つもり, 積もり, 点, どころ, の, はず, 筈, はずみ, 弾み, -# 拍子, ふう, ふり, 振り, ほう, 方, 旨, もの, 物, 者, ゆえ, 故, ゆえん, 所以, わけ, 訳, -# わり, 割り, 割, ん-口語/, もん-口語/ -#名詞-非自立-一般 -# -# noun-affix-adverbial: noun affixes that that can behave as adverbs. -# e.g. あいだ, 間, あげく, 挙げ句, あと, 後, 余り, 以外, 以降, 以後, 以上, 以前, 一方, うえ, -# 上, うち, 内, おり, 折り, かぎり, 限り, きり, っきり, 結果, ころ, 頃, さい, 際, 最中, さなか, -# 最中, じたい, 自体, たび, 度, ため, 為, つど, 都度, とおり, 通り, とき, 時, ところ, 所, -# とたん, 途端, なか, 中, のち, 後, ばあい, 場合, 日, ぶん, 分, ほか, 他, まえ, 前, まま, -# 儘, 侭, みぎり, 矢先 -#名詞-非自立-副詞可能 -# -# noun-affix-aux: noun affixes treated as 助動詞 ("auxiliary verb") in school grammars -# with the stem よう(だ) ("you(da)"). -# e.g. よう, やう, 様 (よう) -#名詞-非自立-助動詞語幹 -# -# noun-affix-adjective-base: noun affixes that can connect to the indeclinable -# connection form な (aux "da"). -# e.g. みたい, ふう -#名詞-非自立-形容動詞語幹 -# -# noun-special: special nouns where the sub-classification is undefined. -#名詞-特殊 -# -# noun-special-aux: The そうだ ("souda") stem form that is used for reporting news, is -# treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the base -# form of inflectional words. -# e.g. そう -#名詞-特殊-助動詞語幹 -# -# noun-suffix: noun suffixes where the sub-classification is undefined. -#名詞-接尾 -# -# noun-suffix-misc: Of the nouns or stem forms of other parts of speech that connect -# to ガル or タイ and can combine into compound nouns, words that cannot be classified into -# any of the other categories below. In general, this category is more inclusive than -# 接尾語 ("suffix") and is usually the last element in a compound noun. -# e.g. おき, かた, 方, 甲斐 (がい), がかり, ぎみ, 気味, ぐるみ, (~した) さ, 次第, 済 (ず) み, -# よう, (でき)っこ, 感, 観, 性, 学, 類, 面, 用 -#名詞-接尾-一般 -# -# noun-suffix-person: Suffixes that form nouns and attach to person names more often -# than other nouns. -# e.g. 君, 様, 著 -#名詞-接尾-人名 -# -# noun-suffix-place: Suffixes that form nouns and attach to place names more often -# than other nouns. -# e.g. 町, 市, 県 -#名詞-接尾-地域 -# -# noun-suffix-verbal: Of the suffixes that attach to nouns and form nouns, those that -# can appear before スル ("suru"). -# e.g. 化, 視, 分け, 入り, 落ち, 買い -#名詞-接尾-サ変接続 -# -# noun-suffix-aux: The stem form of そうだ (様態) that is used to indicate conditions, -# is treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the -# conjunctive form of inflectional words. -# e.g. そう -#名詞-接尾-助動詞語幹 -# -# noun-suffix-adjective-base: Suffixes that attach to other nouns or the conjunctive -# form of inflectional words and appear before the copula だ ("da"). -# e.g. 的, げ, がち -#名詞-接尾-形容動詞語幹 -# -# noun-suffix-adverbial: Suffixes that attach to other nouns and can behave as adverbs. -# e.g. 後 (ご), 以後, 以降, 以前, 前後, 中, 末, 上, 時 (じ) -#名詞-接尾-副詞可能 -# -# noun-suffix-classifier: Suffixes that attach to numbers and form nouns. This category -# is more inclusive than 助数詞 ("classifier") and includes common nouns that attach -# to numbers. -# e.g. 個, つ, 本, 冊, パーセント, cm, kg, カ月, か国, 区画, 時間, 時半 -#名詞-接尾-助数詞 -# -# noun-suffix-special: Special suffixes that mainly attach to inflecting words. -# e.g. (楽し) さ, (考え) 方 -#名詞-接尾-特殊 -# -# noun-suffix-conjunctive: Nouns that behave like conjunctions and join two words -# together. -# e.g. (日本) 対 (アメリカ), 対 (アメリカ), (3) 対 (5), (女優) 兼 (主婦) -#名詞-接続詞的 -# -# noun-verbal_aux: Nouns that attach to the conjunctive particle て ("te") and are -# semantically verb-like. -# e.g. ごらん, ご覧, 御覧, 頂戴 -#名詞-動詞非自立的 -# -# noun-quotation: text that cannot be segmented into words, proverbs, Chinese poetry, -# dialects, English, etc. Currently, the only entry for 名詞 引用文字列 ("noun quotation") -# is いわく ("iwaku"). -#名詞-引用文字列 -# -# noun-nai_adjective: Words that appear before the auxiliary verb ない ("nai") and -# behave like an adjective. -# e.g. 申し訳, 仕方, とんでも, 違い -#名詞-ナイ形容詞語幹 -# -##### -# prefix: unclassified prefixes -#接頭詞 -# -# prefix-nominal: Prefixes that attach to nouns (including adjective stem forms) -# excluding numerical expressions. -# e.g. お (水), 某 (氏), 同 (社), 故 (~氏), 高 (品質), お (見事), ご (立派) -#接頭詞-名詞接続 -# -# prefix-verbal: Prefixes that attach to the imperative form of a verb or a verb -# in conjunctive form followed by なる/なさる/くださる. -# e.g. お (読みなさい), お (座り) -#接頭詞-動詞接続 -# -# prefix-adjectival: Prefixes that attach to adjectives. -# e.g. お (寒いですねえ), バカ (でかい) -#接頭詞-形容詞接続 -# -# prefix-numerical: Prefixes that attach to numerical expressions. -# e.g. 約, およそ, 毎時 -#接頭詞-数接続 -# -##### -# verb: unclassified verbs -#動詞 -# -# verb-main: -#動詞-自立 -# -# verb-auxiliary: -#動詞-非自立 -# -# verb-suffix: -#動詞-接尾 -# -##### -# adjective: unclassified adjectives -#形容詞 -# -# adjective-main: -#形容詞-自立 -# -# adjective-auxiliary: -#形容詞-非自立 -# -# adjective-suffix: -#形容詞-接尾 -# -##### -# adverb: unclassified adverbs -#副詞 -# -# adverb-misc: Words that can be segmented into one unit and where adnominal -# modification is not possible. -# e.g. あいかわらず, 多分 -#副詞-一般 -# -# adverb-particle_conjunction: Adverbs that can be followed by の, は, に, -# な, する, だ, etc. -# e.g. こんなに, そんなに, あんなに, なにか, なんでも -#副詞-助詞類接続 -# -##### -# adnominal: Words that only have noun-modifying forms. -# e.g. この, その, あの, どの, いわゆる, なんらかの, 何らかの, いろんな, こういう, そういう, ああいう, -# どういう, こんな, そんな, あんな, どんな, 大きな, 小さな, おかしな, ほんの, たいした, -# 「(, も) さる (ことながら)」, 微々たる, 堂々たる, 単なる, いかなる, 我が」「同じ, 亡き -#連体詞 -# -##### -# conjunction: Conjunctions that can occur independently. -# e.g. が, けれども, そして, じゃあ, それどころか -接続詞 -# -##### -# particle: unclassified particles. -助詞 -# -# particle-case: case particles where the subclassification is undefined. -助詞-格助詞 -# -# particle-case-misc: Case particles. -# e.g. から, が, で, と, に, へ, より, を, の, にて -助詞-格助詞-一般 -# -# particle-case-quote: the "to" that appears after nouns, a person’s speech, -# quotation marks, expressions of decisions from a meeting, reasons, judgements, -# conjectures, etc. -# e.g. ( だ) と (述べた.), ( である) と (して執行猶予...) -助詞-格助詞-引用 -# -# particle-case-compound: Compounds of particles and verbs that mainly behave -# like case particles. -# e.g. という, といった, とかいう, として, とともに, と共に, でもって, にあたって, に当たって, に当って, -# にあたり, に当たり, に当り, に当たる, にあたる, において, に於いて,に於て, における, に於ける, -# にかけ, にかけて, にかんし, に関し, にかんして, に関して, にかんする, に関する, に際し, -# に際して, にしたがい, に従い, に従う, にしたがって, に従って, にたいし, に対し, にたいして, -# に対して, にたいする, に対する, について, につき, につけ, につけて, につれ, につれて, にとって, -# にとり, にまつわる, によって, に依って, に因って, により, に依り, に因り, による, に依る, に因る, -# にわたって, にわたる, をもって, を以って, を通じ, を通じて, を通して, をめぐって, をめぐり, をめぐる, -# って-口語/, ちゅう-関西弁「という」/, (何) ていう (人)-口語/, っていう-口語/, といふ, とかいふ -助詞-格助詞-連語 -# -# particle-conjunctive: -# e.g. から, からには, が, けれど, けれども, けど, し, つつ, て, で, と, ところが, どころか, とも, ども, -# ながら, なり, ので, のに, ば, ものの, や ( した), やいなや, (ころん) じゃ(いけない)-口語/, -# (行っ) ちゃ(いけない)-口語/, (言っ) たって (しかたがない)-口語/, (それがなく)ったって (平気)-口語/ -助詞-接続助詞 -# -# particle-dependency: -# e.g. こそ, さえ, しか, すら, は, も, ぞ -助詞-係助詞 -# -# particle-adverbial: -# e.g. がてら, かも, くらい, 位, ぐらい, しも, (学校) じゃ(これが流行っている)-口語/, -# (それ)じゃあ (よくない)-口語/, ずつ, (私) なぞ, など, (私) なり (に), (先生) なんか (大嫌い)-口語/, -# (私) なんぞ, (先生) なんて (大嫌い)-口語/, のみ, だけ, (私) だって-口語/, だに, -# (彼)ったら-口語/, (お茶) でも (いかが), 等 (とう), (今後) とも, ばかり, ばっか-口語/, ばっかり-口語/, -# ほど, 程, まで, 迄, (誰) も (が)([助詞-格助詞] および [助詞-係助詞] の前に位置する「も」) -助詞-副助詞 -# -# particle-interjective: particles with interjective grammatical roles. -# e.g. (松島) や -助詞-間投助詞 -# -# particle-coordinate: -# e.g. と, たり, だの, だり, とか, なり, や, やら -助詞-並立助詞 -# -# particle-final: -# e.g. かい, かしら, さ, ぜ, (だ)っけ-口語/, (とまってる) で-方言/, な, ナ, なあ-口語/, ぞ, ね, ネ, -# ねぇ-口語/, ねえ-口語/, ねん-方言/, の, のう-口語/, や, よ, ヨ, よぉ-口語/, わ, わい-口語/ -助詞-終助詞 -# -# particle-adverbial/conjunctive/final: The particle "ka" when unknown whether it is -# adverbial, conjunctive, or sentence final. For example: -# (a) 「A か B か」. Ex:「(国内で運用する) か,(海外で運用する) か (.)」 -# (b) Inside an adverb phrase. Ex:「(幸いという) か (, 死者はいなかった.)」 -# 「(祈りが届いたせい) か (, 試験に合格した.)」 -# (c) 「かのように」. Ex:「(何もなかった) か (のように振る舞った.)」 -# e.g. か -助詞-副助詞/並立助詞/終助詞 -# -# particle-adnominalizer: The "no" that attaches to nouns and modifies -# non-inflectional words. -助詞-連体化 -# -# particle-adnominalizer: The "ni" and "to" that appear following nouns and adverbs -# that are giongo, giseigo, or gitaigo. -# e.g. に, と -助詞-副詞化 -# -# particle-special: A particle that does not fit into one of the above classifications. -# This includes particles that are used in Tanka, Haiku, and other poetry. -# e.g. かな, けむ, ( しただろう) に, (あんた) にゃ(わからん), (俺) ん (家) -助詞-特殊 -# -##### -# auxiliary-verb: -助動詞 -# -##### -# interjection: Greetings and other exclamations. -# e.g. おはよう, おはようございます, こんにちは, こんばんは, ありがとう, どうもありがとう, ありがとうございます, -# いただきます, ごちそうさま, さよなら, さようなら, はい, いいえ, ごめん, ごめんなさい -#感動詞 -# -##### -# symbol: unclassified Symbols. -記号 -# -# symbol-misc: A general symbol not in one of the categories below. -# e.g. [○◎@$〒→+] -記号-一般 -# -# symbol-comma: Commas -# e.g. [,、] -記号-読点 -# -# symbol-period: Periods and full stops. -# e.g. [..。] -記号-句点 -# -# symbol-space: Full-width whitespace. -記号-空白 -# -# symbol-open_bracket: -# e.g. [({‘“『【] -記号-括弧開 -# -# symbol-close_bracket: -# e.g. [)}’”』」】] -記号-括弧閉 -# -# symbol-alphabetic: -#記号-アルファベット -# -##### -# other: unclassified other -#その他 -# -# other-interjection: Words that are hard to classify as noun-suffixes or -# sentence-final particles. -# e.g. (だ)ァ -その他-間投 -# -##### -# filler: Aizuchi that occurs during a conversation or sounds inserted as filler. -# e.g. あの, うんと, えと -フィラー -# -##### -# non-verbal: non-verbal sound. -非言語音 -# -##### -# fragment: -#語断片 -# -##### -# unknown: unknown part of speech. -#未知語 -# -##### End of file diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ar.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ar.txt deleted file mode 100644 index 046829db6a2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ar.txt +++ /dev/null @@ -1,125 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Cleaned on October 11, 2009 (not normalized, so use before normalization) -# This means that when modifying this list, you might need to add some -# redundant entries, for example containing forms with both أ and ا -من -ومن -منها -منه -في -وفي -فيها -فيه -و -ف -ثم -او -أو -ب -بها -به -ا -أ -اى -اي -أي -أى -لا -ولا -الا -ألا -إلا -لكن -ما -وما -كما -فما -عن -مع -اذا -إذا -ان -أن -إن -انها -أنها -إنها -انه -أنه -إنه -بان -بأن -فان -فأن -وان -وأن -وإن -التى -التي -الذى -الذي -الذين -الى -الي -إلى -إلي -على -عليها -عليه -اما -أما -إما -ايضا -أيضا -كل -وكل -لم -ولم -لن -ولن -هى -هي -هو -وهى -وهي -وهو -فهى -فهي -فهو -انت -أنت -لك -لها -له -هذه -هذا -تلك -ذلك -هناك -كانت -كان -يكون -تكون -وكانت -وكان -غير -بعض -قد -نحو -بين -بينما -منذ -ضمن -حيث -الان -الآن -خلال -بعد -قبل -حتى -عند -عندما -لدى -جميع diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_bg.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_bg.txt deleted file mode 100644 index 1ae4ba2ae38..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_bg.txt +++ /dev/null @@ -1,193 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -а -аз -ако -ала -бе -без -беше -би -бил -била -били -било -близо -бъдат -бъде -бяха -в -вас -ваш -ваша -вероятно -вече -взема -ви -вие -винаги -все -всеки -всички -всичко -всяка -във -въпреки -върху -г -ги -главно -го -д -да -дали -до -докато -докога -дори -досега -доста -е -едва -един -ето -за -зад -заедно -заради -засега -затова -защо -защото -и -из -или -им -има -имат -иска -й -каза -как -каква -какво -както -какъв -като -кога -когато -което -които -кой -който -колко -която -къде -където -към -ли -м -ме -между -мен -ми -мнозина -мога -могат -може -моля -момента -му -н -на -над -назад -най -направи -напред -например -нас -не -него -нея -ни -ние -никой -нито -но -някои -някой -няма -обаче -около -освен -особено -от -отгоре -отново -още -пак -по -повече -повечето -под -поне -поради -после -почти -прави -пред -преди -през -при -пък -първо -с -са -само -се -сега -си -скоро -след -сме -според -сред -срещу -сте -съм -със -също -т -тази -така -такива -такъв -там -твой -те -тези -ти -тн -то -това -тогава -този -той -толкова -точно -трябва -тук -тъй -тя -тях -у -харесва -ч -че -често -чрез -ще -щом -я diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ca.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ca.txt deleted file mode 100644 index 3da65deafe1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ca.txt +++ /dev/null @@ -1,220 +0,0 @@ -# Catalan stopwords from http://github.com/vcl/cue.language (Apache 2 Licensed) -a -abans -ací -ah -així -això -al -als -aleshores -algun -alguna -algunes -alguns -alhora -allà -allí -allò -altra -altre -altres -amb -ambdós -ambdues -apa -aquell -aquella -aquelles -aquells -aquest -aquesta -aquestes -aquests -aquí -baix -cada -cadascú -cadascuna -cadascunes -cadascuns -com -contra -d'un -d'una -d'unes -d'uns -dalt -de -del -dels -des -després -dins -dintre -donat -doncs -durant -e -eh -el -els -em -en -encara -ens -entre -érem -eren -éreu -es -és -esta -està -estàvem -estaven -estàveu -esteu -et -etc -ets -fins -fora -gairebé -ha -han -has -havia -he -hem -heu -hi -ho -i -igual -iguals -ja -l'hi -la -les -li -li'n -llavors -m'he -ma -mal -malgrat -mateix -mateixa -mateixes -mateixos -me -mentre -més -meu -meus -meva -meves -molt -molta -moltes -molts -mon -mons -n'he -n'hi -ne -ni -no -nogensmenys -només -nosaltres -nostra -nostre -nostres -o -oh -oi -on -pas -pel -pels -per -però -perquè -poc -poca -pocs -poques -potser -propi -qual -quals -quan -quant -que -què -quelcom -qui -quin -quina -quines -quins -s'ha -s'han -sa -semblant -semblants -ses -seu -seus -seva -seva -seves -si -sobre -sobretot -sóc -solament -sols -son -són -sons -sota -sou -t'ha -t'han -t'he -ta -tal -també -tampoc -tan -tant -tanta -tantes -teu -teus -teva -teves -ton -tons -tot -tota -totes -tots -un -una -unes -uns -us -va -vaig -vam -van -vas -veu -vosaltres -vostra -vostre -vostres diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_cz.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_cz.txt deleted file mode 100644 index 53c6097dac7..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_cz.txt +++ /dev/null @@ -1,172 +0,0 @@ -a -s -k -o -i -u -v -z -dnes -cz -tímto -budeš -budem -byli -jseš -můj -svým -ta -tomto -tohle -tuto -tyto -jej -zda -proč -máte -tato -kam -tohoto -kdo -kteří -mi -nám -tom -tomuto -mít -nic -proto -kterou -byla -toho -protože -asi -ho -naši -napište -re -což -tím -takže -svých -její -svými -jste -aj -tu -tedy -teto -bylo -kde -ke -pravé -ji -nad -nejsou -či -pod -téma -mezi -přes -ty -pak -vám -ani -když -však -neg -jsem -tento -článku -články -aby -jsme -před -pta -jejich -byl -ještě -až -bez -také -pouze -první -vaše -která -nás -nový -tipy -pokud -může -strana -jeho -své -jiné -zprávy -nové -není -vás -jen -podle -zde -už -být -více -bude -již -než -který -by -které -co -nebo -ten -tak -má -při -od -po -jsou -jak -další -ale -si -se -ve -to -jako -za -zpět -ze -do -pro -je -na -atd -atp -jakmile -přičemž -já -on -ona -ono -oni -ony -my -vy -jí -ji -mě -mne -jemu -tomu -těm -těmu -němu -němuž -jehož -jíž -jelikož -jež -jakož -načež diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_da.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_da.txt deleted file mode 100644 index a3ff5fe122c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_da.txt +++ /dev/null @@ -1,108 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/danish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Danish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - -og | and -i | in -jeg | I -det | that (dem. pronoun)/it (pers. pronoun) -at | that (in front of a sentence)/to (with infinitive) -en | a/an -den | it (pers. pronoun)/that (dem. pronoun) -til | to/at/for/until/against/by/of/into, more -er | present tense of "to be" -som | who, as -på | on/upon/in/on/at/to/after/of/with/for, on -de | they -med | with/by/in, along -han | he -af | of/by/from/off/for/in/with/on, off -for | at/for/to/from/by/of/ago, in front/before, because -ikke | not -der | who/which, there/those -var | past tense of "to be" -mig | me/myself -sig | oneself/himself/herself/itself/themselves -men | but -et | a/an/one, one (number), someone/somebody/one -har | present tense of "to have" -om | round/about/for/in/a, about/around/down, if -vi | we -min | my -havde | past tense of "to have" -ham | him -hun | she -nu | now -over | over/above/across/by/beyond/past/on/about, over/past -da | then, when/as/since -fra | from/off/since, off, since -du | you -ud | out -sin | his/her/its/one's -dem | them -os | us/ourselves -op | up -man | you/one -hans | his -hvor | where -eller | or -hvad | what -skal | must/shall etc. -selv | myself/youself/herself/ourselves etc., even -her | here -alle | all/everyone/everybody etc. -vil | will (verb) -blev | past tense of "to stay/to remain/to get/to become" -kunne | could -ind | in -når | when -være | present tense of "to be" -dog | however/yet/after all -noget | something -ville | would -jo | you know/you see (adv), yes -deres | their/theirs -efter | after/behind/according to/for/by/from, later/afterwards -ned | down -skulle | should -denne | this -end | than -dette | this -mit | my/mine -også | also -under | under/beneath/below/during, below/underneath -have | have -dig | you -anden | other -hende | her -mine | my -alt | everything -meget | much/very, plenty of -sit | his, her, its, one's -sine | his, her, its, one's -vor | our -mod | against -disse | these -hvis | if -din | your/yours -nogle | some -hos | by/at -blive | be/become -mange | many -ad | by/through -bliver | present tense of "to be/to become" -hendes | her/hers -været | be -thi | for (conj) -jer | you -sådan | such, like this/like that diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_de.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_de.txt deleted file mode 100644 index f7703841887..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_de.txt +++ /dev/null @@ -1,292 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/german/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A German stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | The number of forms in this list is reduced significantly by passing it - | through the German stemmer. - - -aber | but - -alle | all -allem -allen -aller -alles - -als | than, as -also | so -am | an + dem -an | at - -ander | other -andere -anderem -anderen -anderer -anderes -anderm -andern -anderr -anders - -auch | also -auf | on -aus | out of -bei | by -bin | am -bis | until -bist | art -da | there -damit | with it -dann | then - -der | the -den -des -dem -die -das - -daß | that - -derselbe | the same -derselben -denselben -desselben -demselben -dieselbe -dieselben -dasselbe - -dazu | to that - -dein | thy -deine -deinem -deinen -deiner -deines - -denn | because - -derer | of those -dessen | of him - -dich | thee -dir | to thee -du | thou - -dies | this -diese -diesem -diesen -dieser -dieses - - -doch | (several meanings) -dort | (over) there - - -durch | through - -ein | a -eine -einem -einen -einer -eines - -einig | some -einige -einigem -einigen -einiger -einiges - -einmal | once - -er | he -ihn | him -ihm | to him - -es | it -etwas | something - -euer | your -eure -eurem -euren -eurer -eures - -für | for -gegen | towards -gewesen | p.p. of sein -hab | have -habe | have -haben | have -hat | has -hatte | had -hatten | had -hier | here -hin | there -hinter | behind - -ich | I -mich | me -mir | to me - - -ihr | you, to her -ihre -ihrem -ihren -ihrer -ihres -euch | to you - -im | in + dem -in | in -indem | while -ins | in + das -ist | is - -jede | each, every -jedem -jeden -jeder -jedes - -jene | that -jenem -jenen -jener -jenes - -jetzt | now -kann | can - -kein | no -keine -keinem -keinen -keiner -keines - -können | can -könnte | could -machen | do -man | one - -manche | some, many a -manchem -manchen -mancher -manches - -mein | my -meine -meinem -meinen -meiner -meines - -mit | with -muss | must -musste | had to -nach | to(wards) -nicht | not -nichts | nothing -noch | still, yet -nun | now -nur | only -ob | whether -oder | or -ohne | without -sehr | very - -sein | his -seine -seinem -seinen -seiner -seines - -selbst | self -sich | herself - -sie | they, she -ihnen | to them - -sind | are -so | so - -solche | such -solchem -solchen -solcher -solches - -soll | shall -sollte | should -sondern | but -sonst | else -über | over -um | about, around -und | and - -uns | us -unse -unsem -unsen -unser -unses - -unter | under -viel | much -vom | von + dem -von | from -vor | before -während | while -war | was -waren | were -warst | wast -was | what -weg | away, off -weil | because -weiter | further - -welche | which -welchem -welchen -welcher -welches - -wenn | when -werde | will -werden | will -wie | how -wieder | again -will | want -wir | we -wird | will -wirst | willst -wo | where -wollen | want -wollte | wanted -würde | would -würden | would -zu | to -zum | zu + dem -zur | zu + der -zwar | indeed -zwischen | between - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_el.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_el.txt deleted file mode 100644 index 232681f5bd6..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_el.txt +++ /dev/null @@ -1,78 +0,0 @@ -# Lucene Greek Stopwords list -# Note: by default this file is used after GreekLowerCaseFilter, -# so when modifying this file use 'σ' instead of 'ς' -ο -η -το -οι -τα -του -τησ -των -τον -την -και -κι -κ -ειμαι -εισαι -ειναι -ειμαστε -ειστε -στο -στον -στη -στην -μα -αλλα -απο -για -προσ -με -σε -ωσ -παρα -αντι -κατα -μετα -θα -να -δε -δεν -μη -μην -επι -ενω -εαν -αν -τοτε -που -πωσ -ποιοσ -ποια -ποιο -ποιοι -ποιεσ -ποιων -ποιουσ -αυτοσ -αυτη -αυτο -αυτοι -αυτων -αυτουσ -αυτεσ -αυτα -εκεινοσ -εκεινη -εκεινο -εκεινοι -εκεινεσ -εκεινα -εκεινων -εκεινουσ -οπωσ -ομωσ -ισωσ -οσο -οτι diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_en.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_en.txt deleted file mode 100644 index 2c164c0b2a1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_en.txt +++ /dev/null @@ -1,54 +0,0 @@ -# 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. - -# a couple of test stopwords to test that the words are really being -# configured from this file: -stopworda -stopwordb - -# Standard english stop words taken from Lucene's StopAnalyzer -a -an -and -are -as -at -be -but -by -for -if -in -into -is -it -no -not -of -on -or -such -that -the -their -then -there -these -they -this -to -was -will -with diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_es.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_es.txt deleted file mode 100644 index 2db14760075..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_es.txt +++ /dev/null @@ -1,354 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/spanish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Spanish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | from, of -la | the, her -que | who, that -el | the -en | in -y | and -a | to -los | the, them -del | de + el -se | himself, from him etc -las | the, them -por | for, by, etc -un | a -para | for -con | with -no | no -una | a -su | his, her -al | a + el - | es from SER -lo | him -como | how -más | more -pero | pero -sus | su plural -le | to him, her -ya | already -o | or - | fue from SER -este | this - | ha from HABER -sí | himself etc -porque | because -esta | this - | son from SER -entre | between - | está from ESTAR -cuando | when -muy | very -sin | without -sobre | on - | ser from SER - | tiene from TENER -también | also -me | me -hasta | until -hay | there is/are -donde | where - | han from HABER -quien | whom, that - | están from ESTAR - | estado from ESTAR -desde | from -todo | all -nos | us -durante | during - | estados from ESTAR -todos | all -uno | a -les | to them -ni | nor -contra | against -otros | other - | fueron from SER -ese | that -eso | that - | había from HABER -ante | before -ellos | they -e | and (variant of y) -esto | this -mí | me -antes | before -algunos | some -qué | what? -unos | a -yo | I -otro | other -otras | other -otra | other -él | he -tanto | so much, many -esa | that -estos | these -mucho | much, many -quienes | who -nada | nothing -muchos | many -cual | who - | sea from SER -poco | few -ella | she -estar | to be - | haber from HABER -estas | these - | estaba from ESTAR - | estamos from ESTAR -algunas | some -algo | something -nosotros | we - - | other forms - -mi | me -mis | mi plural -tú | thou -te | thee -ti | thee -tu | thy -tus | tu plural -ellas | they -nosotras | we -vosotros | you -vosotras | you -os | you -mío | mine -mía | -míos | -mías | -tuyo | thine -tuya | -tuyos | -tuyas | -suyo | his, hers, theirs -suya | -suyos | -suyas | -nuestro | ours -nuestra | -nuestros | -nuestras | -vuestro | yours -vuestra | -vuestros | -vuestras | -esos | those -esas | those - - | forms of estar, to be (not including the infinitive): -estoy -estás -está -estamos -estáis -están -esté -estés -estemos -estéis -estén -estaré -estarás -estará -estaremos -estaréis -estarán -estaría -estarías -estaríamos -estaríais -estarían -estaba -estabas -estábamos -estabais -estaban -estuve -estuviste -estuvo -estuvimos -estuvisteis -estuvieron -estuviera -estuvieras -estuviéramos -estuvierais -estuvieran -estuviese -estuvieses -estuviésemos -estuvieseis -estuviesen -estando -estado -estada -estados -estadas -estad - - | forms of haber, to have (not including the infinitive): -he -has -ha -hemos -habéis -han -haya -hayas -hayamos -hayáis -hayan -habré -habrás -habrá -habremos -habréis -habrán -habría -habrías -habríamos -habríais -habrían -había -habías -habíamos -habíais -habían -hube -hubiste -hubo -hubimos -hubisteis -hubieron -hubiera -hubieras -hubiéramos -hubierais -hubieran -hubiese -hubieses -hubiésemos -hubieseis -hubiesen -habiendo -habido -habida -habidos -habidas - - | forms of ser, to be (not including the infinitive): -soy -eres -es -somos -sois -son -sea -seas -seamos -seáis -sean -seré -serás -será -seremos -seréis -serán -sería -serías -seríamos -seríais -serían -era -eras -éramos -erais -eran -fui -fuiste -fue -fuimos -fuisteis -fueron -fuera -fueras -fuéramos -fuerais -fueran -fuese -fueses -fuésemos -fueseis -fuesen -siendo -sido - | sed also means 'thirst' - - | forms of tener, to have (not including the infinitive): -tengo -tienes -tiene -tenemos -tenéis -tienen -tenga -tengas -tengamos -tengáis -tengan -tendré -tendrás -tendrá -tendremos -tendréis -tendrán -tendría -tendrías -tendríamos -tendríais -tendrían -tenía -tenías -teníamos -teníais -tenían -tuve -tuviste -tuvo -tuvimos -tuvisteis -tuvieron -tuviera -tuvieras -tuviéramos -tuvierais -tuvieran -tuviese -tuvieses -tuviésemos -tuvieseis -tuviesen -teniendo -tenido -tenida -tenidos -tenidas -tened - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_eu.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_eu.txt deleted file mode 100644 index 25f1db93460..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_eu.txt +++ /dev/null @@ -1,99 +0,0 @@ -# example set of basque stopwords -al -anitz -arabera -asko -baina -bat -batean -batek -bati -batzuei -batzuek -batzuetan -batzuk -bera -beraiek -berau -berauek -bere -berori -beroriek -beste -bezala -da -dago -dira -ditu -du -dute -edo -egin -ere -eta -eurak -ez -gainera -gu -gutxi -guzti -haiei -haiek -haietan -hainbeste -hala -han -handik -hango -hara -hari -hark -hartan -hau -hauei -hauek -hauetan -hemen -hemendik -hemengo -hi -hona -honek -honela -honetan -honi -hor -hori -horiei -horiek -horietan -horko -horra -horrek -horrela -horretan -horri -hortik -hura -izan -ni -noiz -nola -non -nondik -nongo -nor -nora -ze -zein -zen -zenbait -zenbat -zer -zergatik -ziren -zituen -zu -zuek -zuen -zuten diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fa.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fa.txt deleted file mode 100644 index 723641c6da7..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fa.txt +++ /dev/null @@ -1,313 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Note: by default this file is used after normalization, so when adding entries -# to this file, use the arabic 'ي' instead of 'ی' -انان -نداشته -سراسر -خياه -ايشان -وي -تاكنون -بيشتري -دوم -پس -ناشي -وگو -يا -داشتند -سپس -هنگام -هرگز -پنج -نشان -امسال -ديگر -گروهي -شدند -چطور -ده -و -دو -نخستين -ولي -چرا -چه -وسط -ه -كدام -قابل -يك -رفت -هفت -همچنين -در -هزار -بله -بلي -شايد -اما -شناسي -گرفته -دهد -داشته -دانست -داشتن -خواهيم -ميليارد -وقتيكه -امد -خواهد -جز -اورده -شده -بلكه -خدمات -شدن -برخي -نبود -بسياري -جلوگيري -حق -كردند -نوعي -بعري -نكرده -نظير -نبايد -بوده -بودن -داد -اورد -هست -جايي -شود -دنبال -داده -بايد -سابق -هيچ -همان -انجا -كمتر -كجاست -گردد -كسي -تر -مردم -تان -دادن -بودند -سري -جدا -ندارند -مگر -يكديگر -دارد -دهند -بنابراين -هنگامي -سمت -جا -انچه -خود -دادند -زياد -دارند -اثر -بدون -بهترين -بيشتر -البته -به -براساس -بيرون -كرد -بعضي -گرفت -توي -اي -ميليون -او -جريان -تول -بر -مانند -برابر -باشيم -مدتي -گويند -اكنون -تا -تنها -جديد -چند -بي -نشده -كردن -كردم -گويد -كرده -كنيم -نمي -نزد -روي -قصد -فقط -بالاي -ديگران -اين -ديروز -توسط -سوم -ايم -دانند -سوي -استفاده -شما -كنار -داريم -ساخته -طور -امده -رفته -نخست -بيست -نزديك -طي -كنيد -از -انها -تمامي -داشت -يكي -طريق -اش -چيست -روب -نمايد -گفت -چندين -چيزي -تواند -ام -ايا -با -ان -ايد -ترين -اينكه -ديگري -راه -هايي -بروز -همچنان -پاعين -كس -حدود -مختلف -مقابل -چيز -گيرد -ندارد -ضد -همچون -سازي -شان -مورد -باره -مرسي -خويش -برخوردار -چون -خارج -شش -هنوز -تحت -ضمن -هستيم -گفته -فكر -بسيار -پيش -براي -روزهاي -انكه -نخواهد -بالا -كل -وقتي -كي -چنين -كه -گيري -نيست -است -كجا -كند -نيز -يابد -بندي -حتي -توانند -عقب -خواست -كنند -بين -تمام -همه -ما -باشند -مثل -شد -اري -باشد -اره -طبق -بعد -اگر -صورت -غير -جاي -بيش -ريزي -اند -زيرا -چگونه -بار -لطفا -مي -درباره -من -ديده -همين -گذاري -برداري -علت -گذاشته -هم -فوق -نه -ها -شوند -اباد -همواره -هر -اول -خواهند -چهار -نام -امروز -مان -هاي -قبل -كنم -سعي -تازه -را -هستند -زير -جلوي -عنوان -بود diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fi.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fi.txt deleted file mode 100644 index addad798c4b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fi.txt +++ /dev/null @@ -1,95 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/finnish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - -| forms of BE - -olla -olen -olet -on -olemme -olette -ovat -ole | negative form - -oli -olisi -olisit -olisin -olisimme -olisitte -olisivat -olit -olin -olimme -olitte -olivat -ollut -olleet - -en | negation -et -ei -emme -ette -eivät - -|Nom Gen Acc Part Iness Elat Illat Adess Ablat Allat Ess Trans -minä minun minut minua minussa minusta minuun minulla minulta minulle | I -sinä sinun sinut sinua sinussa sinusta sinuun sinulla sinulta sinulle | you -hän hänen hänet häntä hänessä hänestä häneen hänellä häneltä hänelle | he she -me meidän meidät meitä meissä meistä meihin meillä meiltä meille | we -te teidän teidät teitä teissä teistä teihin teillä teiltä teille | you -he heidän heidät heitä heissä heistä heihin heillä heiltä heille | they - -tämä tämän tätä tässä tästä tähän tallä tältä tälle tänä täksi | this -tuo tuon tuotä tuossa tuosta tuohon tuolla tuolta tuolle tuona tuoksi | that -se sen sitä siinä siitä siihen sillä siltä sille sinä siksi | it -nämä näiden näitä näissä näistä näihin näillä näiltä näille näinä näiksi | these -nuo noiden noita noissa noista noihin noilla noilta noille noina noiksi | those -ne niiden niitä niissä niistä niihin niillä niiltä niille niinä niiksi | they - -kuka kenen kenet ketä kenessä kenestä keneen kenellä keneltä kenelle kenenä keneksi| who -ketkä keiden ketkä keitä keissä keistä keihin keillä keiltä keille keinä keiksi | (pl) -mikä minkä minkä mitä missä mistä mihin millä miltä mille minä miksi | which what -mitkä | (pl) - -joka jonka jota jossa josta johon jolla jolta jolle jona joksi | who which -jotka joiden joita joissa joista joihin joilla joilta joille joina joiksi | (pl) - -| conjunctions - -että | that -ja | and -jos | if -koska | because -kuin | than -mutta | but -niin | so -sekä | and -sillä | for -tai | or -vaan | but -vai | or -vaikka | although - - -| prepositions - -kanssa | with -mukaan | according to -noin | about -poikki | across -yli | over, across - -| other - -kun | when -niin | so -nyt | now -itse | self - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fr.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fr.txt deleted file mode 100644 index c00837ea939..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_fr.txt +++ /dev/null @@ -1,183 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/french/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A French stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -au | a + le -aux | a + les -avec | with -ce | this -ces | these -dans | with -de | of -des | de + les -du | de + le -elle | she -en | `of them' etc -et | and -eux | them -il | he -je | I -la | the -le | the -leur | their -lui | him -ma | my (fem) -mais | but -me | me -même | same; as in moi-même (myself) etc -mes | me (pl) -moi | me -mon | my (masc) -ne | not -nos | our (pl) -notre | our -nous | we -on | one -ou | where -par | by -pas | not -pour | for -qu | que before vowel -que | that -qui | who -sa | his, her (fem) -se | oneself -ses | his (pl) -son | his, her (masc) -sur | on -ta | thy (fem) -te | thee -tes | thy (pl) -toi | thee -ton | thy (masc) -tu | thou -un | a -une | a -vos | your (pl) -votre | your -vous | you - - | single letter forms - -c | c' -d | d' -j | j' -l | l' -à | to, at -m | m' -n | n' -s | s' -t | t' -y | there - - | forms of être (not including the infinitive): -été -étée -étées -étés -étant -suis -es -est -sommes -êtes -sont -serai -seras -sera -serons -serez -seront -serais -serait -serions -seriez -seraient -étais -était -étions -étiez -étaient -fus -fut -fûmes -fûtes -furent -sois -soit -soyons -soyez -soient -fusse -fusses -fût -fussions -fussiez -fussent - - | forms of avoir (not including the infinitive): -ayant -eu -eue -eues -eus -ai -as -avons -avez -ont -aurai -auras -aura -aurons -aurez -auront -aurais -aurait -aurions -auriez -auraient -avais -avait -avions -aviez -avaient -eut -eûmes -eûtes -eurent -aie -aies -ait -ayons -ayez -aient -eusse -eusses -eût -eussions -eussiez -eussent - - | Later additions (from Jean-Christophe Deschamps) -ceci | this -celà  | that -cet | this -cette | this -ici | here -ils | they -les | the (pl) -leurs | their (pl) -quel | which -quels | which -quelle | which -quelles | which -sans | without -soi | oneself - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ga.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ga.txt deleted file mode 100644 index 9ff88d747e5..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ga.txt +++ /dev/null @@ -1,110 +0,0 @@ - -a -ach -ag -agus -an -aon -ar -arna -as -b' -ba -beirt -bhúr -caoga -ceathair -ceathrar -chomh -chtó -chuig -chun -cois -céad -cúig -cúigear -d' -daichead -dar -de -deich -deichniúr -den -dhá -do -don -dtí -dá -dár -dó -faoi -faoin -faoina -faoinár -fara -fiche -gach -gan -go -gur -haon -hocht -i -iad -idir -in -ina -ins -inár -is -le -leis -lena -lenár -m' -mar -mo -mé -na -nach -naoi -naonúr -ná -ní -níor -nó -nócha -ocht -ochtar -os -roimh -sa -seacht -seachtar -seachtó -seasca -seisear -siad -sibh -sinn -sna -sé -sí -tar -thar -thú -triúr -trí -trína -trínár -tríocha -tú -um -ár -é -éis -í -ó -ón -óna -ónár diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_gl.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_gl.txt deleted file mode 100644 index d8760b12c14..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_gl.txt +++ /dev/null @@ -1,161 +0,0 @@ -# galican stopwords -a -aínda -alí -aquel -aquela -aquelas -aqueles -aquilo -aquí -ao -aos -as -así -á -ben -cando -che -co -coa -comigo -con -connosco -contigo -convosco -coas -cos -cun -cuns -cunha -cunhas -da -dalgunha -dalgunhas -dalgún -dalgúns -das -de -del -dela -delas -deles -desde -deste -do -dos -dun -duns -dunha -dunhas -e -el -ela -elas -eles -en -era -eran -esa -esas -ese -eses -esta -estar -estaba -está -están -este -estes -estiven -estou -eu -é -facer -foi -foron -fun -había -hai -iso -isto -la -las -lle -lles -lo -los -mais -me -meu -meus -min -miña -miñas -moi -na -nas -neste -nin -no -non -nos -nosa -nosas -noso -nosos -nós -nun -nunha -nuns -nunhas -o -os -ou -ó -ós -para -pero -pode -pois -pola -polas -polo -polos -por -que -se -senón -ser -seu -seus -sexa -sido -sobre -súa -súas -tamén -tan -te -ten -teñen -teño -ter -teu -teus -ti -tido -tiña -tiven -túa -túas -un -unha -unhas -uns -vos -vosa -vosas -voso -vosos -vós diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hi.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hi.txt deleted file mode 100644 index 86286bb083b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hi.txt +++ /dev/null @@ -1,235 +0,0 @@ -# Also see http://www.opensource.org/licenses/bsd-license.html -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# This file was created by Jacques Savoy and is distributed under the BSD license. -# Note: by default this file also contains forms normalized by HindiNormalizer -# for spelling variation (see section below), such that it can be used whether or -# not you enable that feature. When adding additional entries to this list, -# please add the normalized form as well. -अंदर -अत -अपना -अपनी -अपने -अभी -आदि -आप -इत्यादि -इन -इनका -इन्हीं -इन्हें -इन्हों -इस -इसका -इसकी -इसके -इसमें -इसी -इसे -उन -उनका -उनकी -उनके -उनको -उन्हीं -उन्हें -उन्हों -उस -उसके -उसी -उसे -एक -एवं -एस -ऐसे -और -कई -कर -करता -करते -करना -करने -करें -कहते -कहा -का -काफ़ी -कि -कितना -किन्हें -किन्हों -किया -किर -किस -किसी -किसे -की -कुछ -कुल -के -को -कोई -कौन -कौनसा -गया -घर -जब -जहाँ -जा -जितना -जिन -जिन्हें -जिन्हों -जिस -जिसे -जीधर -जैसा -जैसे -जो -तक -तब -तरह -तिन -तिन्हें -तिन्हों -तिस -तिसे -तो -था -थी -थे -दबारा -दिया -दुसरा -दूसरे -दो -द्वारा -न -नहीं -ना -निहायत -नीचे -ने -पर -पर -पहले -पूरा -पे -फिर -बनी -बही -बहुत -बाद -बाला -बिलकुल -भी -भीतर -मगर -मानो -मे -में -यदि -यह -यहाँ -यही -या -यिह -ये -रखें -रहा -रहे -ऱ्वासा -लिए -लिये -लेकिन -व -वर्ग -वह -वह -वहाँ -वहीं -वाले -वुह -वे -वग़ैरह -संग -सकता -सकते -सबसे -सभी -साथ -साबुत -साभ -सारा -से -सो -ही -हुआ -हुई -हुए -है -हैं -हो -होता -होती -होते -होना -होने -# additional normalized forms of the above -अपनि -जेसे -होति -सभि -तिंहों -इंहों -दवारा -इसि -किंहें -थि -उंहों -ओर -जिंहें -वहिं -अभि -बनि -हि -उंहिं -उंहें -हें -वगेरह -एसे -रवासा -कोन -निचे -काफि -उसि -पुरा -भितर -हे -बहि -वहां -कोइ -यहां -जिंहों -तिंहें -किसि -कइ -यहि -इंहिं -जिधर -इंहें -अदि -इतयादि -हुइ -कोनसा -इसकि -दुसरे -जहां -अप -किंहों -उनकि -भि -वरग -हुअ -जेसा -नहिं diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hu.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hu.txt deleted file mode 100644 index 1a96f1db6f2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hu.txt +++ /dev/null @@ -1,209 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/hungarian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - -| Hungarian stop word list -| prepared by Anna Tordai - -a -ahogy -ahol -aki -akik -akkor -alatt -által -általában -amely -amelyek -amelyekben -amelyeket -amelyet -amelynek -ami -amit -amolyan -amíg -amikor -át -abban -ahhoz -annak -arra -arról -az -azok -azon -azt -azzal -azért -aztán -azután -azonban -bár -be -belül -benne -cikk -cikkek -cikkeket -csak -de -e -eddig -egész -egy -egyes -egyetlen -egyéb -egyik -egyre -ekkor -el -elég -ellen -elő -először -előtt -első -én -éppen -ebben -ehhez -emilyen -ennek -erre -ez -ezt -ezek -ezen -ezzel -ezért -és -fel -felé -hanem -hiszen -hogy -hogyan -igen -így -illetve -ill. -ill -ilyen -ilyenkor -ison -ismét -itt -jó -jól -jobban -kell -kellett -keresztül -keressünk -ki -kívül -között -közül -legalább -lehet -lehetett -legyen -lenne -lenni -lesz -lett -maga -magát -majd -majd -már -más -másik -meg -még -mellett -mert -mely -melyek -mi -mit -míg -miért -milyen -mikor -minden -mindent -mindenki -mindig -mint -mintha -mivel -most -nagy -nagyobb -nagyon -ne -néha -nekem -neki -nem -néhány -nélkül -nincs -olyan -ott -össze -ő -ők -őket -pedig -persze -rá -s -saját -sem -semmi -sok -sokat -sokkal -számára -szemben -szerint -szinte -talán -tehát -teljes -tovább -továbbá -több -úgy -ugyanis -új -újabb -újra -után -utána -utolsó -vagy -vagyis -valaki -valami -valamint -való -vagyok -van -vannak -volt -voltam -voltak -voltunk -vissza -vele -viszont -volna diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hy.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hy.txt deleted file mode 100644 index 60c1c50fbc8..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_hy.txt +++ /dev/null @@ -1,46 +0,0 @@ -# example set of Armenian stopwords. -այդ -այլ -այն -այս -դու -դուք -եմ -են -ենք -ես -եք -է -էի -էին -էինք -էիր -էիք -էր -ըստ -թ -ի -ին -իսկ -իր -կամ -համար -հետ -հետո -մենք -մեջ -մի -ն -նա -նաև -նրա -նրանք -որ -որը -որոնք -որպես -ու -ում -պիտի -վրա -և diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_id.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_id.txt deleted file mode 100644 index 4617f83a5c5..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_id.txt +++ /dev/null @@ -1,359 +0,0 @@ -# from appendix D of: A Study of Stemming Effects on Information -# Retrieval in Bahasa Indonesia -ada -adanya -adalah -adapun -agak -agaknya -agar -akan -akankah -akhirnya -aku -akulah -amat -amatlah -anda -andalah -antar -diantaranya -antara -antaranya -diantara -apa -apaan -mengapa -apabila -apakah -apalagi -apatah -atau -ataukah -ataupun -bagai -bagaikan -sebagai -sebagainya -bagaimana -bagaimanapun -sebagaimana -bagaimanakah -bagi -bahkan -bahwa -bahwasanya -sebaliknya -banyak -sebanyak -beberapa -seberapa -begini -beginian -beginikah -beginilah -sebegini -begitu -begitukah -begitulah -begitupun -sebegitu -belum -belumlah -sebelum -sebelumnya -sebenarnya -berapa -berapakah -berapalah -berapapun -betulkah -sebetulnya -biasa -biasanya -bila -bilakah -bisa -bisakah -sebisanya -boleh -bolehkah -bolehlah -buat -bukan -bukankah -bukanlah -bukannya -cuma -percuma -dahulu -dalam -dan -dapat -dari -daripada -dekat -demi -demikian -demikianlah -sedemikian -dengan -depan -di -dia -dialah -dini -diri -dirinya -terdiri -dong -dulu -enggak -enggaknya -entah -entahlah -terhadap -terhadapnya -hal -hampir -hanya -hanyalah -harus -haruslah -harusnya -seharusnya -hendak -hendaklah -hendaknya -hingga -sehingga -ia -ialah -ibarat -ingin -inginkah -inginkan -ini -inikah -inilah -itu -itukah -itulah -jangan -jangankan -janganlah -jika -jikalau -juga -justru -kala -kalau -kalaulah -kalaupun -kalian -kami -kamilah -kamu -kamulah -kan -kapan -kapankah -kapanpun -dikarenakan -karena -karenanya -ke -kecil -kemudian -kenapa -kepada -kepadanya -ketika -seketika -khususnya -kini -kinilah -kiranya -sekiranya -kita -kitalah -kok -lagi -lagian -selagi -lah -lain -lainnya -melainkan -selaku -lalu -melalui -terlalu -lama -lamanya -selama -selama -selamanya -lebih -terlebih -bermacam -macam -semacam -maka -makanya -makin -malah -malahan -mampu -mampukah -mana -manakala -manalagi -masih -masihkah -semasih -masing -mau -maupun -semaunya -memang -mereka -merekalah -meski -meskipun -semula -mungkin -mungkinkah -nah -namun -nanti -nantinya -nyaris -oleh -olehnya -seorang -seseorang -pada -padanya -padahal -paling -sepanjang -pantas -sepantasnya -sepantasnyalah -para -pasti -pastilah -per -pernah -pula -pun -merupakan -rupanya -serupa -saat -saatnya -sesaat -saja -sajalah -saling -bersama -sama -sesama -sambil -sampai -sana -sangat -sangatlah -saya -sayalah -se -sebab -sebabnya -sebuah -tersebut -tersebutlah -sedang -sedangkan -sedikit -sedikitnya -segala -segalanya -segera -sesegera -sejak -sejenak -sekali -sekalian -sekalipun -sesekali -sekaligus -sekarang -sekarang -sekitar -sekitarnya -sela -selain -selalu -seluruh -seluruhnya -semakin -sementara -sempat -semua -semuanya -sendiri -sendirinya -seolah -seperti -sepertinya -sering -seringnya -serta -siapa -siapakah -siapapun -disini -disinilah -sini -sinilah -sesuatu -sesuatunya -suatu -sesudah -sesudahnya -sudah -sudahkah -sudahlah -supaya -tadi -tadinya -tak -tanpa -setelah -telah -tentang -tentu -tentulah -tentunya -tertentu -seterusnya -tapi -tetapi -setiap -tiap -setidaknya -tidak -tidakkah -tidaklah -toh -waduh -wah -wahai -sewaktu -walau -walaupun -wong -yaitu -yakni -yang diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_it.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_it.txt deleted file mode 100644 index 4cb5b0891b1..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_it.txt +++ /dev/null @@ -1,301 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/italian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | An Italian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -ad | a (to) before vowel -al | a + il -allo | a + lo -ai | a + i -agli | a + gli -all | a + l' -agl | a + gl' -alla | a + la -alle | a + le -con | with -col | con + il -coi | con + i (forms collo, cogli etc are now very rare) -da | from -dal | da + il -dallo | da + lo -dai | da + i -dagli | da + gli -dall | da + l' -dagl | da + gll' -dalla | da + la -dalle | da + le -di | of -del | di + il -dello | di + lo -dei | di + i -degli | di + gli -dell | di + l' -degl | di + gl' -della | di + la -delle | di + le -in | in -nel | in + el -nello | in + lo -nei | in + i -negli | in + gli -nell | in + l' -negl | in + gl' -nella | in + la -nelle | in + le -su | on -sul | su + il -sullo | su + lo -sui | su + i -sugli | su + gli -sull | su + l' -sugl | su + gl' -sulla | su + la -sulle | su + le -per | through, by -tra | among -contro | against -io | I -tu | thou -lui | he -lei | she -noi | we -voi | you -loro | they -mio | my -mia | -miei | -mie | -tuo | -tua | -tuoi | thy -tue | -suo | -sua | -suoi | his, her -sue | -nostro | our -nostra | -nostri | -nostre | -vostro | your -vostra | -vostri | -vostre | -mi | me -ti | thee -ci | us, there -vi | you, there -lo | him, the -la | her, the -li | them -le | them, the -gli | to him, the -ne | from there etc -il | the -un | a -uno | a -una | a -ma | but -ed | and -se | if -perché | why, because -anche | also -come | how -dov | where (as dov') -dove | where -che | who, that -chi | who -cui | whom -non | not -più | more -quale | who, that -quanto | how much -quanti | -quanta | -quante | -quello | that -quelli | -quella | -quelle | -questo | this -questi | -questa | -queste | -si | yes -tutto | all -tutti | all - - | single letter forms: - -a | at -c | as c' for ce or ci -e | and -i | the -l | as l' -o | or - - | forms of avere, to have (not including the infinitive): - -ho -hai -ha -abbiamo -avete -hanno -abbia -abbiate -abbiano -avrò -avrai -avrà -avremo -avrete -avranno -avrei -avresti -avrebbe -avremmo -avreste -avrebbero -avevo -avevi -aveva -avevamo -avevate -avevano -ebbi -avesti -ebbe -avemmo -aveste -ebbero -avessi -avesse -avessimo -avessero -avendo -avuto -avuta -avuti -avute - - | forms of essere, to be (not including the infinitive): -sono -sei -è -siamo -siete -sia -siate -siano -sarò -sarai -sarà -saremo -sarete -saranno -sarei -saresti -sarebbe -saremmo -sareste -sarebbero -ero -eri -era -eravamo -eravate -erano -fui -fosti -fu -fummo -foste -furono -fossi -fosse -fossimo -fossero -essendo - - | forms of fare, to do (not including the infinitive, fa, fat-): -faccio -fai -facciamo -fanno -faccia -facciate -facciano -farò -farai -farà -faremo -farete -faranno -farei -faresti -farebbe -faremmo -fareste -farebbero -facevo -facevi -faceva -facevamo -facevate -facevano -feci -facesti -fece -facemmo -faceste -fecero -facessi -facesse -facessimo -facessero -facendo - - | forms of stare, to be (not including the infinitive): -sto -stai -sta -stiamo -stanno -stia -stiate -stiano -starò -starai -starà -staremo -starete -staranno -starei -staresti -starebbe -staremmo -stareste -starebbero -stavo -stavi -stava -stavamo -stavate -stavano -stetti -stesti -stette -stemmo -steste -stettero -stessi -stesse -stessimo -stessero -stando diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ja.txt deleted file mode 100644 index d4321be6b16..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ja.txt +++ /dev/null @@ -1,127 +0,0 @@ -# -# This file defines a stopword set for Japanese. -# -# This set is made up of hand-picked frequent terms from segmented Japanese Wikipedia. -# Punctuation characters and frequent kanji have mostly been left out. See LUCENE-3745 -# for frequency lists, etc. that can be useful for making your own set (if desired) -# -# Note that there is an overlap between these stopwords and the terms stopped when used -# in combination with the JapanesePartOfSpeechStopFilter. When editing this file, note -# that comments are not allowed on the same line as stopwords. -# -# Also note that stopping is done in a case-insensitive manner. Change your StopFilter -# configuration if you need case-sensitive stopping. Lastly, note that stopping is done -# using the same character width as the entries in this file. Since this StopFilter is -# normally done after a CJKWidthFilter in your chain, you would usually want your romaji -# entries to be in half-width and your kana entries to be in full-width. -# -の -に -は -を -た -が -で -て -と -し -れ -さ -ある -いる -も -する -から -な -こと -として -い -や -れる -など -なっ -ない -この -ため -その -あっ -よう -また -もの -という -あり -まで -られ -なる -へ -か -だ -これ -によって -により -おり -より -による -ず -なり -られる -において -ば -なかっ -なく -しかし -について -せ -だっ -その後 -できる -それ -う -ので -なお -のみ -でき -き -つ -における -および -いう -さらに -でも -ら -たり -その他 -に関する -たち -ます -ん -なら -に対して -特に -せる -及び -これら -とき -では -にて -ほか -ながら -うち -そして -とともに -ただし -かつて -それぞれ -または -お -ほど -ものの -に対する -ほとんど -と共に -といった -です -とも -ところ -ここ -##### End of file diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_lv.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_lv.txt deleted file mode 100644 index e21a23c06c3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_lv.txt +++ /dev/null @@ -1,172 +0,0 @@ -# Set of Latvian stopwords from A Stemming Algorithm for Latvian, Karlis Kreslins -# the original list of over 800 forms was refined: -# pronouns, adverbs, interjections were removed -# -# prepositions -aiz -ap -ar -apakš -ārpus -augšpus -bez -caur -dēļ -gar -iekš -iz -kopš -labad -lejpus -līdz -no -otrpus -pa -par -pār -pēc -pie -pirms -pret -priekš -starp -šaipus -uz -viņpus -virs -virspus -zem -apakšpus -# Conjunctions -un -bet -jo -ja -ka -lai -tomēr -tikko -turpretī -arī -kaut -gan -tādēļ -tā -ne -tikvien -vien -kā -ir -te -vai -kamēr -# Particles -ar -diezin -droši -diemžēl -nebūt -ik -it -taču -nu -pat -tiklab -iekšpus -nedz -tik -nevis -turpretim -jeb -iekam -iekām -iekāms -kolīdz -līdzko -tiklīdz -jebšu -tālab -tāpēc -nekā -itin -jā -jau -jel -nē -nezin -tad -tikai -vis -tak -iekams -vien -# modal verbs -būt -biju -biji -bija -bijām -bijāt -esmu -esi -esam -esat -būšu -būsi -būs -būsim -būsiet -tikt -tiku -tiki -tika -tikām -tikāt -tieku -tiec -tiek -tiekam -tiekat -tikšu -tiks -tiksim -tiksiet -tapt -tapi -tapāt -topat -tapšu -tapsi -taps -tapsim -tapsiet -kļūt -kļuvu -kļuvi -kļuva -kļuvām -kļuvāt -kļūstu -kļūsti -kļūst -kļūstam -kļūstat -kļūšu -kļūsi -kļūs -kļūsim -kļūsiet -# verbs -varēt -varēju -varējām -varēšu -varēsim -var -varēji -varējāt -varēsi -varēsiet -varat -varēja -varēs diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_nl.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_nl.txt deleted file mode 100644 index f4d61f5092c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_nl.txt +++ /dev/null @@ -1,117 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/dutch/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Dutch stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large sample of Dutch text. - - | Dutch stop words frequently exhibit homonym clashes. These are indicated - | clearly below. - -de | the -en | and -van | of, from -ik | I, the ego -te | (1) chez, at etc, (2) to, (3) too -dat | that, which -die | that, those, who, which -in | in, inside -een | a, an, one -hij | he -het | the, it -niet | not, nothing, naught -zijn | (1) to be, being, (2) his, one's, its -is | is -was | (1) was, past tense of all persons sing. of 'zijn' (to be) (2) wax, (3) the washing, (4) rise of river -op | on, upon, at, in, up, used up -aan | on, upon, to (as dative) -met | with, by -als | like, such as, when -voor | (1) before, in front of, (2) furrow -had | had, past tense all persons sing. of 'hebben' (have) -er | there -maar | but, only -om | round, about, for etc -hem | him -dan | then -zou | should/would, past tense all persons sing. of 'zullen' -of | or, whether, if -wat | what, something, anything -mijn | possessive and noun 'mine' -men | people, 'one' -dit | this -zo | so, thus, in this way -door | through by -over | over, across -ze | she, her, they, them -zich | oneself -bij | (1) a bee, (2) by, near, at -ook | also, too -tot | till, until -je | you -mij | me -uit | out of, from -der | Old Dutch form of 'van der' still found in surnames -daar | (1) there, (2) because -haar | (1) her, their, them, (2) hair -naar | (1) unpleasant, unwell etc, (2) towards, (3) as -heb | present first person sing. of 'to have' -hoe | how, why -heeft | present third person sing. of 'to have' -hebben | 'to have' and various parts thereof -deze | this -u | you -want | (1) for, (2) mitten, (3) rigging -nog | yet, still -zal | 'shall', first and third person sing. of verb 'zullen' (will) -me | me -zij | she, they -nu | now -ge | 'thou', still used in Belgium and south Netherlands -geen | none -omdat | because -iets | something, somewhat -worden | to become, grow, get -toch | yet, still -al | all, every, each -waren | (1) 'were' (2) to wander, (3) wares, (3) -veel | much, many -meer | (1) more, (2) lake -doen | to do, to make -toen | then, when -moet | noun 'spot/mote' and present form of 'to must' -ben | (1) am, (2) 'are' in interrogative second person singular of 'to be' -zonder | without -kan | noun 'can' and present form of 'to be able' -hun | their, them -dus | so, consequently -alles | all, everything, anything -onder | under, beneath -ja | yes, of course -eens | once, one day -hier | here -wie | who -werd | imperfect third person sing. of 'become' -altijd | always -doch | yet, but etc -wordt | present third person sing. of 'become' -wezen | (1) to be, (2) 'been' as in 'been fishing', (3) orphans -kunnen | to be able -ons | us/our -zelf | self -tegen | against, towards, at -na | after, near -reeds | already -wil | (1) present tense of 'want', (2) 'will', noun, (3) fender -kon | could; past tense of 'to be able' -niets | nothing -uw | your -iemand | somebody -geweest | been; past participle of 'be' -andere | other diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_no.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_no.txt deleted file mode 100644 index e76f36e69ed..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_no.txt +++ /dev/null @@ -1,192 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/norwegian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Norwegian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This stop word list is for the dominant bokmål dialect. Words unique - | to nynorsk are marked *. - - | Revised by Jan Bruusgaard , Jan 2005 - -og | and -i | in -jeg | I -det | it/this/that -at | to (w. inf.) -en | a/an -et | a/an -den | it/this/that -til | to -er | is/am/are -som | who/that -på | on -de | they / you(formal) -med | with -han | he -av | of -ikke | not -ikkje | not * -der | there -så | so -var | was/were -meg | me -seg | you -men | but -ett | one -har | have -om | about -vi | we -min | my -mitt | my -ha | have -hadde | had -hun | she -nå | now -over | over -da | when/as -ved | by/know -fra | from -du | you -ut | out -sin | your -dem | them -oss | us -opp | up -man | you/one -kan | can -hans | his -hvor | where -eller | or -hva | what -skal | shall/must -selv | self (reflective) -sjøl | self (reflective) -her | here -alle | all -vil | will -bli | become -ble | became -blei | became * -blitt | have become -kunne | could -inn | in -når | when -være | be -kom | come -noen | some -noe | some -ville | would -dere | you -som | who/which/that -deres | their/theirs -kun | only/just -ja | yes -etter | after -ned | down -skulle | should -denne | this -for | for/because -deg | you -si | hers/his -sine | hers/his -sitt | hers/his -mot | against -å | to -meget | much -hvorfor | why -dette | this -disse | these/those -uten | without -hvordan | how -ingen | none -din | your -ditt | your -blir | become -samme | same -hvilken | which -hvilke | which (plural) -sånn | such a -inni | inside/within -mellom | between -vår | our -hver | each -hvem | who -vors | us/ours -hvis | whose -både | both -bare | only/just -enn | than -fordi | as/because -før | before -mange | many -også | also -slik | just -vært | been -være | to be -båe | both * -begge | both -siden | since -dykk | your * -dykkar | yours * -dei | they * -deira | them * -deires | theirs * -deim | them * -di | your (fem.) * -då | as/when * -eg | I * -ein | a/an * -eit | a/an * -eitt | a/an * -elles | or * -honom | he * -hjå | at * -ho | she * -hoe | she * -henne | her -hennar | her/hers -hennes | hers -hoss | how * -hossen | how * -ikkje | not * -ingi | noone * -inkje | noone * -korleis | how * -korso | how * -kva | what/which * -kvar | where * -kvarhelst | where * -kven | who/whom * -kvi | why * -kvifor | why * -me | we * -medan | while * -mi | my * -mine | my * -mykje | much * -no | now * -nokon | some (masc./neut.) * -noka | some (fem.) * -nokor | some * -noko | some * -nokre | some * -si | his/hers * -sia | since * -sidan | since * -so | so * -somt | some * -somme | some * -um | about* -upp | up * -vere | be * -vore | was * -verte | become * -vort | become * -varte | became * -vart | became * - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_pt.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_pt.txt deleted file mode 100644 index 276c1b446f2..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_pt.txt +++ /dev/null @@ -1,251 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/portuguese/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Portuguese stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | of, from -a | the; to, at; her -o | the; him -que | who, that -e | and -do | de + o -da | de + a -em | in -um | a -para | for - | é from SER -com | with -não | not, no -uma | a -os | the; them -no | em + o -se | himself etc -na | em + a -por | for -mais | more -as | the; them -dos | de + os -como | as, like -mas | but - | foi from SER -ao | a + o -ele | he -das | de + as - | tem from TER -à | a + a -seu | his -sua | her -ou | or - | ser from SER -quando | when -muito | much - | há from HAV -nos | em + os; us -já | already, now - | está from EST -eu | I -também | also -só | only, just -pelo | per + o -pela | per + a -até | up to -isso | that -ela | he -entre | between - | era from SER -depois | after -sem | without -mesmo | same -aos | a + os - | ter from TER -seus | his -quem | whom -nas | em + as -me | me -esse | that -eles | they - | estão from EST -você | you - | tinha from TER - | foram from SER -essa | that -num | em + um -nem | nor -suas | her -meu | my -às | a + as -minha | my - | têm from TER -numa | em + uma -pelos | per + os -elas | they - | havia from HAV - | seja from SER -qual | which - | será from SER -nós | we - | tenho from TER -lhe | to him, her -deles | of them -essas | those -esses | those -pelas | per + as -este | this - | fosse from SER -dele | of him - - | other words. There are many contractions such as naquele = em+aquele, - | mo = me+o, but they are rare. - | Indefinite article plural forms are also rare. - -tu | thou -te | thee -vocês | you (plural) -vos | you -lhes | to them -meus | my -minhas -teu | thy -tua -teus -tuas -nosso | our -nossa -nossos -nossas - -dela | of her -delas | of them - -esta | this -estes | these -estas | these -aquele | that -aquela | that -aqueles | those -aquelas | those -isto | this -aquilo | that - - | forms of estar, to be (not including the infinitive): -estou -está -estamos -estão -estive -esteve -estivemos -estiveram -estava -estávamos -estavam -estivera -estivéramos -esteja -estejamos -estejam -estivesse -estivéssemos -estivessem -estiver -estivermos -estiverem - - | forms of haver, to have (not including the infinitive): -hei -há -havemos -hão -houve -houvemos -houveram -houvera -houvéramos -haja -hajamos -hajam -houvesse -houvéssemos -houvessem -houver -houvermos -houverem -houverei -houverá -houveremos -houverão -houveria -houveríamos -houveriam - - | forms of ser, to be (not including the infinitive): -sou -somos -são -era -éramos -eram -fui -foi -fomos -foram -fora -fôramos -seja -sejamos -sejam -fosse -fôssemos -fossem -for -formos -forem -serei -será -seremos -serão -seria -seríamos -seriam - - | forms of ter, to have (not including the infinitive): -tenho -tem -temos -tém -tinha -tínhamos -tinham -tive -teve -tivemos -tiveram -tivera -tivéramos -tenha -tenhamos -tenham -tivesse -tivéssemos -tivessem -tiver -tivermos -tiverem -terei -terá -teremos -terão -teria -teríamos -teriam diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ro.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ro.txt deleted file mode 100644 index 4fdee90a5ba..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ro.txt +++ /dev/null @@ -1,233 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -acea -aceasta -această -aceea -acei -aceia -acel -acela -acele -acelea -acest -acesta -aceste -acestea -aceşti -aceştia -acolo -acum -ai -aia -aibă -aici -al -ăla -ale -alea -ălea -altceva -altcineva -am -ar -are -aş -aşadar -asemenea -asta -ăsta -astăzi -astea -ăstea -ăştia -asupra -aţi -au -avea -avem -aveţi -azi -bine -bucur -bună -ca -că -căci -când -care -cărei -căror -cărui -cât -câte -câţi -către -câtva -ce -cel -ceva -chiar -cînd -cine -cineva -cît -cîte -cîţi -cîtva -contra -cu -cum -cumva -curând -curînd -da -dă -dacă -dar -datorită -de -deci -deja -deoarece -departe -deşi -din -dinaintea -dintr -dintre -drept -după -ea -ei -el -ele -eram -este -eşti -eu -face -fără -fi -fie -fiecare -fii -fim -fiţi -iar -ieri -îi -îl -îmi -împotriva -în -înainte -înaintea -încât -încît -încotro -între -întrucât -întrucît -îţi -la -lângă -le -li -lîngă -lor -lui -mă -mâine -mea -mei -mele -mereu -meu -mi -mine -mult -multă -mulţi -ne -nicăieri -nici -nimeni -nişte -noastră -noastre -noi -noştri -nostru -nu -ori -oricând -oricare -oricât -orice -oricînd -oricine -oricît -oricum -oriunde -până -pe -pentru -peste -pînă -poate -pot -prea -prima -primul -prin -printr -sa -să -săi -sale -sau -său -se -şi -sînt -sîntem -sînteţi -spre -sub -sunt -suntem -sunteţi -ta -tăi -tale -tău -te -ţi -ţie -tine -toată -toate -tot -toţi -totuşi -tu -un -una -unde -undeva -unei -unele -uneori -unor -vă -vi -voastră -voastre -voi -voştri -vostru -vouă -vreo -vreun diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ru.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ru.txt deleted file mode 100644 index 64307693457..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_ru.txt +++ /dev/null @@ -1,241 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/russian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | a russian stop word list. comments begin with vertical bar. each stop - | word is at the start of a line. - - | this is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | letter `ё' is translated to `е'. - -и | and -в | in/into -во | alternative form -не | not -что | what/that -он | he -на | on/onto -я | i -с | from -со | alternative form -как | how -а | milder form of `no' (but) -то | conjunction and form of `that' -все | all -она | she -так | so, thus -его | him -но | but -да | yes/and -ты | thou -к | towards, by -у | around, chez -же | intensifier particle -вы | you -за | beyond, behind -бы | conditional/subj. particle -по | up to, along -только | only -ее | her -мне | to me -было | it was -вот | here is/are, particle -от | away from -меня | me -еще | still, yet, more -нет | no, there isnt/arent -о | about -из | out of -ему | to him -теперь | now -когда | when -даже | even -ну | so, well -вдруг | suddenly -ли | interrogative particle -если | if -уже | already, but homonym of `narrower' -или | or -ни | neither -быть | to be -был | he was -него | prepositional form of его -до | up to -вас | you accusative -нибудь | indef. suffix preceded by hyphen -опять | again -уж | already, but homonym of `adder' -вам | to you -сказал | he said -ведь | particle `after all' -там | there -потом | then -себя | oneself -ничего | nothing -ей | to her -может | usually with `быть' as `maybe' -они | they -тут | here -где | where -есть | there is/are -надо | got to, must -ней | prepositional form of ей -для | for -мы | we -тебя | thee -их | them, their -чем | than -была | she was -сам | self -чтоб | in order to -без | without -будто | as if -человек | man, person, one -чего | genitive form of `what' -раз | once -тоже | also -себе | to oneself -под | beneath -жизнь | life -будет | will be -ж | short form of intensifer particle `же' -тогда | then -кто | who -этот | this -говорил | was saying -того | genitive form of `that' -потому | for that reason -этого | genitive form of `this' -какой | which -совсем | altogether -ним | prepositional form of `его', `они' -здесь | here -этом | prepositional form of `этот' -один | one -почти | almost -мой | my -тем | instrumental/dative plural of `тот', `то' -чтобы | full form of `in order that' -нее | her (acc.) -кажется | it seems -сейчас | now -были | they were -куда | where to -зачем | why -сказать | to say -всех | all (acc., gen. preposn. plural) -никогда | never -сегодня | today -можно | possible, one can -при | by -наконец | finally -два | two -об | alternative form of `о', about -другой | another -хоть | even -после | after -над | above -больше | more -тот | that one (masc.) -через | across, in -эти | these -нас | us -про | about -всего | in all, only, of all -них | prepositional form of `они' (they) -какая | which, feminine -много | lots -разве | interrogative particle -сказала | she said -три | three -эту | this, acc. fem. sing. -моя | my, feminine -впрочем | moreover, besides -хорошо | good -свою | ones own, acc. fem. sing. -этой | oblique form of `эта', fem. `this' -перед | in front of -иногда | sometimes -лучше | better -чуть | a little -том | preposn. form of `that one' -нельзя | one must not -такой | such a one -им | to them -более | more -всегда | always -конечно | of course -всю | acc. fem. sing of `all' -между | between - - - | b: some paradigms - | - | personal pronouns - | - | я меня мне мной [мною] - | ты тебя тебе тобой [тобою] - | он его ему им [него, нему, ним] - | она ее эи ею [нее, нэи, нею] - | оно его ему им [него, нему, ним] - | - | мы нас нам нами - | вы вас вам вами - | они их им ими [них, ним, ними] - | - | себя себе собой [собою] - | - | demonstrative pronouns: этот (this), тот (that) - | - | этот эта это эти - | этого эты это эти - | этого этой этого этих - | этому этой этому этим - | этим этой этим [этою] этими - | этом этой этом этих - | - | тот та то те - | того ту то те - | того той того тех - | тому той тому тем - | тем той тем [тою] теми - | том той том тех - | - | determinative pronouns - | - | (a) весь (all) - | - | весь вся все все - | всего всю все все - | всего всей всего всех - | всему всей всему всем - | всем всей всем [всею] всеми - | всем всей всем всех - | - | (b) сам (himself etc) - | - | сам сама само сами - | самого саму само самих - | самого самой самого самих - | самому самой самому самим - | самим самой самим [самою] самими - | самом самой самом самих - | - | stems of verbs `to be', `to have', `to do' and modal - | - | быть бы буд быв есть суть - | име - | дел - | мог мож мочь - | уме - | хоч хот - | долж - | можн - | нужн - | нельзя - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_sv.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_sv.txt deleted file mode 100644 index 22bddfd8cb3..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_sv.txt +++ /dev/null @@ -1,131 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/swedish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - - | A Swedish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | Swedish stop words occasionally exhibit homonym clashes. For example - | så = so, but also seed. These are indicated clearly below. - -och | and -det | it, this/that -att | to (with infinitive) -i | in, at -en | a -jag | I -hon | she -som | who, that -han | he -på | on -den | it, this/that -med | with -var | where, each -sig | him(self) etc -för | for -så | so (also: seed) -till | to -är | is -men | but -ett | a -om | if; around, about -hade | had -de | they, these/those -av | of -icke | not, no -mig | me -du | you -henne | her -då | then, when -sin | his -nu | now -har | have -inte | inte någon = no one -hans | his -honom | him -skulle | 'sake' -hennes | her -där | there -min | my -man | one (pronoun) -ej | nor -vid | at, by, on (also: vast) -kunde | could -något | some etc -från | from, off -ut | out -när | when -efter | after, behind -upp | up -vi | we -dem | them -vara | be -vad | what -över | over -än | than -dig | you -kan | can -sina | his -här | here -ha | have -mot | towards -alla | all -under | under (also: wonder) -någon | some etc -eller | or (else) -allt | all -mycket | much -sedan | since -ju | why -denna | this/that -själv | myself, yourself etc -detta | this/that -åt | to -utan | without -varit | was -hur | how -ingen | no -mitt | my -ni | you -bli | to be, become -blev | from bli -oss | us -din | thy -dessa | these/those -några | some etc -deras | their -blir | from bli -mina | my -samma | (the) same -vilken | who, that -er | you, your -sådan | such a -vår | our -blivit | from bli -dess | its -inom | within -mellan | between -sådant | such a -varför | why -varje | each -vilka | who, that -ditt | thy -vem | who -vilket | who, that -sitta | his -sådana | such a -vart | each -dina | thy -vars | whose -vårt | our -våra | our -ert | your -era | your -vilkas | whose - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_th.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_th.txt deleted file mode 100644 index 07f0fabe692..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_th.txt +++ /dev/null @@ -1,119 +0,0 @@ -# Thai stopwords from: -# "Opinion Detection in Thai Political News Columns -# Based on Subjectivity Analysis" -# Khampol Sukhum, Supot Nitsuwat, and Choochart Haruechaiyasak -ไว้ -ไม่ -ไป -ได้ -ให้ -ใน -โดย -แห่ง -แล้ว -และ -แรก -แบบ -แต่ -เอง -เห็น -เลย -เริ่ม -เรา -เมื่อ -เพื่อ -เพราะ -เป็นการ -เป็น -เปิดเผย -เปิด -เนื่องจาก -เดียวกัน -เดียว -เช่น -เฉพาะ -เคย -เข้า -เขา -อีก -อาจ -อะไร -ออก -อย่าง -อยู่ -อยาก -หาก -หลาย -หลังจาก -หลัง -หรือ -หนึ่ง -ส่วน -ส่ง -สุด -สําหรับ -ว่า -วัน -ลง -ร่วม -ราย -รับ -ระหว่าง -รวม -ยัง -มี -มาก -มา -พร้อม -พบ -ผ่าน -ผล -บาง -น่า -นี้ -นํา -นั้น -นัก -นอกจาก -ทุก -ที่สุด -ที่ -ทําให้ -ทํา -ทาง -ทั้งนี้ -ทั้ง -ถ้า -ถูก -ถึง -ต้อง -ต่างๆ -ต่าง -ต่อ -ตาม -ตั้งแต่ -ตั้ง -ด้าน -ด้วย -ดัง -ซึ่ง -ช่วง -จึง -จาก -จัด -จะ -คือ -ความ -ครั้ง -คง -ขึ้น -ของ -ขอ -ขณะ -ก่อน -ก็ -การ -กับ -กัน -กว่า -กล่าว diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_tr.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_tr.txt deleted file mode 100644 index 84d9408d4ea..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/stopwords_tr.txt +++ /dev/null @@ -1,212 +0,0 @@ -# Turkish stopwords from LUCENE-559 -# merged with the list from "Information Retrieval on Turkish Texts" -# (http://www.users.muohio.edu/canf/papers/JASIST2008offPrint.pdf) -acaba -altmış -altı -ama -ancak -arada -aslında -ayrıca -bana -bazı -belki -ben -benden -beni -benim -beri -beş -bile -bin -bir -birçok -biri -birkaç -birkez -birşey -birşeyi -biz -bize -bizden -bizi -bizim -böyle -böylece -bu -buna -bunda -bundan -bunlar -bunları -bunların -bunu -bunun -burada -çok -çünkü -da -daha -dahi -de -defa -değil -diğer -diye -doksan -dokuz -dolayı -dolayısıyla -dört -edecek -eden -ederek -edilecek -ediliyor -edilmesi -ediyor -eğer -elli -en -etmesi -etti -ettiği -ettiğini -gibi -göre -halen -hangi -hatta -hem -henüz -hep -hepsi -her -herhangi -herkesin -hiç -hiçbir -için -iki -ile -ilgili -ise -işte -itibaren -itibariyle -kadar -karşın -katrilyon -kendi -kendilerine -kendini -kendisi -kendisine -kendisini -kez -ki -kim -kimden -kime -kimi -kimse -kırk -milyar -milyon -mu -mü -mı -nasıl -ne -neden -nedenle -nerde -nerede -nereye -niye -niçin -o -olan -olarak -oldu -olduğu -olduğunu -olduklarını -olmadı -olmadığı -olmak -olması -olmayan -olmaz -olsa -olsun -olup -olur -olursa -oluyor -on -ona -ondan -onlar -onlardan -onları -onların -onu -onun -otuz -oysa -öyle -pek -rağmen -sadece -sanki -sekiz -seksen -sen -senden -seni -senin -siz -sizden -sizi -sizin -şey -şeyden -şeyi -şeyler -şöyle -şu -şuna -şunda -şundan -şunları -şunu -tarafından -trilyon -tüm -üç -üzere -var -vardı -ve -veya -ya -yani -yapacak -yapılan -yapılması -yapıyor -yapmak -yaptı -yaptığı -yaptığını -yaptıkları -yedi -yerine -yetmiş -yine -yirmi -yoksa -yüz -zaten diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/userdict_ja.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/userdict_ja.txt deleted file mode 100644 index 6f0368e4d81..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/lang/userdict_ja.txt +++ /dev/null @@ -1,29 +0,0 @@ -# -# This is a sample user dictionary for Kuromoji (JapaneseTokenizer) -# -# Add entries to this file in order to override the statistical model in terms -# of segmentation, readings and part-of-speech tags. Notice that entries do -# not have weights since they are always used when found. This is by-design -# in order to maximize ease-of-use. -# -# Entries are defined using the following CSV format: -# , ... , ... , -# -# Notice that a single half-width space separates tokens and readings, and -# that the number tokens and readings must match exactly. -# -# Also notice that multiple entries with the same is undefined. -# -# Whitespace only lines are ignored. Comments are not allowed on entry lines. -# - -# Custom segmentation for kanji compounds -日本経済新聞,日本 経済 新聞,ニホン ケイザイ シンブン,カスタム名詞 -関西国際空港,関西 国際 空港,カンサイ コクサイ クウコウ,カスタム名詞 - -# Custom segmentation for compound katakana -トートバッグ,トート バッグ,トート バッグ,かずカナ名詞 -ショルダーバッグ,ショルダー バッグ,ショルダー バッグ,かずカナ名詞 - -# Custom reading for former sumo wrestler -朝青龍,朝青龍,アサショウリュウ,カスタム人名 diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/protwords.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/protwords.txt deleted file mode 100644 index 1dfc0abecbf..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/protwords.txt +++ /dev/null @@ -1,21 +0,0 @@ -# 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. - -#----------------------------------------------------------------------- -# Use a protected word file to protect against the stemmer reducing two -# unrelated words to the same base word. - -# Some non-words that normally won't be encountered, -# just to test that they won't be stemmed. -dontstems -zwhacky - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/schema.xml b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/schema.xml deleted file mode 100644 index aed9f441fd6..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/schema.xml +++ /dev/null @@ -1,893 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - id - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/solrconfig.xml b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/solrconfig.xml deleted file mode 100644 index 05de0b628c9..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/solrconfig.xml +++ /dev/null @@ -1,1426 +0,0 @@ - - - - - - - - - ${tests.luceneMatchVersion:LATEST} - - - - - - - - - - - - - - - - - - - - - - - - - - - - ${solr.data.dir:} - - - - - - - - - - - - - - - - - - - - - 128 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ${solr.ulog.dir:} - - - - - ${solr.autoCommit.maxTime:60000} - false - - - - - - ${solr.autoSoftCommit.maxTime:1000} - - - - - - - - - - - - - - - - - - - - 1024 - - - - - - - - - - - - - - - - - - - - - - true - - - - - - 20 - - - 200 - - - - - - - - - - - - static firstSearcher warming in solrconfig.xml - - - - - - false - - - 4 - - - - - - - - - - - - - - - - - - - - - - - explicit - 10 - text - - - - - - - - - - - - - - explicit - json - true - text - - - - - - - - - textSpell - - - - - - default - name - solr.DirectSolrSpellChecker - - internal - - 0.5 - - 2 - - 1 - - 5 - - 4 - - 0.01 - - - - - - wordbreak - solr.WordBreakSolrSpellChecker - name - true - true - 10 - - - - - - - - - - - - - - - - text - - default - wordbreak - on - true - 10 - 5 - 5 - true - true - 10 - 5 - - - spellcheck - - - - - - - - - - text - true - - - tvComponent - - - - - - - - - default - - - org.carrot2.clustering.lingo.LingoClusteringAlgorithm - - - 20 - - - clustering/carrot2 - - - ENGLISH - - - stc - org.carrot2.clustering.stc.STCClusteringAlgorithm - - - - - - - true - default - true - - name - id - - features - - true - - - - false - - edismax - - text^0.5 features^1.0 name^1.2 sku^1.5 id^10.0 manu^1.1 cat^1.4 - - *:* - 10 - *,score - - - clustering - - - - - - - - - - true - - - terms - - - - - - - - string - elevate.xml - - - - - - explicit - text - - - elevator - - - - - - - - - - - 100 - - - - - - - - 70 - - 0.5 - - [-\w ,/\n\"']{20,200} - - - - - - - ]]> - ]]> - - - - - - - - - - - - - - - - - - - - - - - - ,, - ,, - ,, - ,, - ,]]> - ]]> - - - - - - 10 - .,!? - - - - - - - WORD - - - en - US - - - - - - - - - - - - - - - - - - - - - - text/plain; charset=UTF-8 - - - - - - - - - 5 - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/stopwords.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/stopwords.txt deleted file mode 100644 index ae1e83eeb3d..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/stopwords.txt +++ /dev/null @@ -1,14 +0,0 @@ -# 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. diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/synonyms.txt b/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/synonyms.txt deleted file mode 100644 index 7f72128303b..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcelltest/collection1/conf/synonyms.txt +++ /dev/null @@ -1,29 +0,0 @@ -# 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. - -#----------------------------------------------------------------------- -#some test synonym mappings unlikely to appear in real input text -aaafoo => aaabar -bbbfoo => bbbfoo bbbbar -cccfoo => cccbar cccbaz -fooaaa,baraaa,bazaaa - -# Some synonym groups specific to this example -GB,gib,gigabyte,gigabytes -MB,mib,megabyte,megabytes -Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming -#after us won't split it into two words. - -# Synonym mappings can be used for spelling correction too -pixima => pixma - diff --git a/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml b/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml deleted file mode 100644 index 42ab0565f07..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml +++ /dev/null @@ -1,1437 +0,0 @@ - - - - - - - - - ${tests.luceneMatchVersion:LATEST} - - - - - - - - - - - - - - - - - - - - - - - - ${solr.data.dir:} - - - - - ${solr.hdfs.home:} - ${solr.hdfs.confdir:} - ${solr.hdfs.security.kerberos.enabled:false} - ${solr.hdfs.security.kerberos.keytabfile:} - ${solr.hdfs.security.kerberos.principal:} - ${solr.hdfs.blockcache.enabled:true} - ${solr.hdfs.blockcache.slab.count:1} - ${solr.hdfs.blockcache.direct.memory.allocation:true} - ${solr.hdfs.blockcache.blocksperbank:16384} - ${solr.hdfs.blockcache.read.enabled:true} - ${solr.hdfs.nrtcachingdirectory.enable:true} - ${solr.hdfs.nrtcachingdirectory.maxmergesizemb:16} - ${solr.hdfs.nrtcachingdirectory.maxcachedmb:192} - ${solr.hdfs.blockcache.global:false} - - - - - - - - - - - - - - - - - - 128 - - - - - - - - - - ${solr.lock.type:hdfs} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ${solr.ulog.dir:} - - - - - ${solr.autoCommit.maxTime:60000} - false - - - - - ${solr.autoSoftCommit.maxTime:1000} - - - - - - - - - - - - - - - - - - - 1024 - - - - - - - - - - - - - - - - - - - - - - true - - - - - - 20 - - - 200 - - - - - - - - - - - - static firstSearcher warming in solrconfig.xml - - - - - - false - - - - - - - - - - - - - - - - - - - - - - - explicit - 10 - text - - - - - - - - - - - - - - explicit - json - true - text - - - - - - - - text_general - - - - - - default - text - solr.DirectSolrSpellChecker - - internal - - 0.5 - - 2 - - 1 - - 5 - - 4 - - 0.01 - - - - - - wordbreak - solr.WordBreakSolrSpellChecker - name - true - true - 10 - - - - - - - - - - - - - - - - text - - default - wordbreak - on - true - 10 - 5 - 5 - true - true - 10 - 5 - - - spellcheck - - - - - - - - - - text - true - - - tvComponent - - - - - - - - - default - - - org.carrot2.clustering.lingo.LingoClusteringAlgorithm - - - 20 - - - clustering/carrot2 - - - ENGLISH - - - stc - org.carrot2.clustering.stc.STCClusteringAlgorithm - - - - - - - true - default - true - - name - id - - features - - true - - - - false - - edismax - - text^0.5 features^1.0 name^1.2 sku^1.5 id^10.0 manu^1.1 cat^1.4 - - *:* - 10 - *,score - - - clustering - - - - - - - - - - true - false - - - terms - - - - - - - - string - elevate.xml - - - - - - explicit - text - - - elevator - - - - - - - - - - - 100 - - - - - - - - 70 - - 0.5 - - [-\w ,/\n\"']{20,200} - - - - - - - ]]> - ]]> - - - - - - - - - - - - - - - - - - - - - - - - ,, - ,, - ,, - ,, - ,]]> - ]]> - - - - - - 10 - .,!? - - - - - - - WORD - - - en - US - - - - - - - - - - - - - - - - - - - - - - text/plain; charset=UTF-8 - - - - - - - - - 5 - - - - - - - - - - - - - - - diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/NullHeader.docx b/solr/contrib/morphlines-core/src/test-files/test-documents/NullHeader.docx deleted file mode 100644 index cc62b8d6bebaf06dc46dbce5d56089778d6f528f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4355 zcmaJ^1yoeq8m4<_kd7gghCxbthHj9MZV`|f!J(zQ8$?7T1Qe8%E|CF|29dgemz<$f zO5hFm-FM}>-o5```>b=;I{!ZZ|Lt#|^?iDp*f`V}1n40k9xjJ*Ehy0U4lsK?57?u} zLiR8ZM?oLAhi_9xG(HMbM{b`{vmb4$H;LS?WiBAWL4;5tnv+>(>~g|6r%%4$pDn*- zMbs#D(tf#^Ouc?SoaPO06KyJzu}^<#T)YLpI~q01EnB$Gt4`#|1kJ5Nf}>t^Wq#*! zgH&R5l+og&YUtLVw$Mp@B3GRV&R8=NCy=t|qZ9sQhDV5R?&8EOdm3JS`L2AtycYH@ z59dABhPwi@%uf7A3C3Z-#ZYH@|I(bds92)CxFEM6wCi`&_mx01xy@QK1HK|7u$p@{ z@vSEjO*t(&XhmyEM!UQ6FXBb~Q&0P;xIxm- z=u9lVV@~+6BUN5=luzFS_}XW^tP_eE`q1MJkAxNY9xyFM@yt_ZX!+WW>@d|RpIJfq zXm^T+*W@FP*_ilels~_hk6YYxHj3DCX=HnK5cLIdVo;J5{33#%Fh3?c$iz?ciR7UV zoD8y0bOp@0r25V`blYlJ{{{>d`VDgk3<`II!mSN`A2~j@{DWPo%{o`?I+BT4ra_RB z5ae?*H^dL;%LF^--KO1DmV@%ZhSW8Qz_q-*g@@K{^O@^Y-GGMxLbn}2J2j2dH&5ZO zLW~#HKK!uba)DUCT^3CGY+3P^A*nfym?r!Zy>NmT>4KakAm%Xxl~>l|luJfNmYU2@ z@bda(ymsnr%G;uJ-bVJbh!<^465=gwKv$XF0VX1(#${Sf7XJI#hJj!lu6mXYw=QDP zBVPhgY!toqh?8Hc1E8`|<3wEwq(OBc6A_&XT<{qYvWHBV0=`svP*Vs)K{lV&GLow! zw)>J6Jk@r*FAnb6pTd#mYARc*dH$%Xgkc79R8XVh22Dx1ydx2KkQ#BB2NKieb7?%^ zbKAwdLUvQ0D2X0THxBH7LzWzk>|dwW_R*s&ShW*B5OoTZD{Y5fl6%1F{m4^pzhVo$ z=|4G;dWgZ;ZW$(!+WOHKWTnLM1ZhOUEfVeP>MsmUx=yGe-w%twwz6g|9wAqHh~ z-3Zf<%HX3B>LNO*3GQWUXkOR_`pn=Hh(p8J)6nHMpyxlIp{w7JH_XF92;G&Z8=5ay z9lbZ5fim_|1n30+fP@I0fh}7JY0Wf=)e|R(%c~Q)Zho?NFP0k9*FAbEWQVL6dcF3$ z9*C9z>#iT0RQ=YQyGA8C33C?*c^^j7VtVm{8YQLWER%P&AH3ZbyphK`$9i6Qj9SwT zdeYqZ0CE>^QS^7hJVEiSIJwo`g(Xh0R(eV1s1Y9v^C-zj-L@M#;_P+yBnj$-#H3qG zKMoFrZep9aH*}_!4MUgx)00YBSr7RF@&^!9&3%ZReMIm0OxlS?+o|3YM(vzWr$IM< zbV!`{b$~i`ri%EzS5M;u6EE%K_F3EJ7FJd=w;s;ifdnl>_Wq?XQ&_`CJy5%M}-K~ZC5EJvd718e!b`HR+#_owHRcFM|F_BcoF8M zX0QQF`0Z&Qf=(N>+spsms!eLvw|7#m%8du#r!-NY%B>C{?Gg&x6+O21?T_f>=*+YgMZ)<1{vsE8H~^lg{+9EPv$w{YE^MlNQe{_-M^iG%v6t4p4$-G zIk6v_IP|0OaZ=xyz>B+Pr0-?CzQf$0c7G83zOi2!C_{`OGhXh6yh z&{O-xKXp^4*^f1JoKbciJ0UTE5|$jKyF%VDUp0@F^PsH#M>X?xY}XmoxHK6;M2{gu95vbcN~Fk!NkY8t=bv7#aby$CMH0MBuaQ= z@dd#+l1p81(L5#|Q7~A3_}XmBm&)svtn@s?JGmq2g?YBnx6WkWeiI6`wy=L>9NVFx z>u!v^`(2#|=0NDV!%*DDCao>(X%`EG;>><8wHII)4(c;!03R4|M02h_jMR@STwn-) zzfw?T^Dc#%mD6V%T&U8yW{EsaR>`S~+2f7}on?-E6=#3=zDNS8HC#ykz+XJ{PoR}ep=T0sKB=$rQ8$QOYNoev> zXK?w3+mML5q0owHM=GnJHm-PaTN0^QGKTkwkXp-9KC05fhM9M6KiE=2+@Iq)qs8F} zWAbh9TRWSW3az;7`=#8Bwf({mt;>SakZ-t^p402of~Ta{%AQ-Gnryc*Y3M3Wg_LAP z&2e;Jo@REya~=$xYaD^2!GM$vjPq0%2P#7#^Ct2)+?rAsPg0&<&DQ1E{h zA!*tR5V1zm4mZDHY1urLiJndd&6Ipfs(y}0t3>+E9*(ZDFDHh|Yv9d#IDQ@XZ7@+9 z+0qKKaWeVg_sk!*AosxtOU#1pd)!LR5_u95H~#rEjLhfxXZqRx5}=D{YvX&Y4Gh9> zR0;vZPa5;454(2?>%@CCtR2Dxr}Z@)Ns=!L?2CNyaG9&yRkKYnd75cim2W6k2e+8J z8WOpxrG6PFwJVC!y;II6%v4C7K}{5^g~jP0Nh^?}|LOx@v8RC1m&9YyAv~F*i5U0x zwC*-R!tYpIJa>5)abj@rzCo6st~n{+orLT?7Rr>Hx5uxUD*Kp{aX$`8@!Pw!Pi0bP z(z73<0Dgdl1VN$iKk61jP^a0c1ZIgk4AsXRs=W>fRg^+5+^e7sl6*c617|iW5%f;y zyi@@X(Aq@!jak>*1`EDAV$W1`0@=s|;)S@Yt+b3fTq~ETWnchGlJ!e$za`|9R)lY? z-0quUeCVd)Qukc{EIFHBX>D<0hAWM0-5BO^c8OXqsJIa>0;rfUI!AG++E?xYu?weV zVq(h{PzX%BN~;+>rIfXtjMGuYFQqK8R#QM?Ad;mnT3efLwYx-xIi?^SO0RfJW5P5t zh$NVbU0vRR!oT&_edce~yc32~<~^lJX)PI_=5AEb?|L$9+g}2?$X6q>-!#OC6XqXw z)UK}*vP5q=1*)e->#x2Rr-FK3XvJYKb%;gI%PtBiKO=nE7sE|6dS{BcfZSH86kOy@Bl4c{F0sdk2gRh>D)0$YpQapR3Ez8pATgmq+22o1mc zh^y3#;3(V3YD0t^gM`hU|vO#jnB|1}fWmCzr?B4McYO5ow| z$m&u+>z$dEikxpyG-(6`CtHo#>?;b!wzZn#;Nth3oibn61vO;Mi=+Uoz<0+xM%IN* zVVoA-WB17ddhYpm8!cBu;u5<0Qf?Az4(_typm1{wc$E$uK~YPFf{q{{Z-5aVqVAnH zFo0oLWU&37oEGJ%uvkZrQ&-Tj@R0GM2r>tf7xaW8Tq=v-P%DGvKH;{G5kEPzE5f8M zvGottXHQ+2Rf_J1`d^*XU#jo1BOLAwb$WcI`T%j7q@8T!3P;>?A^mftO~R1}HTyVz z7~r1FqJHwo=$@=)e<7GfAWGPpmm%V211T+ud$QP`_=mH@78#rDp^_@8S7yydV`Rn? z+>@>Yr1~Fqde_;q8|stNmt1?QZPTh(fFXuw&q!NL{d<-;um^*#wy5mcQMw1Z;3w#* zr-_M0jq%ejU$=o*CI3DC(&7Je&%dT$H%nJu@n=BL=v;G-zb0Py0#{z@XIP+p*Z*-= zzbXE_4FU0;ZZ^W diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/boilerplate.html b/solr/contrib/morphlines-core/src/test-files/test-documents/boilerplate.html deleted file mode 100644 index 615f84e85ee..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-documents/boilerplate.html +++ /dev/null @@ -1,58 +0,0 @@ - - - - - - - - Title - - - - - - - -
- - - - - -
boilerplatetext
-
- -

This is the real meat of the page, -and represents the text we want. -It has lots of juicy content. - -We assume that it won't get filtered out. -And that all of the lines will be in the -output. -

- -

-Here's another paragraph of text. -This is the end of the text. -

- -

footer

- - - diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/cars.csv b/solr/contrib/morphlines-core/src/test-files/test-documents/cars.csv deleted file mode 100644 index 8f1f9e1ae6a..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-documents/cars.csv +++ /dev/null @@ -1,6 +0,0 @@ -Age,Color,Extras,Type,Used -2,blue,GPS,"Gas, with electric","" -10,green,"Labeled ""Vintage, 1913""",,yes -100,red,"Labeled ""Vintage 1913""",yes -5,orange,none,"This is a -multi, line text",no \ No newline at end of file diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/cars.csv.gz b/solr/contrib/morphlines-core/src/test-files/test-documents/cars.csv.gz deleted file mode 100644 index ee2a951eddc077ee2c382113b9b9c62141fabdbb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 167 zcmV;Y09gMYiwFpht))=_17l%wb1q|Zb^vY6u?oUK5CqUZU$JaGMWR?(iinMcV6?v^ zOLB1D$=wE%zjtk{gz4tZ97e%0g_LnxG`oU!eF^SGhH8T@%!2dv20mLuZ?6+ckY2Oz z8O&8vW6UBEd~uyM12I2*RA;Z$?3*n!1FIC1HL?tU;Lm;84k^26>zJZ|+sPNQxav6v V^`PLRkjkQL;sY8H&w{)F001z!Ok)55 diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/cars.tar.gz b/solr/contrib/morphlines-core/src/test-files/test-documents/cars.tar.gz deleted file mode 100644 index 5ca3cf1cdcbaf7d468424a7c2a4afd402f3858e6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 298 zcmV+_0oDE=iwFP#+m=!Q1MSp7ON2la2XN2(DTX;e?2V2Np380voq~=w&a$$lF}h2A z`^Lng!$w5w68(P&Pjpb^%SRM8&p0{Qf(}{m8XGI)9~9m_KXI zv)qi=RmMg|Vv$Y!=kvdFm8W&JbF}Vi_dr*!x4wBAny#fA-&nU&MPcZ=^EH)e#w06C zqPp)%Ja%0xDGklW%&T)Z0zdL8|H+y1pO}yQ>vYV&;sp6$!ng8Y(B;-0UP#`$cG!C- wq%5BY*Ve8UW!;R6rIZv7UEps3000000000000000006MKS5bhNHvlLA0A22urT_o{ diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/complex.mbox b/solr/contrib/morphlines-core/src/test-files/test-documents/complex.mbox deleted file mode 100644 index 27f7017d265..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-documents/complex.mbox +++ /dev/null @@ -1,291 +0,0 @@ -From core-user-return-14700-apmail-hadoop-core-user-archive=hadoop.apache.org@hadoop.apache.org Mon Jun 01 04:28:28 2009 -Return-Path: -Delivered-To: apmail-hadoop-core-user-archive@www.apache.org -Received: (qmail 19921 invoked from network); 1 Jun 2009 04:28:28 -0000 -Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) - by minotaur.apache.org with SMTP; 1 Jun 2009 04:28:28 -0000 -Received: (qmail 84995 invoked by uid 500); 1 Jun 2009 04:28:38 -0000 -Delivered-To: apmail-hadoop-core-user-archive@hadoop.apache.org -Received: (qmail 84895 invoked by uid 500); 1 Jun 2009 04:28:38 -0000 -Mailing-List: contact core-user-help@hadoop.apache.org; run by ezmlm -Precedence: bulk -List-Help: -List-Unsubscribe: -List-Post: -List-Id: -Reply-To: core-user@hadoop.apache.org -Delivered-To: mailing list core-user@hadoop.apache.org -Received: (qmail 84885 invoked by uid 99); 1 Jun 2009 04:28:38 -0000 -Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) - by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 01 Jun 2009 04:28:38 +0000 -X-ASF-Spam-Status: No, hits=1.2 required=10.0 - tests=SPF_NEUTRAL -X-Spam-Check-By: apache.org -Received-SPF: neutral (athena.apache.org: local policy) -Received: from [69.147.107.21] (HELO mrout2-b.corp.re1.wahoo.com) (69.147.107.21) - by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 01 Jun 2009 04:28:26 +0000 -Received: from SNV-EXPF01.ds.corp.wahoo.com (snv-expf01.ds.corp.wahoo.com [207.126.227.250]) - by mrout2-b.corp.re1.wahoo.com (8.13.8/8.13.8/y.out) with ESMTP id n514QYA6099963 - for ; Sun, 31 May 2009 21:26:35 -0700 (PDT) -DomainKey-Signature: a=rsa-sha1; s=serpent; d=wahoo-inc.com; c=nofws; q=dns; - h=received:user-agent:date:subject:from:to:message-id: - thread-topic:thread-index:in-reply-to:mime-version:content-type: - content-transfer-encoding:x-originalarrivaltime; - b=YVtSNdgjeeSBS1yY3XDolul49i+HrgNG7QszMo9LzGnrwejjgsl5+iUM6EiQgEpV -Received: from SNV-EXVS08.ds.corp.wahoo.com ([207.126.227.9]) by SNV-EXPF01.ds.corp.wahoo.com with Microsoft SMTPSVC(6.0.3790.3959); - Sun, 31 May 2009 21:26:34 -0700 -Received: from 10.66.92.213 ([10.66.92.213]) by SNV-EXVS08.ds.corp.wahoo.com ([207.126.227.58]) with Microsoft Exchange Server HTTP-DAV ; - Mon, 1 Jun 2009 04:26:33 +0000 -User-Agent: Microsoft-Entourage/12.17.0.090302 -Date: Mon, 01 Jun 2009 09:56:31 +0530 -Subject: Re: question about when shuffle/sort start working -From: Sam Judgement -To: -Message-ID: -Thread-Topic: question about when shuffle/sort start working -Thread-Index: AcnicSNoBw19cMU8UEaXwAdZ1YYhuw== -In-Reply-To: <440622.41041.qm@web111005.mail.gq1.wahoo.com> -Mime-version: 1.0 -Content-type: text/plain; - charset="US-ASCII" -Content-transfer-encoding: 7bit -X-OriginalArrivalTime: 01 Jun 2009 04:26:34.0501 (UTC) FILETIME=[257EAB50:01C9E271] -X-Virus-Checked: Checked by ClamAV on apache.org - -When a Mapper completes, MapCompletionEvents are generated. Reducers try to -fetch map outputs for a given map only on the receipt of such events. - -Sam - - -On 5/30/09 10:00 AM, "Jianmin Foo" wrote: - -> Hi, -> I am being confused by the protocol between mapper and reducer. When mapper -> emitting the (key,value) pair done, is there any signal the mapper send out to -> hadoop framework in protocol to indicate that map is done and the shuffle/sort -> can begin for reducer? If there is no this signal in protocol, when the -> framework begin the shuffle/sort? -> -> Thanks, -> Jianmin -> -> -> -> - - -From core-user-return-14701-apmail-hadoop-core-user-archive=hadoop.apache.org@hadoop.apache.org Mon Jun 01 05:31:14 2009 -Return-Path: -Delivered-To: apmail-hadoop-core-user-archive@www.apache.org -Received: (qmail 38243 invoked from network); 1 Jun 2009 05:31:14 -0000 -Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) - by minotaur.apache.org with SMTP; 1 Jun 2009 05:31:14 -0000 -Received: (qmail 15621 invoked by uid 500); 1 Jun 2009 05:31:24 -0000 -Delivered-To: apmail-hadoop-core-user-archive@hadoop.apache.org -Received: (qmail 15557 invoked by uid 500); 1 Jun 2009 05:31:24 -0000 -Mailing-List: contact core-user-help@hadoop.apache.org; run by ezmlm -Precedence: bulk -List-Help: -List-Unsubscribe: -List-Post: -List-Id: -Reply-To: core-user@hadoop.apache.org -Delivered-To: mailing list core-user@hadoop.apache.org -Received: (qmail 15547 invoked by uid 99); 1 Jun 2009 05:31:24 -0000 -Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) - by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 01 Jun 2009 05:31:24 +0000 -X-ASF-Spam-Status: No, hits=2.2 required=10.0 - tests=HTML_MESSAGE,SPF_PASS -X-Spam-Check-By: apache.org -Received-SPF: pass (nike.apache.org: local policy) -Received: from [68.142.237.94] (HELO n9.bullet.re3.wahoo.com) (68.142.237.94) - by apache.org (qpsmtpd/0.29) with SMTP; Mon, 01 Jun 2009 05:31:11 +0000 -Received: from [68.142.237.88] by n9.bullet.re3.wahoo.com with NNFMP; 01 Jun 2009 05:30:50 -0000 -Received: from [67.195.9.82] by t4.bullet.re3.wahoo.com with NNFMP; 01 Jun 2009 05:30:49 -0000 -Received: from [67.195.9.99] by t2.bullet.mail.gq1.wahoo.com with NNFMP; 01 Jun 2009 05:30:49 -0000 -Received: from [127.0.0.1] by omp103.mail.gq1.wahoo.com with NNFMP; 01 Jun 2009 05:28:01 -0000 -X-wahoo-Newman-Property: ymail-3 -X-wahoo-Newman-Id: 796121.97519.bm@omp103.mail.gq1.wahoo.com -Received: (qmail 35264 invoked by uid 60001); 1 Jun 2009 05:30:49 -0000 -DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=wahoo.com; s=s1024; t=1243834249; bh=R8qzdi/IbLyO8UwpnaujDpT9E+6bJ7nkmZN2803EmRk=; h=Message-ID:X-YMail-OSG:Received:X-Mailer:References:Date:From:Subject:To:In-Reply-To:MIME-Version:Content-Type; b=vq4c6RIDbkuLPYd8mirusIXf6DqTb/IeT55In7W00Y5Sxx1ZiXBb78yE9+TDfXJ0elsEZvqv4ocyvolGE0eGtyYeJA0mZikpRNu6pidxPNpCplOcLHBRz7YQ7iERwv3TagRlWy2Xd3oD9ZeV0A05P7WUOiNNX1PUUJD1IVdrEZo= -DomainKey-Signature:a=rsa-sha1; q=dns; c=nofws; - s=s1024; d=wahoo.com; - h=Message-ID:X-YMail-OSG:Received:X-Mailer:References:Date:From:Subject:To:In-Reply-To:MIME-Version:Content-Type; - b=6HXZV98ON5vBwmE/xS8stVD0D2F4dkMY7a0suX5KVTb736JdR8G59mqBq/dWcpbFTLiCLtxi18LMb/dU1RKRGOEdn3l3j/jKXhBrhIgfg3qtNskPedXDKBvn7JGXiSkqpA/tUtPjvc0Uuk8/LaA01SQTz40Engg7nD8/EJdIAhA=; -Message-ID: <592088.35091.qm@web111010.mail.gq1.wahoo.com> -X-YMail-OSG: KzhhrJYVM1m.MCS6vRpRP2ZZO2PrfnbngosELDCIa91ZqvhJph4RdmzfUW0jw9W04RCSch1K730bPohwNpNBIk2QR_zt4_mfbhfq7YEPkSoz9LSXG90P9vIo5Fc8qyZN0U6vA9gtdyGQTpN5ahvillUH9nAF0TMWv2SvZJLjPlQ0Z0p8oK8ltBwGTgLrM8Jtdn9D29yoRyi3_EpVOfdD9OP.EK50Vr1XwSUYMbnpZ0WGHMwd.Yig7A6Elwadm3YVbfOdx2mfrG.jQsUAxQjRBNvbrOM57.FaE11kHTe9aoBWSeihNg-- -Received: from [216.145.54.7] by web111010.mail.gq1.wahoo.com via HTTP; Sun, 31 May 2009 22:30:49 PDT -X-Mailer: wahooMailRC/1277.43 wahooMailWebService/0.7.289.10 -References: -Date: Sun, 31 May 2009 22:30:49 -0700 (PDT) -From: Jianmin Foo -Subject: Re: question about when shuffle/sort start working -To: core-user@hadoop.apache.org -In-Reply-To: -MIME-Version: 1.0 -Content-Type: multipart/alternative; boundary="0-1193839393-1243834249=:35091" -X-Virus-Checked: Checked by ClamAV on apache.org - ---0-1193839393-1243834249=:35091 -Content-Type: text/plain; charset=us-ascii - -Thanks a lot for your explanation, Sam. - -So is this event generated by hadoop framework? Is there any API in mapper to fire this event? Actually, I am thinking to implement a mapper that will emit some pairs, then fire this event to let the reducer works, the same mapper task then emit some other pairs and repeat. Do you think is this logic feasible by current API? - -Thanks, -Jianmin - - - - - -________________________________ -From: Sam Judgement -To: core-user@hadoop.apache.org -Sent: Monday, June 1, 2009 12:26:31 PM -Subject: Re: question about when shuffle/sort start working - -When a Mapper completes, MapCompletionEvents are generated. Reducers try to -fetch map outputs for a given map only on the receipt of such events. - -Sam - - -On 5/30/09 10:00 AM, "Jianmin Foo" wrote: - -> Hi, -> I am being confused by the protocol between mapper and reducer. When mapper -> emitting the (key,value) pair done, is there any signal the mapper send out to -> hadoop framework in protocol to indicate that map is done and the shuffle/sort -> can begin for reducer? If there is no this signal in protocol, when the -> framework begin the shuffle/sort? -> -> Thanks, -> Jianmin -> -> -> -> - - - ---0-1193839393-1243834249=:35091-- - - -From core-user-return-14702-apmail-hadoop-core-user-archive=hadoop.apache.org@hadoop.apache.org Mon Jun 01 06:04:30 2009 -Return-Path: -Delivered-To: apmail-hadoop-core-user-archive@www.apache.org -Received: (qmail 53387 invoked from network); 1 Jun 2009 06:04:29 -0000 -Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) - by minotaur.apache.org with SMTP; 1 Jun 2009 06:04:29 -0000 -Received: (qmail 39066 invoked by uid 500); 1 Jun 2009 06:04:39 -0000 -Delivered-To: apmail-hadoop-core-user-archive@hadoop.apache.org -Received: (qmail 38970 invoked by uid 500); 1 Jun 2009 06:04:39 -0000 -Mailing-List: contact core-user-help@hadoop.apache.org; run by ezmlm -Precedence: bulk -List-Help: -List-Unsubscribe: -List-Post: -List-Id: -Reply-To: core-user@hadoop.apache.org -Delivered-To: mailing list core-user@hadoop.apache.org -Received: (qmail 38955 invoked by uid 99); 1 Jun 2009 06:04:39 -0000 -Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) - by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 01 Jun 2009 06:04:39 +0000 -X-ASF-Spam-Status: No, hits=1.2 required=10.0 - tests=SPF_NEUTRAL -X-Spam-Check-By: apache.org -Received-SPF: neutral (athena.apache.org: local policy) -Received: from [216.145.54.172] (HELO mrout2.wahoo.com) (216.145.54.172) - by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 01 Jun 2009 06:04:28 +0000 -Received: from SNV-EXBH01.ds.corp.wahoo.com (snv-exbh01.ds.corp.wahoo.com [207.126.227.249]) - by mrout2.wahoo.com (8.13.6/8.13.6/y.out) with ESMTP id n5163FGq038852 - for ; Sun, 31 May 2009 23:03:15 -0700 (PDT) -DomainKey-Signature: a=rsa-sha1; s=serpent; d=wahoo-inc.com; c=nofws; q=dns; - h=received:user-agent:date:subject:from:to:message-id: - thread-topic:thread-index:in-reply-to:mime-version:content-type: - content-transfer-encoding:x-originalarrivaltime; - b=rChE4SCnwtWaZpjhovkiXDKfDiVNdRRvsadSGG9S9bgvOexn/9/5JjEQx1pOR7Nb -Received: from SNV-EXVS08.ds.corp.wahoo.com ([207.126.227.9]) by SNV-EXBH01.ds.corp.wahoo.com with Microsoft SMTPSVC(6.0.3790.3959); - Sun, 31 May 2009 23:03:15 -0700 -Received: from 10.66.92.213 ([10.66.92.213]) by SNV-EXVS08.ds.corp.wahoo.com ([207.126.227.58]) with Microsoft Exchange Server HTTP-DAV ; - Mon, 1 Jun 2009 06:03:15 +0000 -User-Agent: Microsoft-Entourage/12.17.0.090302 -Date: Mon, 01 Jun 2009 11:33:13 +0530 -Subject: Re: question about when shuffle/sort start working -From: Sam Judgement -To: -Message-ID: -Thread-Topic: question about when shuffle/sort start working -Thread-Index: AcnifqWrLG6N7GAk7kqy9QalVWfegQ== -In-Reply-To: <592088.35091.qm@web111010.mail.gq1.wahoo.com> -Mime-version: 1.0 -Content-type: text/plain; - charset="US-ASCII" -Content-transfer-encoding: 7bit -X-OriginalArrivalTime: 01 Jun 2009 06:03:15.0462 (UTC) FILETIME=[A7231260:01C9E27E] -X-Virus-Checked: Checked by ClamAV on apache.org - - -No you cannot raise this event yourself, this event is generated internally -by the framework. - -I am guessing that what you probably want is to have a chain of MapReduce -Jobs where the output of one is automatically fed as input to another. You -can look at these classes: JobControl and ChainMapper/ChainReducer. - -Sam - -On 6/1/09 11:00 AM, "Jianmin Foo" wrote: - -> Thanks a lot for your explanation, Sam. -> -> So is this event generated by hadoop framework? Is there any API in mapper to -> fire this event? Actually, I am thinking to implement a mapper that will emit -> some pairs, then fire this event to let the reducer works, the -> same mapper task then emit some other pairs and repeat. Do you -> think is this logic feasible by current API? -> -> Thanks, -> Jianmin -> -> -> -> -> -> ________________________________ -> From: Sam Judgement -> To: core-user@hadoop.apache.org -> Sent: Monday, June 1, 2009 12:26:31 PM -> Subject: Re: question about when shuffle/sort start working -> -> When a Mapper completes, MapCompletionEvents are generated. Reducers try to -> fetch map outputs for a given map only on the receipt of such events. -> -> Sam -> -> -> On 5/30/09 10:00 AM, "Jianmin Foo" wrote: -> ->> Hi, ->> I am being confused by the protocol between mapper and reducer. When mapper ->> emitting the (key,value) pair done, is there any signal the mapper send out ->> to ->> hadoop framework in protocol to indicate that map is done and the ->> shuffle/sort ->> can begin for reducer? If there is no this signal in protocol, when the ->> framework begin the shuffle/sort? ->> ->> Thanks, ->> Jianmin ->> ->> ->> ->> -> -> -> - - diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/email.eml b/solr/contrib/morphlines-core/src/test-files/test-documents/email.eml deleted file mode 100644 index d45f430869c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-documents/email.eml +++ /dev/null @@ -1,40 +0,0 @@ -MIME-Version: 1.0 -Received: by 10.216.199.5 with HTTP; Wed, 27 Nov 2013 12:01:23 -0800 -(PST) -Date: Wed, 27 Nov 2013 13:01:23 -0700 -Delivered-To: foo@cloudera.com -Message-ID: - -Subject: Test EML -From: Patrick Foo -To: Patrick Foo -Content-Type: multipart/alternative; -boundary=001a11c3815cb55dda04ec2e0f3b - ---001a11c3815cb55dda04ec2e0f3b -Content-Type: text/plain; charset=ISO-8859-1 - -This is a test - --- -Patrick Foo -Customer Operations Engineer - - - ---001a11c3815cb55dda04ec2e0f3b -Content-Type: text/html; charset=ISO-8859-1 -Content-Transfer-Encoding: quoted-printable - -
This is a test

-- -
Patrick Foo
Customer Operations -Engineer

= -

-
- - ---001a11c3815cb55dda04ec2e0f3b-- diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/rsstest.rss b/solr/contrib/morphlines-core/src/test-files/test-documents/rsstest.rss deleted file mode 100644 index 758f6a18363..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-documents/rsstest.rss +++ /dev/null @@ -1,36 +0,0 @@ - - - - - TestChannel - http://test.channel.com/ - Sample RSS File for Junit test - en-us - - - Home Page of Chris Mattmann - http://www-scf.usc.edu/~mattmann/ - Chris Mattmann's home page - - - Awesome Open Source Search Engine - http://www.nutch.org/ - Yup, that's what it is - - - diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120521-100919.avro b/solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120521-100919.avro deleted file mode 100644 index 36f01a2d48c22f18a91026005bbb741651d43f8e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3192 zcmb7`&x_MQ6vv;7$X>*Yig=J^yzF6DGHuf)J$vz_hn$6TC-5)Fqg8z=- z)sttBA|fIpvWSR?h=^z3WZQLKXVlqOXnE=Pz0bUv>6cy~-aB#fY`j&C{qQq z231{V#a*_)cJ%GE5@j?hGf|{fl*nmOAE^f{YcjPUwy{`~%OtKdS!nWEcU+#eA{A9q zW|QL;RdrnB8jrWOk<4?sEz09nHJdu$ssa1YG&qY8MdVzB^0*dh6zd!OcSbFC+=Qj5 zw}q%(&u%6ce=fy<4W-OQtWMH6xU7guG0AtMT1L%P(Qea#xKD6loSm(=w07bv>z^SYl@q7b%_rmi?n(V;{oD&mKK_di%!naeVIDgPXzX!^_Lu z^E`NQ1}@BRe)9le311gS@qMwPHmG^nq|JUc=_(Hcf8F!fIlsldz8Cg+cn2;u=eD8_ zdS%uMFivfYAvRr7JPt1@kE6ig@k9Ka$H9rGTB*=^EqFzFEfg5M7Ui|vc&e2Oo!5rf zl-EXq!D~}q+l{AMsnB^Hctd#|6d1e?<#pV6s+9_z=fhjd^HE^%e9H6Pc&e2Oo!5nT zl-EUp!Ru09*NvxIsnB^ncu#pf6d1f7<@MZns+9_z7r+O~3s7M20?G^Ac&e2OofpCZ z<%K9Pcp>G5ZamdWg~sC^d?Y-4VDj*x?k||S{N9^Zi#!gWD9`3EocarZu^FRCHS9$n#{-1yG-S_za{rPu) z^_#7~{q-Nd`-@-v>%SlR4KDoM3;yuakH7idfBoIR`SXAHzy0GMw*L6bzxwevTYvZK zKm77nKm74`KmEh>FTVRtIpG)d-9P6$zy0A?-~aTRAAYwn{&#QBU!J`2>#cwI)sMgV z{Xg!~AAb7&r~Ebk<9=;_^}}y}`@`RC{qY}nt9(-b{!{gf@BV+b$z8T?{oOC^OF91i zPg{Te%kMWo!9UOVY3tv5gFkNl>2J1f{mK2>{oVN1fBT!_7vKHm4?p~N>-YFduD)=; z`Q0yn-1__9{>xwf^usU9ZCUNTzKm75>Uv0I1XKVt#eXW0KA^v*n zuYdFXFaP-CZ-4nOe){S6f3R3v|MvIa|L)iMHOOoK9~b`lzlk6J)nENT5B~34AAb1T zKlu+XpZZt-@xDKO_W4i#NwXzC;@FWtiT~v9fA@cT6!E9||NW>L?TOl(RJ;uz6?KTyOdWD&O&y3jm{hzC9~E_sGfW+GW=$Q5I+|3x z4IdSCinC0ea%N4Ph&q{6ybT`}b&hjPopWYQoryY|RJ;uz6?KX8OkHwjOOI#poJ`&V?TDd9DmLKre=(5OqFiR z!FpHIO?gl3I>mLSN;fs+raXJQDmOKA++eD7Qx4XaIsnSg? zxhc=yuF6gA6c3py-IRm%uBMyvp4fGcM@*G&YROG`_I6cnYL|G-ROzN1tamltl=s%V zc3a{JQS*^-Q$DSzdA4>fZv%JJ5?vEdnJV3sgEiH3Q{MabQBm7?##HI1j@*=IZ&&4} zu8ZePm2S$xdRNm;c~9)x#|x%PH+AHuJbSw;H+4h2WU6#i4%WMxZpwRN*D+o(Rl2Do zH|5#eRk^8~;x$vHn{u$;)pS$d6T8mwhN;p`9l0sb-mc0`-4bt^D&3TW^{%Fy^4@ya zeoMR~YCaNf%BK}I&(^NxZQyQNqHp3oQ>B}7u%?=B%6s2FDry@am@3`WlbiDF?W)|= zckz*_(oH#7?`pa!?}=Ue_{3D{rk>oCXKz>KrhbUeOqFiR!FpHIO?gl3I>r~KN;mc7 zraXJQDmV30d}XS1Qx4X>z4B5^l<;)vkHAsO4?E>(Inbrb;*EU`;jMl=r@UY<`A^Hg+*px+w>1 zs_CY@C#YTQW~y}4KyJ#j_Ze1h8v5A7ROzN1tamltl=sB0L+oX$bkjg?%Conta?>!z zKBh`H889VcAa8BQ>B{*a#NnYU6q@LISw#Yx+w?iT}?OTJ+bQ&2bn6}G?1I} z?Cq-DG;WDQM9oLSP5HFiHP05cybarKT4HSCFjJ+Qa6snSh3Snq1ODesA0`#8o_>86p~lxJ^O<)(3n<4l!q%E5Y9(@l9#>^jB? zrb;)BB}7u-?^l zQ{G$eI&FzFM9oLSP5HE<=Goe{ybats?JZ~c;SLLQ@ipxxuZpy)WSJO>-PwYC!6{bozP2{FLd%G$(O-o#5s&rEh*1MW+ z%6scw=PhxKsQE~^DW6uIGgZ1N2kTu;H|0IC>lk;K zD%~`boAT`Ks@ybBahIvmO*vTaYPu=!iCyQo$5iR2ncS3TZ&&4}d5QZ>m2S$xdRNm; zd2hYzvLzl6H6IB#<82d4cQxIV_r$J!JYlMI(?V{_v$v~q(=xe&_bn%v{%1s!oclF$adt%o<-Z53Vso7Gy31@Ft;ihKG5bv3)+=Rh;SIGre@0$ zpP8!Mgu!}O&rP_u-ZiuBFGMXRH`nM*KCP&EwstLV19#IBnRkC>s&o@|;f>jaV@~Ya z#)2UDe+s0(lM?Cq-DlzI0~ zrb;(a7v7j%IOfEzW9(w8bQ5*qO`g47m76l}-py3$ChEc)vkS+Z*maIQOqFh;F1*RJ zx2tkf=G}XlD&0g~cw=_qm|O3f+4er7mXdq*{hCiJH|5#dwY&}7O*dt>y`QPlP1J=q zdG@HvO__HeV5U5(y>3|<)+NLPcc=xiMsH{?7}g({<>zieVVD#P1J=qdG>ZyZpys-3{$0>s0(k*E*x`W z*CEa_Rl13~@FvgRuI#3t`%COiHs9y?mm-Blu{bVzzZ4~`Y`M=fRSJt@@g~n6RS7Gb z?+Z+o!lGEbF=KJei7)mN7nv%BMX`93XKz;}tZcb25w+yqD`Dl+N?3WesO4?gc2ii{ za$jbu6c)weO`bif5>__fSC}e=MX`8e#^RV;e_gZXzRFZ7EQ-aOJbSw;VP*4ujj2*t z6pJ@zERH#`>k!wODuqR{c#~&uR~Ac}ws&o^@;!U1Cs&Z2{-}jj+-9)i?W5(i`6JOUp z9xzq9iDL04&)%-eP1$@uWU6!%#o~<_i(^jgI>sZWN;gp~-sIWaRkXQ4YRS7-Zpx>XoAPY! zTHXfkrkk?me$G_sCW^(IJbP5-rfj}nFjcyVV)4d|#W5#7!(F^&s&o^@;!U2tU6q@% z`F_Pz=_ZQB8#5NioY-}U*G!deqFB7iv$v~qQ#Rjkm@3^wv3O&~;+PY=PVtth(oGbL zH+lAURc^}W`yErInZm%YRS7-Zpx>XoAPW? z%iFN+rkk?m{>W76CW^(IJbP5-rfj}HF;%*WV)4d|#WA=3x@OD$nW@rE6pJ@`_I6cn z%I5nEQ>B|I7H`a09CKpVA-*zIx`|@(CePlk%1zmPZ~GtWKa|Z!B95V0yfI^O%!ys6 z*v?exCW^(IJbSw;H)Zp^gQ?O@6pJ@zERH#`>k>PeD&0h}c#~&uSLLQ`xpxt@y`Krfj|sFjcyVV)4d|#W5## zo#G%}o%H|5!)mbYQsO*duBeUz!vO%#hadG@HvP1$@OW2$r$#o~<_i(_v6bC$>?Cq-Dl+E`Erb;(aEZ&&0IOfEzL!4x)bQ8tmO`g47m7B8pKE+h&CW^%yGZx33 z*ma82OqFh;SiH%zx2tkfHs5EMD&0h}c>XuFO{#KJw%li#D&0h}c#~&uSK+2Mo9}Zq z6_3_G598^kn{alh-`<9=?1r1#Y`M=fRk;a+HPv$y?tS~HsBK(es&Z3Hu{h4&uEI@i zw%iw)s@#OZdRNa)xF>e);}TPqn_7y+arSlJyWHd zC>C$cSR8YL+Q$c`N;gp~-kPyE<^*+!k4%+rqFB5&V{yz0>KLDxD&0g~cx!gym=n|~ zJ~LIiiMsIC?7}f8sB?T_s&o@|;jP()V@^<)_{vo2ChEdlvkS-Enwr`6wqw|m;N6LsOO*@a_HP`lX4ROu$_ z!dtTo$DE+{v5TqFP1J?AW*3e*K^B}z3vbOX9CLy?#vZ0hH&GYfnq4^N1a*qN zOqFh;F1$6naLftn9Q&9m-9%k@Yj)w76VxU4GgZ2Yy71QQ!ZEj|X10BRsHNmyxhbDk zZpyRert&s$H{FzZ_d%vgH&GYfnq4^N#I9`|Vybi#b>Xeqg=0=oyEx2L=_cyJTeAzt zoS^n`gsIX^)P=WZ7mhhW9pWfcrJJY=Z_O?obAmd?F{Vm4Q5W8tT{z|hb&BImm2RRg zyfwRU%n9loCzvYTL|u4mcHx*4)Fnwtb4IrQ}|@DW6tu%CqIB z@-}cc-IRIvX{JgyQ5W8tT{z~%u5FxQs&o@|;jP()V@^=JILlP&ChEdlvkS+Zp!RW& zsnSiaf_)^SQLx5W-N|5L7n0@ zQ>Cyd7H`d19CLy?#~r3hVNopJnz1c|_YRS7-!pf(W zu<~sE9xiVKcT-r|eBWoP6c)wetr?4BPVCyo1ExwhQ7qn?u{h=gwTp*Lm2RR~yftHS z%n52AkC-amM6q~l#^RV0)FB=-Rl12{@z#vRF(;^FJYlMI6UE}K8H;01P^WmxROu#) z#alBL$DE+f@rB|I7H`d19CK@Gw%jj>TJr9doAPPpraW72 zDsKaK(@oiYzhtU(6UE}K8H;01?ApdFrb;(aEZ&;2IOYVki`Pt*ZlYMcHDhti32Gm2 zm@3^wv3P67;+PZEA>J}ox`|@(){Mn5C#Yk*W2$r$#p10Qi(^hur+Cj)=_ZQBTQe5N zoS@F}fvM6>6pOcJERH!rUE(8CrJE=gZ_QX7b1Q0>&G)C8ibv~M98WjhgtOzOZ*K#4 z!%bbb+@G1M+=RiJ>c!%?_wA#iw(*6j%1s@`;y8P|3O9Axa(`v2auWvYUA9V?Mzi}!eG6t7mMSb*maB@OjT~`C>F=r+f}%! z%a(g5Q#c}p_6>jRX<=)LynbAmd@5vEEvQ7qn>u{h=gb%~=)m2RR~yfb5Q%&n=}avvjV$-7r> z%BPi^@@%=OybataC>HO` zSR8YLI>!yBN;gp~-kGsD<^*+#n@p8%qFB5$V{y!_so8ShB5KLIS8mFum7DTxxv9Jj z+)X!S%YB=v(oGbLcV;Y(Ik9UScbF>OM6q~h#^RV0)GqEaRl12{@y?9JF(;^f++(VA z6UE}48H;01P=~nBROu#)#XB<=$DE*!@qnq)O%#iFW-N|5L7n0uQ>B|I7Vpeh9CLy? z$0MdnH&HC!nXx$L1a*nWOqFh;SiCc1am=l$*>XQ2YRS7-Zpx>XoAPYAsk{x`O*duB z{gkQFO%#iFW-N|5v1=R8m@3^wv3O_3;+PZEE}k=0x`|@(&Wyz|C#ZeAV5)Qz#p0bA zi(^huhj__U=_ZQBJ2Mu?oS=^JimB2~6pMFeERH!ro#Hi9rJE=g@61>nbAmd@8>UJ( zQ7qn>u{h=gb&0o3m2RR~yfb5Q%&n=}a=#;L$-7r>%BPi^@@%=Oybat!4CW^&7GZx33ppNl{snSgpi+5%$jyXY{;ww|7nrg4)Gyrb;(a7v7m&IOYVkk3CG4ZlW%{GrMri z3F;7gnJV2xU3h19;g}QDG4?T4x{12*&g{Z5C#X~GXR34)b>W@ag=0=o=QzMr`EwI> z;houqV@^<)ILK7#ChEdFvkS-Enwr`6A)=O&d*!BlTDd9DmYd4kz}<9HX4{9ED&0g~ zcxQIum=n9UafGSTP1J>VW*3e*LG9uwQ>B}z3-8P>9CL!&$1$c#H&GYfnO!*M1a*kx zOqFh;F1$0laLftn7$=x2-9%k@XLjM36VxeAGF7^Xy712I!Z9bPbDUzTbQ5*qo!NzB zPEeOP%~a_o>cTs-3&-4=n%VXlqLz|-<)(aExhc<oC+h>_7-9%k@XLjM3 z6T7x?j;Yd3)P;9u7mhhW?czLBrJJY=@60Y7bAsB(1*S?jQ5W8sT{z|hb%={hm2RRg zyfeFS%n9lkmzXNuL|u4icHx*4)G01ARl13~@XqYQF(;^VTw$to6LsO8*@a_HP?xyM zROu$_!aK7I$J~nAXWo6SrsC1MAI8&7H{tBK>D$}D-EdQ%+4gm&DmP)Urg~jC?tS~H zsBPR}s&Z3LT{zC(uEI@yX4^NJs@#OZdRMOt$33xYAGesQ+|<*5L>y;tSK+2Uv+dhV zRc^vyy{p%SUH6`SMU0Be~G=%=KCK1 zQk1YTSpQPISRD82UrPRU@;+Pc`%G2B!eCAHVsYGCQ?uoMK-7|V^AA&fKCP&Ew!YZq zZQyRIBU|o=OqIf-SiCo5amIE`Q7qn@u{h=gb&O|BmBOM}yfe4Nt=yDn%T48N;BLAp zTkh9Pm2RR~yf3IOYU(j1NqeZlYMcH)C~d_P33LiZn`O3?ypRh zZlYMcH)C+_CHmBDBG93d%bH*yPE%?r(*FwTkh>lm2RR~92dRMuyRwj+&h>m z-9)i?Z^q)76QAKBb~07EiDL2IjKwi0sAKG6s&o^@;=LJ*V@^<~*v(YwCW^&-GZx33 zpw6*}snSgpi}z+MjyXYHVlPvrnHO{SR8YLy2Np&N;gp~-kY&F=GN3~xla(aB|I z7Vphi9CL!Y#6_k`H&HC!o3S|N*3@jdFA=rm-77cc)5=YGw%k}qNUGr>F%iHid-gHy8+;^EO-9)i? zZ^q)7TYp`%<-W&M=_ZQBdovctoS=4bpQ+MK6pQy}ERH!r?c)JcrJE=g@6A{obAmd= zL#9eMQ7qn@u{h=gb&N+$m2RR~yfe4Nt=yDn%T48N;BLApTkhvfm2RR~yf3IOYU(j5kb`ZlYMcH)Cf~d_P33LiZn`O3?vG5BZlYMcH)CHO{SR8YLI>c9|N;gp~-kY&F z<^*+&Z71qKe=T|U`uFfizlY5rQ^n$aw%ps9D&0h}I4*kMIm%7ha_?ZObQ8tmy%~#R zPJD)!*vVAsCW^&-GZx3(iaKQTy{o3;(K;5#(@i(w?0km5y$#$AHx1cx?`EoU69#Ll z7mMTGw~vb2#vZ0BHw_evB}z3m?ob9CKpVHjXh>x{12*!R*2@C#YQV~Ag=0=o`#8Z==_cyJ z2eS*uoS+VIlBv>7)P)ab7mhhW9pe;JrJJY=AIvTsbAmd>X{JgyQ5Qa#T{z|hb&fMk zm2RRgd@#Fk%n9leXPGM9L|yn`cHx*?Q#0E>N7PbsuiTVRD>voYa#ML5xSMXuZ2LS@ zrJJY=AIvTsb7I#vE-+QPiMsH??7}f8s9juSs&o@|;e*+QV@^=}xWrWHChEcmvkS+Z zpbl}FsnSirf;z=jrb;(a7e1I>IOYU(j%!SnZlW%H zFuQQf3F;EpnJV2xUHD*j;h0-fGuysF)KYS<+>}o%H|5!KQ+XS>n{LW%`zBMRo2UyP z%q|>rV%IiqF;%)L`|f&I^PlNd7d~X(eVeJ$P1J?sqW2kAZpv)?4pXI@s0$y=E*x{> zGd#pyrb;(a7e1I>IOYU(jC)L#ZlW%HFuQQf3F;L0nJV2xUHD*j;g}QDIUX=ox{12* z!R*2@C#Xw2WU6!%b>V~Ag=20_&20M-QA^3aa#KF7zF+ffedm<7fxGFZ%(fphRl13~ z@WJfDF(-Cy;|Wuxo2UyP%q|>rg4)GXrb;(a7e1I>IOb~V&;2F#AzSWe{7aF-qF5Xk zy)U*BR<_*FnJR@vvG`!d;+U&H0r^+whitiDFjWeRVsTvbc2&a4mir}BrLZU#AIw-B zb7I#yUNKb)i(>J?jKwi0s7t(NsuUK*;)5BBV{T2&mirA+OWwT_Rz9tSm1pbgTHXfk zrm(W*e#=xTEQ-YkGZx33*tLy!OqIf-SbQ*Jam)#77w?%Wg+;OWV8-H@6VyIFFjcyV zV)4O@#W5$SLwsbabQ8tmgBgosPEg1A#8l}fip2*r7RQ{RPVt$k(oGbL4`wWmIYFJ{ z3sa?=C>9^gSR8YLy2MweN;gp~KA5pM=GN3~xwrk#)gQ_ZCGTFjDW6tu%CqIB@-}cc z-IOi&cBV=~c)*YY-SH{Fyi z_d%vgH&HA;n6WtK#I9`|Vybi##o~h*i(^huyEx2L=_ZQB2QwDOoS^n`gsIX^6pIgL zERH!r9pWfcrJE=gAIw-BbAmd?F{Vm4Q7k@~u{h=gb&BImm2RR~d@y5i%n9loCzvYT zM6vi_#^RV0)Fn%YBNdCGTFjDW6tu%CqIB@-}cc-IOi&X{Jgy zQ7k@~u{h?$u5FxQs&o^@;)5BBV@^=JILlP&CW^%eGZx33p!RW&snSgpiw|ZjjyXXc z;yhEOn9^gSR8YLI>s%gN;hTnUH=|7|6RC>#fNOZZ!=Z8iDGeF^uBYHo3iD;!&K=eip2*r z7RQ|U3@>q)snSgpiw|Zjj=2?e%;x)EO~s>iERLs}Uss%+&+xamfxF?RF-TSR7|> zSK+2HTkgk9Rc^vyy{i|CGrZHRY=S)>@!eG6t7mMTGde>~ZUl6tA-TZUYm`^Keo~>QW+rZtl zM7G>7nJV2xvG{1l;+PY=w(*Lo(oGbLk7g{6IYI5>HB+UVC>9^hSR8YL+Q%EFN;gp~ zKAN#O<^*+!w@j68qF8)1V{yz0>KN~sD&0h}_-MxBm=n|~-ZNFYiDL25jKwi0sB?T^ zs&o^@;-eXhV@^<)_{dc0CW^&JGZx3(nwl;5C!&_Td*!BlTDd9DmYd4kz}<9Hw%nhY zD&0h}_-MxBm=n9U@r9|B|I79Y)69CL!&$F`HzpTCYJ?_Pgh zd-}SXKR2mZe9V@6J5!~bC>F;>?{|}OQ?}eYm@3^wvG{1l;+PYk;VE`9Rl12{@zIRM zF(;^V>|&~P6UE}A8H;01P?y-vROu#)#YZz1$K0BlE%zRxmb`oQdpMs~ZpyRuyQ#bl z+)X!S%e|MW(oGbLk7g{6Ik9US`Krfk0VGgZ2YV)4<8#W5##?c)Gb z<tW*3e*L7n3q zQ>B}z3m?rc9CL!Y#CfJlH&GWpnq4^N*3`_lFA%kq+$%Tb)5=YGw%kXAgg=0=o$GFB+=_cyJN3#pZoS;r|ovG4I)P;{`7mhhWo#O^mrJJY= zAI&ZtbAr0WO{PjWQ5Qa%T{z~})XcVT5w(=uD>voS%1wE;+*IBM?xveE+rG_I=_cyJ zN3#pZoY=LEJ4}^sqAq+iyKu}2Y8Q8zD&0g~_-J7g>cU5}3&)(GPVt(l(oNKbk7gH+IYFJ{4O69?s0$y>E*x`$y2M+i zN;gp#KAK%P=4$HC{U!D>Tkd!CODTExN?7@{5>}oqVU@Sx7qcm>Y`NbvRSJt@@zIRM zF;{;A@-MNE*>ZnisuUK*;<)JTs)Us-_eZ8mVNonTnz17`3oS=^Jg{e|l6pN2$ERH!ro#HD~rLZU#AI(@CbAmd@w*RI6L)nsd zufJdG|3#M3jKwi0s7q{TsuUK*;-eXhV{T2&mU{Jl%8?&d%5M+uOk1aMP46 z_hF_gH({`*da*d}efy}WZ5&~$a??byIL_X#!c9}Q+(((J+=Rh;S1%UFJ+W&a$C#?z zG*K*$v$v~o)08duai%IaVX)rSi=J>#>^jB?rYbj06pQ2R?JC?fWy^h%sme_ltatTd zaoiKT&T)#V%1sl+;y8P|3O7yJa-U|ZauWvYUALam)#77w4HO-9)kYWX9r{6VyH~ zFjcyVV)4n0#W5$SLtJF4bQ8tmlNpO+PEg0V#8l}fip3{07RQ{RPH~y3(oGbLPi8ER zIYFJ{3R9(Ed0SR8YLy2MqcN;gp~KAEvN=GN3~xvvqmEkM6vi}#^RV0)IM%8Rk|so z@A~U%{@kQu@hO|{+f0>iqF5Xkz28mBP1$naVXAZ!#p070i(^iFhNrm8ROu#)#V0cs z$DE+fagV9eO%#hyW-N|5L0#fLQ>B|I7N5*m9CK@Gw%iYhTJr9doAPP(dpOV5@22uL za5vqQE%!sFN;gp~KAEvN=ESaTJYuSJ6UE|_8H;01P`h}{ROu#)#V0cs$DE+{@r0?; zO%#hyW-N|5K^@{LQ>B|I7N5*m9CLy?#xtf$H&HA;nXx$L1a*q%OqFh;SbQ>Lam)$o z950wE-9)kYWX9r{6VxSMGF7^XV)4n0#WA<0X3PDGs3q@SxhbDkZpyRert&s$H{Fyi z_iLs~H&HA;nXx$L#I9|;VXAZ!#p070i(^huyLih~=_ZQBCo>kuoS^pcj;Yd36pK%0 zERH!r9pXJxrJE=gpUhYsbAmd?2c}9lQ7k^0u{h=gb&8Knm2RR~d@^Hk%n9lopO`A$ zM6vi}#^RV0)FnPMRl12{@yU$EF}J2>%l(C@CGTFjDW6tu%CqIB@-}cc-IOi&SEfog zQ7k^0u{h?$u5D~PRsH$vRPyfiuJ!*S3#PnXm7B8V-p*9%CW^%;GZx33*tL%xOqFh; zSbQ>Lam)$o5IdPF-9)kYWX9r{6Vx$wF;%*WV)4n0#W5$SQ|xA{bQ8tmlNpO+PEhCA z!&K=eip3{07RQ{RF0q%X(oGbLPi8ERxivLg?tMfpdH3r3HJ?^)%Cq(TTHXfkrkk?m z-p^F&CW^%;GZx33*tLxVOqD-3Q7k^0u{h=gwTpvHm2RR~d@^Hk%n52AhnOneM6vi} z#^RV0)FBQtRl12{@yU$EF(;^F9AT<-6UE|_8H;01P^UP`ROu#)#V0cs$DE+fag3?b zO%#hyW-N|5L0#fFQ>B|I7N5*m9CK@Gw%jL(TJr9doAPPpraW72DsKaK(@oiOpJb|Z z6UE|_8H;01?ApdDrb;(aEIygBIOYVki_=V%ZlYLxGGlSf32Gl_m@3^wvG`=h;+PZE zAkuoS@EefvM6>6pK%0 zERH!rUE(5BrJE=gpUhYsb8Bj5+n0!1O74}L@@eI!JX>xmZv%JJO_^<9W~y`(b>WlQ zg=0?a+Qt>8N;gp#KABxO<^;8it4x({qAq+gyKu}2Y9H5_D&0g~_+)nBm=n|?t}|7- ziMsH~?7}f8sAJqiqAna4JxeGzWww2XsHNmyxhbDkyXM)Vmbc+^yy>RQw(l}kx{12*$?U>0 zxBj|jwtbJO(oNKbPi7a6IYI5>K2xQes0*LWE*x`$+Q$Q?N;gp#KABxO<^*+!hfI}j zqAq+gyKu}2>KKohD&0g~_+)nBm=n|~9y3+CiMsH~?7}f8sB=7Fs&o@|;gi{gV@^<) zc*<1iChEc`vkS-EiaKZB{j8?q(Yg!A(@i(w?6~RM+rZs$)12A%bEYacVX&rpT{!N2 z`>3dGykM$w(@b4B&fc!VO><`3FPW;`gu!}OuM5XLv1=c%n5x_~Qx}f2x2tf|oZ0qk zrYbjKu-?_{n{ZF;I>sBODmTs4h2!k)D%><@w*8i=%1s!oclEk(+!MRb@s6p=O*3`j zID5MaH_e%Czh|m)69(&Dy`P(KZ@p_~+aHKpN^btSY0jq=HP6B}z3!lv{9CLy? z#8;+DH&GWpn_W2OYUo@Wd|Lm;Hie~P@i|-W?M#)zqF5Xk zy>BNatZcb=FjWeRV)5CG#W7cZ+Vii@&)IVCWU3Sv#p1Z=?W%;8E%z>>mb`lF%iFN+rm(W*-py1gEQ-ZvGZx3(`ZJs@_a3H7VNonTo3S|N1htF3OqIf-SbR2P zam)#7AN!apg+;OWoM-Rrs)Uuz_kN~IVNonTo3S|N#I9o;V5)Qz#p1IWi(^hur#Q$| z=_ZQBXEPSZoS@Eeh^f*|6pPPhERH!rUE(lPrJE=gpUqetb8Bk0+((F7^6r(J@@eI! zJX^nq%iF-+bW^t6N0}KnB2%TCC>Ed1SR8YL+Q%iP zN;gp~KAW*P<^*+!%S@GSqF8)3V{yz0>KIp;D&0h}_-w}Fm=n|~t}<1+iDL2DjKwi0 zsB>Ins&o^@;Y2D=WMxeGgZ2YVsTvbKEujQ*>c}u zs&o^@;em2RR~d^Tfo%&n=}az7$!$-7r>%BR)$Yo4v|obons zH{Fyi_hY6?H&HA;o3S|N#I9{TVXAZ!#p1IWi(^huyLie}=_ZQBXEPSZoS^pcjH%L1 z6pPPhERH!r9pX7trJE=gpUqetbAmd?3#LjpQ7k^2u{h=gb&8ivm2RR~d^Tfo%n9lo zub3*`M6vj6#^RV0)FoasRl12{@!5>UF}J2>%l(F^CGTFjDW6tu%CqIB@-}cc-IOi& zTc%1kQ7k^2u{h?$u5G+ys&o^@;|m;N6UE}Q8H;01P`lX4ROu#)#b+}X$DE+{v5TqFO%#jIW-N|5 zK^B|I7N5;n9CLy?#vZ0hH&HA;o3S|N1a*qNOqFh;SbR2Pam)$o9Q&9m-9)kY zoM-R*RkEd1SR8YLI>s@kN;gp~KAW*P<^*+$<4l!qqF8)3V{yz0>KrGSD&0h}_-w}F zm=n|`PBK-xiDL2DjKwjxqAuBdpQ@>NwC>jMbkj{ZJ8t^+HgGrGv}DVDnyJc77_6yY zERK8MJ}PP(XPBzov`{RLv$v~o(~>RsS*9vCVX)rS`*-2ECwA@Q98;B>7K+7j_I4F+ zTC(Ln&s60m4A#4Pu{iFDUB|el;g}P<&T)sS z(oNKbFJ>2xIYC|GE>oqOs0&}rE*x`fYG&K_h+0bSm7DTu<)%DaU)S*@a_HP`h}@ROu$_!WXj($DE+{@rbF?P1J=iW*3e* zK^@{TQ>B}z3t!AG9CLy?#uKJWH&GY9m|ZyL1a*q1OqFh;E_^Y&aLftn9M704-9%mZ zVs_z}6VxT1GgZ2Yy70y9!ZEj|X14u;sHNmyxhbDkZpyRert&s$H{F!k_DiNpH&GY9 zm|ZyL#I9|;Vybi#b>WNIg=0=oyLio1=_cyJ7qbh;oS^pchN;p`)P*l*7mhhW9pWuh zrJJY=U(7BXbAmd?JElrEQ5U|LT{z|hb&B^)m2RRgd@;Ll%n9loADAlLL|yn|cHx*4 z)FnPLRl13~@Wt%HF}J2>w*85yrQ}|@DW6tu%CqIB@-}cc-IUq(XQoOwQ5U|LT{z~% zu5El_s&o@|;fvXYV@^=J_{vo2ChEc$vkS+Zp!Tutf2;odbtxtH`s-T%kFQ|L`?@MO zWwyPYsnSicSVZ3&-4=n%VXqqLz|-^?NvCyd7GKO*9CKpVF%B_R3X5X##f-%}qA-^1l? z;BE>lTkhjbmBOM}d@*Bj%!yswIKfouCW^%uGZx33pmuSRsnSgpi!Wv@jyXZ?;}lb+ zn~d_P33LiZn`O3 z?u$&7ZlYLxF=KJeiCx>c#8l}fip3W*7RQ{Rc5#`h(oGbLFJ>%`IYI5?3R9(CGL zSR8YLI>c3`N;gp~zL>E%<^*+&YfP1HqF8(}V{yz0>J-i zqF8(}V{y!hUE8?BROu#)#TPRc$DE*cahIvmO%#hSW-N|5LG9xnQ>B|I7GKO*9CLy? z#C@hpH&HCUn6WtK1a*uDOqFh;SbQ;Kam)$o6c3py-9)kYV#eZ_6Vy2#F;%*WV)4a{ z#W5$SOFU+(bQ8tmiy4b!ZcWXW`w3A?-o0{DKCRr8XUk3HZQyRYDO>KROqFh;SbQ;K zambAmd=OQuRU zQ7pcgu{h=gb&OX`m2RR~d@*Bj%n9lgubC>{M6vi{#^RV0)H&WTRl12{@x_eAF(;@? zyk)9%6UE|-8H;0%sm+#bzTZ{U@@N%{m!}(UDrdt@fA=Ww74Wip9%4v1=cnm@3?qx2r$AD`#)l zwVU$t5TBVU+*AhZU9DKW+!MQw@r9|vO?kWe5A~L_x9i$Xd3lPjOcib_gY~XfEMD%3 zUFX<#X7dkab56LaJguo#EdF^g`4tr6^oa%_Ze1h%9eWvQA^%UKQ}d7 z@@cheo-Jy58}FJe_fDotH`TFt{whp2iC8=@x3P<<(oJ1hO?51u zRMSl&7SGFl>|v^OQyq(!i{7rvP1$nqWvX;j9g8Q`bd!k1^YR$`m@3^=$KvIpx2tkf zw%q%fD&17a;z>2#Bx3QrJjVg1N;lQ9c)959y8gK-FE4SBsnSh#EM6{pROO~@xepPw zXGGZ&&4}Y`IS|Rl2E;#gl5fNyOrLd5%*|m2RqI@p94IRkvoY+O@omEs-tvS*A)i)vus{>R7y7^ghGNP1$l^W~y{k9g8Q`bd!k1^YR>5m@3^=$KvIpx2tkf zw%k{lD&17a;z>2#Bx3PqOSatCh+6XQm7DTu<)%DayOy`{cTTq4*O@BaRLA1^%QgR* zPQ>DQxs4l4m2RqI@p94IRsAzvw%j+FD&17a;z>1sZW6J0Uhd--Q>B}7u>QK5ZYt^d z>gzhhZKg^$)vA<)&=8?=n@osgA{yYPw0p;(2+F zdrXyXs$=nT(c4wIDO>LQOqFh`WAUV#ZW6J0vn5;Z2ShD-_sUKAwE8`qXKUB;HvT=F zE%!sFN;lQ9c>Z!tH?3mvf70aTHXbomx~cBM%SCTj<)+NGA2U_DsqVs)YPv~u;d!}_ zCrp)Ys=M%V(c4x1GhJrePnjy+RCnP?HQgk-@Vq?6Gp0&6)m?bG=bFKq6=@fWVZc^sHNmyxhbDk zZpyQ@Yk3>HDYNa@OqFh`yYT$wnr;$ZcwTPf4O69?>Mp!o^mbKl%53{BQ>B~gE8831PpauA(S_&bF+MO=x~cBM%SCTj<)+NG zKQdLisqVs)YPv~u;dyzEPfV3=s=M%V(c4wIDYNa*OqFh`yYQr%ZW3L1vn8|bFGMXR z_sUKAv~p9PtzFC8*iD&je`Tt4Q{9E z|C^u8wzo4?x~cBM^KGV^L>Hcy``E!$>8831FBiS9t8!Cj+dG*m-BfqsNj2Rhy70U_ z#xABxH`QHux#;bx+?3h&Zl+2%)m?Z}O*e@yJTK3&hpEy{br)VPdb=t&WwyPSsnSh# z7oJqpO`;3W|Bc}-v5%;wfZ{zRR%(nM4Rl2F}!tpz{bW`1hC)ND-O`;3W%R?My zs&rG`g_n!ou6WyO*D;PTRl2F}!jo$K2j|M1piXg=snSh#7oKl3-L&e$f95Z-=NsoZ z#=jIPtU4Ai7rjrr5>~d{$C)aHRmb8xJTJF#im6gqbu3;kdb_HB(94$lG*hLp>R3Ff<_}XM7SGFl zoMEaIRvn9%i{7s4hpBA2&oWgCtB%E!YW^@KV)48@#yO@+Vb!sCx#;bxewfOZ`#e*n zudFRDmUf7IVGy+CfpO$F>Wwbxhemh2vNNs zrf^SCr?|;f<)-|n9in<}!aYHq;}%nuoAO5%qI$77?g{D=x0$Nklrkc!=O)}+Q?up1 zL)4OY^UqEBXJ0in&z76Y+rZtlM7G>_nJV2xv3O&~;+PY=wsDWC(oGbLH+lAURc^|b z`#w{pnXQ*s&o^@;*A-LV@~Wk#v`UmH&HC! zc>iVb{LQ4?lr8sTrb;(aEZ&&0IOfEzb39?HbQ8tmO`g54t8!B|;7^$<-9)i?lV^{r z+>|Z%GoqHfd*!BlTKz1MXNy|ihR^Y)o3izd8?3#LjpQ7qn= zu{h=gwTqWbm2RR~yfI^O%n52Aub3*`M6r02XYcE(+>|Z%YoZyZpxN> zCsU=HC>C$bSR8ZfU9;uhMbwgaufDGNv~p9PtzFC8z}<9Hw%og!D&0h}cw@%mm=n9U zv4^SBO%#hadG>ZyZpxN>FH@zPC>C$bSR8X=*FN?!Rl12{@g~pSuF6f>a_?uVbQ8tm zjTwt$PV73y0jA2In|Z%L8eMKQ7qn=u{h?$u5%n>s&o^@;!U2tU6q@% zfZv%JJP1$lEWvX-&#o~<_i(^jg z+Qu=aN;gp~-gy5J@%)z|%1zmFA7`p`6UE|<8H;01?Apf(rb;(aEZ%tk5%JtrxhY%j zlT4LvqFB5!V{y!hUB@`ZROu#)#hW~PzlW8ZvgJO_ROu#)#Tzpg$DG)8jx$V^ZlYMc z$+Nesa#Ob4XPGM9M6q~d#^RV;@0u<5Iii-ld*!BlTDd9D)~@Ak;BLApTki8rm2RR~ zyfI^O%!yswxWH8DCW^(IJbSw;H)YFxk*U&66pJ@zERH#`Yaf@GD&0h}c#~&uSLLQ` zxi2$Sx`|@(#*D==Cw3j<3R9(C$>?Cq-Dlr8sFrb;(aEZ&&0IOfEzb6jJpbQ8tm zO`g47m7B8VzRpzXCW^%yGZx3(de>~ZZxFTQ-77cc)5=YGwstLV19#I+*>c}xs&o^@ z;*A-LV@~Ya#x15wH)R}N?`pcKr01($ySU9%=_ZQB8#5NioY=LGJ4}^sqFB7iv-fpX zZpxPXE>oqOC>C$bSR8X=*D>xfRl12{@g~pSuF6f>a^GjFbQ8tmjTwt$PV73z1Exwh zQ7qo%+1pjQDO>J`OqFh;SiCV~am=lE&6fKSQA^&va#KF7zF+ff?ONUj?xvfv`F_k) z=_ZQB8#5NioY=LECrp)YqFB5!V{yz0Y8OwLD&0h}c#~)EGpyW{E%!5~N;gp~-k7mC z=ESZ;JZGwO6UE|9p1oa_o3iWTKg=0?a+Q$c`N;gp#-sIWa zRkcX2md%G$(Ww!m9snSigEiIb!g250M@4O82UC@sTI#}a_I4F+YBSs3$yDVg4A#4PKR4l?*tL&c zOjT}bsSC&1+f}%!&1`!&Q}mRk^99E*xiXSK+2Mv+cc1Rc^vy zy{p%Sa~}+x860g?E^$DB{%=v)aKKQ znrCa*@-}ccEs@#wL8eMKQ5W8tT{z~%u5BD*s&o@|;jP()V@^=JILuV(ChEdlvkS+Z zp!RWusnSicU&I3&&hd{kgxy z-e&WCj(;gqSQLxnqW8sC!pfHWJX58xC>C$cSR8ZpCm{dpf^9b67nmxAMX@+8db=uN zWy^h$sZv-Ji??PhjybXG9G93Xg+;M=YsTW36VxRxGgS(UV)53D#WA<0X3Kqrs3q@S z2`isg!pgJtbuDiLcT-r|a$jYt6c)wetr?4BPVCyoHKs~oQ7qn?u{h=gwTtUamBOM} zyftHS%n52AH<&8jM6q~l#^RV0)FEy%Rl12{@z#vRF(;^F++wP96aAOPTQe5NoS;r| zo2k-G6pOcJERH!ro#PHurJE=gZ_QX7bAr0WU8YJmQ7qn?u{h?|)NHx$5w+yqD>voS z%1wE;+*IBM?xvfv<-X5U=_ZQBTQe5NoY=LE2TYZ2qFB5&V{yz0Y8MZgD&0h}cx%Su zm=n}K9x+wAiDL2AjKwi0s6#wvs&o^@;;k8rV@^=Vc*0caCW^&dGZx33pic3WsnSgp zi??PhjyXY{;~7(>n4&C>C$cSR8YLy2MAON;gp~-kPyE=GN3~xjzxLTvJ{KAW-N|5K^B|I7H`d19CLy?#tx=RH&HC!nz1Ys3q@SxhbDkZpyRert&s$ zH{Fyi_g~d_P33LiZn`O3 z?&D0AZlYMcHDhtiiCx<`!Bpucip5(q7RQ{Rc5#xa(oGbLw`MGkIYI5?6jP;}C>C$c zSR8YLI>c$FN;gp~-kPyE<^*+&Gfb6kqFB5&V{yz0>J(?0D&0h}cx%Sum=n}F&M{TG ziDL2AjKwi0s7st@s&o^@;;k8rV{T2&miq!xOWwV5Q$DTSlxNFLB|I7H`d19CLy? z#8swBH&HC!nz1XQ2YRS9#=cX>7R@6LOyOy_syJ?AR zxt}ssx`|@(&Wyz|Cw6V)8B?X3C>HO`SR8YL+QoCGN;gp~-kGsD<^;8m7fh9IqFB5$ zV{yz0>JTrPD&0h}cxT4qm=n}7UNKd=iDL22jKwi0s8hUVs&o^@;++|bV@^=#c*9ib zCW^&7GZx33pf2&2snSgpi+5%$j=428Tkdy6EqV9KP5HEPQ=Tn1mA8Sr>85PC-!oOZ ziDL22jKwi0c5UMWQ>B|I7Vpeh9CL!&#Yd(}H&HC!nXx$L1htP(OqFh;F1$0laLftn z5TBVU-9%k@XLjM36Vx%jFjcyVy712I!Z9bPQ+#ErbQ5*qo!NzBPEhCA_J3A?{@Rt2 zd;R@d|BtU=%KLs*Zpv(XJ5!~bs0;7RE*x{~T{GL>LDW)muXfF+m7DTx?ONUj?xveE z+uq4k=_cyJJF^SNoY=LET}+j3qAt8MyKu}2Y8ShiD&0g~cxQIum=n}K_AphtiMsI4 z?7}f8s6*^!s&o@|;houqV@^=V*vC}qChEdFvkS+ZpiZ%$snSiV zW*3e*L0#fBQ>B}z3-8P>9CK@GX4_|oT1xJfoAPPpraW72DsKaK(@mLepJl3a6LsO8 z*@a_H?ApdTrb;(a7v7m&IOYVki}OsCZlW%{GrMri32Gk~m@3^wU3h19;g}QDAucji zx{12*&g{Z5C#YjwVybi#b>W@ag=0=or?|{i=_cyJJF^SNoS@Eeg{jg_)P;9u7mhhW zUE(TJrJJY=@60Y7b2atn{t|naE%!D0rIfsTC9Hf}2`kT*u*%!;i`f)bw%pg5DuqR{ zcxT4qn5#bl`Ip$cY`JeRRSJt@aa{CvRl>@a`zBMRuqYPq%vcPn6c)weof(T`PEg0V!&E6Oip4uK7RQ{RPH~s1Qdks=cV;Y(IYFJ{9#f^T zC>HO`SR8YLy2O2^N?}nf-kGsD=GN3~xgQX<EmNhNC>HO`SR8YL+Q&PlN;gp~-kGsD<^*+!_e_;; zqFB5$V{yz0>KGrGD&0h}cxT4qm=n|~J~CChiDL22jKwi0sB?T`s&o^@;++|bV@^<) z_{>!4CW^&7GZx3(nwl;57owKDd*!BlTDd9DmYd4kz}<9Hw%lKtD&0h}cxT4qm=n9U zvF&{Qhq5K_Uhi7}FS2xIERH!r?P5DqrJE=g@61>nbAsB(4yH;sQ7qn>u{h=gb%>oz zm2RR~yfb5Q%n9lkyO=87M6q~h#^RV0)G2l|Rl12{@y?9JF(;^V>|v^O6UE}48H;01 zP?y-tROu#)#XB<=$J~nAXY;+UrsB~$7RS>~H{tBK>D$}D-EdQ%E%$z=DmP)Urh2hB z?tS~HsBIizs`|O9r&t_kZ&%@_K3ndCOjT~eV7;psi{qZywU0wgRc`7j7RTA!Rk*3o zmisVMm76eF@9M?kxF>cU;|Nogn|g}HarSlyFygY~XnERK8YU9;srLDZ6W^Uo4}KCP&EwstLV19#IB*>ay` zs&o^@;=LJ*V@~Ya#wn&sH&HC!o3S|N1htFPOqFh;SiCo5am)#7A7_{<-9)i?Z^q)7 z6VxHjGF7^XV)5RL#W5$SW1M5EbQ8tmy%~#RPEe;f&s6Cqip6^~7RQ{R&T)aM(oGbL z_hu}PIYC|GB2%TCC>HO{SR8X}YPQ^$h+6XQm7DTu<)%DaZYpmBchgPTa$jbubQ8tm zy%~#RPVCyo6{bozQ7qn@u{h=gwTr7vm2RR~yf>G+ zP3k`)-e=2wo2k-G6pQ1cX9?w|Y`O0cwdCC^H|5i6*F0O)@-}>qH{Fyi_g$t+H&HC! zo3S|N)?e3bx$iMmx`|@(-i*aDC#YTAXR34)#p1mgi(^hu`*^@q=_ZQBdovctoS+Wz zkg3v56pQy}ERH!r9pe#GrJE=g@6A{obAmd>W2QvoYa#ML5xSMXumiswVrJE=g@6A{o zb7I#vUNBX3IOYU(j(1FzZlYMcH)C znJV2xv3PIB;+R`gv*rFk)RK3v+>}o%H|5!KQ+XS>n{LXM`y*4OnoC+q;=6-9%k@ zZ+79B6T7ythpEy{)P?tE7mhhW?P4!erJJY=@69e8bAsB(KBh`HQ5W8uT{z|hb%^~; zm2RRgyf?dW%n9lk2bd~Y3)g=20_&20M!QA^3aa#KF7+>~d_P33LiZn`P6?W0VUZlW%{H@k4m ziCx<`##HGh>cV@o3&)(Gc5$4k(oNKb_huK4IYI5?1XHD(s0;7SE*x`$I>bq)N;gp# z-kV)G<^*+&Q%sd^qAt8QyKu}2>J+D$D&0g~cyD&$m=n}F&M;NFiMsIK?7}f8s7st> zs&o@|;l0^~V{T2&Z2KHhOUb=*Q$DTSlxNFLB}z3-8S?9CLy?#AT*RH&GYfn_W2O z1a*unOqFh;F1$CpaLftn6jzxl-9%k@Z+79B6Vy4bF;%*Wy71oY!Z9bPOI&BFbQ5*q zz1f9hZcWW>`vy@<$-Qz@KCRr8XUk3HZQyRYDYNaHOqFh;F1$CpaLkEa+qlJ4>89+v z>s`(EQgz{dX4|)!D&0g~I4*jhVdbXGw(l@ix{12*-t59LCqBbN+-0hC6LsOe*@a`S zrvBVtV(+u%zQ?~5DJ+V`anbvuqYPC zMQ>LntZca-GF1wTV)5RL#WAT z4AxXH7RSAB9~HHYCrnks8YmXW+1pk8Fg0Y${gkOnSQxB#^@N3cV%I*NF;xj`pjaGd zZ&x9#AzSX}OjW|dV7;psi{qZyb&MBGRc;z67RTA!Rk&%$mir}Bm76eF@9O;va@-TU z&hd(=%1r~s;y8P|3O5bea=&J(auWvYUA^cD_tv{+%l(F^CGX}RriOf4QS)r=THXfk zrX{lFe#=zpCW^%eGZx33*tLy!OqFh;SbQ*Jam)#77w?%W-9)kYV8-H@6VyIFFjcyV zV)4O@#W5$SLwsbabQ8tmgBgosPEg1A#8l}fip2*r7RQ{RPVt$k(oGbL4`wWmIYFJ{ z3sa?=C>9^gSR8YLy2MweN;gp~KA5pM=GN3~xwl=Y{!n%(dH2dq`LuFVo-H?(w}HFq zrfj*lGgZ2YV)4O@#W5##ZDR*hrJE=gAIw-BbAsB%PNqsXQ7k@~u{h=gwU1p)m2RR~ zd@y5i%n9lcyO}E8M6vi_#^RV0)G_ujRl12{@xhG6F(;@~>}9HS6UE|#8H;01Q0Lgk zROu#)#Rv1hERH!rU1C2|rJE=gAIw-Bb8Bk0+y{tS^6r(J@@eI!JX>xmZv%JJP1$lE zWU6!%#o~h*i(^jg+QuQKN;gp~KA5pM<^;8i!%UTKqF8(|V{yz0Y9B|KD&0h}_+ZB3 zm=n|?jxtrciDL1=jKwi0sAC*ss&o^@;)5BBV@^<~IL=h*CW^%eGZx33pw4lEsnSgp ziw|ZjjyXYH;v`e0nay|s&o^@ z;)5BBV@~Ya#u=tcH&HA;n6WtK1htE^OqFh;SbQ*Jam)#7ALp1V-9)kYV8-H@6VxHj zGgZ2YV)4O@#W5$SV_aaWbQ8tmgBgosPEe<~$W-Yjip2*r7RQ{R&T)yU(oGbL4`wWm zIYC|GGE=3SC>9^gSR8X}YPQ^0h+6XQm7DTu<)%DaZYpmBchgPTa$jYtbQ8tmgBgos zPVCyoHKs~8Q7k@~u{h=gwTtUam2RR~d@y5i%n52AH<&8jM6vi_#^RV0)FEy%Rl12{ z@xhG6F(;^F++wP9Q%2wQ?_u-bg{xS6$maVtQ>B|I7RN>JJ4d-GTkbndm2RR~d@y5i z%!$wN5_g#@-9)kYV8-H@TT`>;zDLxOcdy)(PphwMo~_Stc^kN!ZpxPXK2xQeC>9^g zSR8X=*ESw7Rl12{@xhG6F(;^9JY=eL6UE|#8H;01Q2ThqROu#)#RoGM$DE)J@tCR7 zO%#g{W-N|5K^@}>Q>B|I79Y%59CLy?#Z#tAH&HA;n6WtK1a*#QOqFh;SbQ*Jam)$o z63>|`-9)kYV8-H@TT`>;enHfdcdy)(Pb)X&*>Y2P8@QWp%9i^jQ>B|I79Y%59CKpV zHeNARx`|@(!HmT*C#YS#W~y`(#o~h*i(^hu`*_1t=_ZQB2QwDOoS+WzmZ{QB6pIgL zERH!r9pfESrJE=gAIw-BbAmd>d!|Y^Q7k@~u{h=gb&d~Am2RR~d@y5i%n9leADJrM zM6vi_#^RV;Q?up%MAVXZuiTVRD>voYa#ML5xSMXumisePrJE=gAIw-Bb7I#vzA#m~ ziDL1=jKwi0s9k(zs&o^@;)5BBV@^=}_8{3a1Ns=T< zk|arzB-^&#Zb_0PNs=T-5|b3GfQbBx0^=X5mJng?s|f6hOD z9eMZi^SYtWtJycHSbUo;_dcddH&HB(2feQ*<)&=8_cK+xiDL1s8H-~lKEgE)FjcyV zV)3mRi(@CKEeX3Kq;s3Y%Qd=2N_$xV4} zeKn1@fxGLbY`KpxRl12{@vRw)V<*nKila=GZlYLxo7di1m7B8pKE_n(CW^(kW-N}K zIO`^kGgW@yM6vkRjK#4N)NPz#s&o^@;#)Ho$4*d7oMftW6LsNRvkS*gP-~oGs&o@| z;ajr{$4*dNoMx(Y6LsNRvkS*gPz6=ay&9Z2LM< zN6F3KHv$WuyOzjo`vy~`o2UyfW*3f~IO{5IGF7^Xy6|Fl;n)f4I&Lvl zx{12*Vs_!!3F;2xojB_%9y3+CiMsG&cH!6w>N=hNcJ+Rl13~@M3o1*a>Qh=S-DuqAt9cT{w1vTH^&%rJJY= zFJ>2xouIaO$yDhk>cWfJg<~hEJzgB}z3om9Dj@^9LfAxphOSar^`IjPvMX@*@^gaMeSlM#FW2zJu#p1<` z#j%?|0Qra5OSatanJR@vu{a*|&Z>l!E%yheN?}nfUd&h=J8{+$ADJqJMX`7>V{z;R zwZYT-UjZju(IX;##AXRip7f=i(@Cwx{B{im2RR~yqK{#c7nQ&A54{QqFB6` zu{d^ux{04mm2RR~yqK{#c7nQ%Urd#5qFB6`u{d^uTH-fTrJE=gFJ>%`ouJnE!&K=e zip7f=i(@CKE%x4+ekfZ;-o5<&I{zP8u=0MtDmP`zy^pEVO%#h4GZx2gdDd*X_Y-yG z-HWs4-N{XPZJl+z4cuKfWy^hlsnSgpix)E%$4;De6$hCr-9)i?F=KJ;1a%#Um@3^w zv3N0KaqI+j6Ni~9-9)i?F=KJ;1a%unm@3^wv3N0KaqI-O#8IY7H&HBJ^4j|vR&L7X z`xsNDnB z-ksc(*VgBCybau4H)YFximB2~6pI%#7ROGUbrq+XD&0h}crjyf>;!ckXP7G8M6q}= zV{z;RbrWZqD&0h}crjyf>;!ci=a?$pM6q}=V{z;RwZwU*N;gp~Ud&h=J3*~+fvM6> z6pI%#7ROFdTU=zSbQ8tm#f-(V6Vx7;m@3^wv3N0KaqO1VY`HHJb>!U(H|5>QO?hp( zX}k^GT{mUReTAvgO%#h4GZx2AoOKmfnJV2xv3N0KaqI+j9oLvD-9)i?F=KJ;1a%YF znJV2xv3N0KaqI+j8#kCL-9)i?F=KJ;1hvFXrb;(aEMCl596Ld+af_+aO%#h4GZx2A zP+Qz)s&o^@;>C=`u@lrDcbF>OM6q}=V{z=3)NHx$5_RO=3peH6$xV4}xoNx&++8;!ca51A_6l+pM6 zc{TrAxQfL~Hs6n!D&0h}I3D!Anv|Qe<$lam=_ZQBiy4b!CqBY8o-kFqiDL0$#^Trs zYKx~#m2RR~yqK{#c7odD8B?X3C>AefERNlhnl1NpqK>?K;ikMheGTWe_0=@q2JWt# zvgLlkROu#)#fuq>V<*nKikD24ZlYMcn6Wr^g1U}ZOqFh;SiG3AICg@%iPub(ZlYMc zn6Wr^g1U`2OqFh;SiG3AICg?s;w@99nV<)IB-ZNFY ziDL0$#^TrsYL5?0m2RR~yqK{#b_;6F=KJGJ#jRy5j=Q^V!nNb3zi$I~$4xa`?oUir zZo+V;da*d3`}@yCUBzdnDmPV%#c}PORk*2U%l(C^%1s!~XZ2!nJSWb&iLXpmZmJZE zV4mY=fqh{d}pe1Q>9oO*WOu$n`*Y)KbWfAgyDQvFBZph z;;b!xGF7>$QY?;Z@2tX2HCygqOjT~ea6YRSi{rWFS+nK-P1KQh^Y=|P?@rXbw$3`< z2JWsUvgQ86ROu#)#j6>MV<*nKioO3e{rt6#ynA`p`Txj*m3LOx08^!#C>F0~ERLO^mN>{%=_ZQBs~L-9C#W?JF;%*W zV)1Im;@Amli^EKnZlYMcnz1-`g4*KB|I7O!S3j@^=)E%#BPj=X#E`!(-QZpv%x z_v?5YxVvu3mirh}rJE=guVyTcojB_%jx$w$-$b!^HDht?1a%!Jm@3^wv3NCOaqI+j z6DOG}-9)i?HDht?1a%vym@3^wv3NCOaqI-O#A&8VH&HBJ%~%{eL9KCysnSgpi&rxi z$4*dNoMozX6UE}yjK#4N)E?)UD&0h}cr{~j?3UDQxz7`I6pL3g7ROGUbrlzxD&0g~cs09l>;!ckmzXNuL|u3_yKw9TbrYAFD&0g~ zcs09l>;!ciSC}f@L|u3_yKw9TwZv7XN;gp#Ud=8XJ3*~+jj7U2)P+~G3&&1STU=+V zbQ5*q)$GEt6Vx6zm@3^wU3fLSaO{@U%(ib5b(Gu-H|5>QO?hp(X}k^GT{mU6eT%8m zP1J=~vkS*goOKnqnJV2xU3fLSaO?zi9e0>2-9%k@HM?-^1a%X4nJV2xU3fLSaO?zi z8~2zh-9%k@HM?-^1hvF{rb;(a7hcUS96Ld+@qnq)P1J=~vkS*gP+L4?s&rF!;PdZS z^S?K#zeK!dw*82y(oNKb<3Z07%1xPVKPKuZxfgEAyVF_o+MWvX-&b>Y?Q!m$(7bv$FLbQ5*q)$GEt6Vy#SXR34)b>Y?Q!m$(7 zZM$1kQzVNonz%~%{eLEXe}rb=N^EMCo696LeX z#vi6iVNonz%~%{eK`pWO=KMq1k#{e@hUfnyOEqJ0>;$#OKBh`xQ7m4~SR6Y+ZLy!J z(oGbLS2GsJPEdOsV5)Qz#p2bB#j#sbv*kWW)RA{DKCgLqa#LPgZW?a`ch^nXavx%< zbQ8tm)r`fl6K7q;VWvtqQ7m4~SR6Y+UB?loN;gp~Ud>n>J3-yVQKm{aQ7m5b+WWjJ zH)ZpEjH%L16pL3g7ROGUwZw6z%5SD97O!S3j-8;^IKfouCW^(Y8H-~ls4Y%1Rl12{ z@oL86*a>QnQ%sd^qFB6|u{d^1YPQ^`i8}J`g`4v3;$#NWu{6uQ7m4~SR6Y+ z?Qw;v(oGbLS2GsJZb5C?d|#cZxV4PMad+2ExOUw1_if{C!i)yAw69t+S4|fxByoY`GsW zRl12{@n**2*om{Q;vrL|n=%fc&uY3!{mtSnTkc0pm2RR~91nUQVdbW5xgRrCx`|@( zX2#;!iI4C$o-kFqiDL0)#^TrsYKf;zm2RR~yqU2$c7j^t8B?X3C>C#KERLO^ws_7| z=_ZQBn;DB^C#XGMFjcyVV)16i;@B;z*>b-m>d3nnZpypU@7KJxe&>w0fxGLbY`I@C zRl12{@n**2*om{Q;x$vHnB|I7H?)Oj-8-x;w@99nB|I7H?)Oj-8;Ec+XVnCW^(I8H-~ls5L$?Rl12{@n**2*a>Qjk4%+rqFB6{ zu{d^u+T#;brJE=gZ)Plx-IAIu_h+JxynErMygRumuPrx?w}HFsrfj*tFjcyVV)16i z;@F9^uHq|GrJE=gZ)PlxouIDc8&jp5C>C#KERLO^ZsI#rrJE=gZ)PlxouF>x2UDe+ zC>C#KERLO^miWn3=_ZQBn;DB^C#W@kF;%*WV)16i;@Amli{DI@ZlYMcnXx!_g4*K` zQ>B|I7H?)Oj@^=)E%)C4o__w?M&7+}Q{J82l-HJ<#@oQ%byK$7`2snSgpi#IbC$4*e!ae%4PO%#haGZx2AP&aXqsnSgpi#IbC$4*eUafqqXO%#ha zGZx2AP)i(Us&o^@;?0c3u@lr9N0=(zM6q}?V{z;RwZ&1UN;gp~-ps#596Lenag3?b zO%#haGZx2gNzIn~I8jI5y>L_Bo!peymYc@gz}pO%#haGZx2AP`7c0snSgpi#IbC$4*d7 zoMozX6UE}qjK#4N)EeiQD&0h}cr#;h>;$#Nd8SG?Q7qofSR6Y+?Qwyr(oGbLH!~K; zZb{9S`yx?C-o0>B-ksc(*Or^c+rZs*Q?}ffm@3^wv3N6MaqPrdS8OL|u3@yKwBpSyyqFsnSi_g?{|)JQ)b(bnJV2xU3fFQaO}iKxW^NwN;gp#-pno>yCpTV?WaT? zCHKNjd3XA}=C$<^9&ZD8*G-viKVzzN6LsOu?831VXI;f}rb;(a7v9V+96Lc>#|x%P zH&GYf%q|=|LEXelrb;(a7v9V+96LeX#w(^uH&GYf%q|=|K`rr`snSi;$#PJElrEQ5W9KE*!fhHM8yaL>(pf!cBR1 za#LPgZW?a`ch^msZGT{@bQ5*q&FsRl6K7q;N2W?QQ5W9KE*v{SUB@S;N;gp#-pno> zJ3-yVXQoOwQ5W9KE*v{S-NqNDN;gp#-pno>J3%e+m8sHA)P*;*3&&1SYkXs>bQ5*q z&FsRl6Vw*pnJV2xU3fFQaO?!N#}B4TH&GYf%q|?e1+{11{d1<`*0Kx7-CZ}~+Huq0 zw}HFkrk>gMFQzItVK`I0E*#JO{b!=C;x|*3n>uyjxc1H}+|)DM{=-z|CJg7ZdhI2i zo6q{M{t$c5=6mn0>4&nor9Rp7|B(eN?*kx&)wAW^$5bUO4CfDk7mMS$`2&!Dh`nd? zy`QN{SQyS!?_c!roS@b?z*HryPO&(yy^pXER?p`9AXAmFFr3fo#o~BQoVCXxrYd1| zip6p5omB}dTkgX|9eFo@Gu89%bk@AKsN-!o?XEhq;!ck zx0ov3M6q}`V{z;RbrZLlD&0h}csFBl>;!cicbF>OM6q}`V{z;RwZvVfN;gp~-pyDX zJ3*~+kEzm46pME=7ROFdTij==bQ8tm-HgSt6Vx6Lm@3^wv3NIQaqO1VY`Gs2b>!U( zH|5>QO?hp(X}k^GT{mUR{fMd3O%#iFGZx2AoOKnCnJV2xv3NIQaqI+j9Z#4l-9)i? zH)C<^1a%WnnJV2xv3NIQaqI+j8_$?3-9)i?H)C<^1hvF-rb;(aEZ)sn96Ld+@q($+ zO%#iFGZx2AP+PoYs&o^@;@ynJu@lrDub3*`M6q}`V{z=3)NHw56LsX>3peH6$xV4} zxoNx&++8;!ca z@0lvyM6q}`V{z;RbsHa;D&0h}csFBl>;$#MN2W?QQ7qogSR6Y+t?`Me(oGbLcQY2p zPEcEXW~y`(#p2zJ#jz9A9$%O$-9)i?H)C<^meg#yzY=xi-3vG6-N{XPZMkW@4cuKf zWy}4IsnSgpi+3{?$4;De72laE-9)i?H)C<^1a%!hm@3^wv3NIQaqI+j6F-?M-9)i? zH)C<^1a%v~m@3^wv3NIQaqI-O#BZibH&HC!%~%{eL9OwJsnSgpi+3{?$4*dN?EOFU z&tFI0z5M-3Kc_Qe@vdU=o-Ox2rb;(aERF{~ODH#G%e|kdBkx|EHSbPt%4>@{-iD9y zuA8#uKEPDzCW^(o8H;1L{Jdt%eUPcrO%#iFGZx2AP}gyYsnSgpi+3{?$4*c;ahR#n zO%#iFGZx2AP`7b}snSgpi+3{?$4*d79A&C>6UE{^uf4Bf<)&=Dk1cYphch;qw^5Gg!m@3>fhVxmi zE_^&E&f4NBQ-z!I>FK{Wjcf0$OE=}iJ)SXDxM>XMvs(YXX*{<)Yi8Tei8@Md`o3wk zC+|+wytd9d-UjZjn=;#e!Bpv{xeL!B~cE_^)b zomIIhv+Z|Gm2R55@T8h<5?y#c+~Pe`rJLq1d_3r#RkB~cE#qOqFh$yYTU#cUI-5%(lNVRk~^J!jo#cNp#`)aEb3s zm2R55@bRE`R^_J5wtp~Hx@qphlWMw2bm94Mi=RxDZkoIB@t}8B<)+NGe=$|MY3{<4 zYPv~u;j2BFZT}|fD7hDI%Da=B^4dD&(w|9U3 zI&O)7f1SJVvGUHU+?3h&KBh`H&0Tm>O*e@yJRjb~ex^z{&0Y9-&^xPgQ)b%-m@3^g zci~Aj-6Xp3e7M9xrb;)>UHEv=JF9Y2X4{9DD%~`9;Yl^!B)af?xW!?nN;l12_;}De zt8!Cj+eerx-86UMNj2TH=)(CAvG4u2z5ns=|E%_8%6*i6C8O?M_$u#CzRGLs{Nrux zt4z6%F;)6%ZpHJLYx+vG;`#6@jx$w$FEzK~<3aDN%2%0kpJ1x=)!d3F)%2BU#q;4! zoMfu>)!d4Y2fec@UuDXDimB39b1R-y(^sMu&xcE#W~%hn+=`C}y|XG`Wy*bqsnS<- zE1p!-SE3cqhg+Ows`S;|ijN1qvnpR@%6*Qh(pPgUo>bFUq7`55$&~v%QAgdqa8ur$ z+?3bWS;yPhO__3EV5)S}+=}Nf*L0I;#q;4+Tx6lqvTWrb;)>t$0#RH;Gm}A1-l~snShzD?T3d&Z^v$ zDfcy|N;l1|cv4L_iB>!xZgHKd(oJ(KJ|6VWs@#+*_YI~>H_fehQcX9BR(!Q5Q|_BY z9d-A@O?h{6Q(jwV9dBbdWy*bvsnShzE1thx(@mlk&xcoWo2k-Gb1Oa`^vlqvTe zrb;)>t$0#RH;Gm}AKt`Wrb;)>t@wD*JF9Y2rrh_KD%~`<;z>2#BwF!&xWs*?N;l1| z_;}Det8!DO+z*&4-88r2Nj2RhTJe0i#Y3h_H|22t{c8Q4P6L47t@F>bGWmYQROzO< z70+I;ikMhoi(qm&+B*_pEXnNCrp)Ynp^SJ9{G`Hg8 zLGP^U`=(5}-!N6WX>P@nYPw0Z;;TKGa=#_&sJj<#%Da=B^4dDRk~?z#gl5fNwnhm zaEq@@m2R3_@$sN{R^_HlxxX=0x@m63lWMw2wBoBhnR0(8>ZrRHZpyopoATN^>v$Wx zDO2trOqFh$Tk-tmnr;%Ucs{&}pG=i*np^Slpm$c~rcAkiF;%*0ZpD*ox=FO+`S2!w zGgZ22ZpFug-dUBKGUfimROzO<6;G<^Cee!L!zK3q-}z^7hJXIwzpL}lWd8@zU8c(b diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120906-141433.avro b/solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120906-141433.avro deleted file mode 100644 index 4dbf180dc1db685eeb37d339e837a04c382b1b8d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1208 zcmb7CL2lGA6!lEIs>G%mv4BnF1!6I6`qF+ z2{ypB$#5SlR)qSg=Dx`MH^D6Y`ZyOLGmDe-^co}rVSa&iMyR*=tiL#|EEt*_0V=bU z>q@x=I>n-{R3#-oT;@p4nG;$y;PczPyc-!ASZ+kQ8p%55yvEhEe28K+HG#UT^Ii@B| znlP9qCYS`kA%ZX?096P=WD`vac#R~`dY+@y^)%XQH1!^)PtuQ5)NlwC^+)wCL4aZ~ zH4^-RfahdI2;1088PWuk2qoeqnIPMRAg6jN5WQr&ff#%!L@WlhG}|G>JQ$qaJF6{_ zha%u#VaL+*j*`P&64R~4mr)D%rzduKB`hlN>EBY z2`Gq^f>JD$B`5E2lh@tI_0>FF5~V1Tl=!FjCL&gTrWbfxi=y5X@?>EtN>auw!b_5s zR*F*i>dE#Yk+%JZS|SpUEhC@jB$8$X6c>C5sp?f@SqTc07%Lzlj+W;rmC7Wdf@5j- z&Z*@_teUdI6|T0Azwn|plqH;{wN!sf;@|MTQk*mXce}3E_Gycz?ws)-$k#QmzjD*7 zY_}fP`nzYhV(oH7LeVi9N;q3`$otXjDF}~icYIq$uj_l?ZWq7CX({+qu^w;9#r!>V zVGk3rkK>}PNV+mUNdK?E{e*tE&);;~-s+n!(GPnJ7E)5pf>M%Fx=TM2p2mfc)wyO0 zG!&QqLsf{mFLNAC#`l>O)QD{RoUD1jOTsvO-m?R6u~8slGkdKuGM)X45I6$9huDL!G>gK z5o5rJ*Q>9lMIm;s0PaR)IG)d{iz75Nb~rsQ(&oxZ8Ijz>?QTaVi25V7>MnlMc3q4ePX6rsWGVKB~zPG?u4(qEWrp=-I18nI&fS_X5op zU57UnCBq0=yy%G7WrUtlxycctJuXM6Ro#*GcxpswUCfByY?53`Qc)^P$d*P6qEaP1 Y4rpfN`6Xw=&0qXo$rRy2L(HQQ9N^IIrvLx| diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120906-141433.gz b/solr/contrib/morphlines-core/src/test-files/test-documents/sample-statuses-20120906-141433.gz deleted file mode 100644 index 3e7a44cb5887dc7bd614fc37bfdc967430a73b55..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 907 zcmV;619bc!iwFQPiKS5h1JzYsZ`(E$eJ{}eK=5H`F)yZMJFx}yA$8m50_m`ac_3)% zN@hcmDoHi%iv0Io>Z@|>d~HSBF#%{&SL8kCo~siv#-2X;LOp!)G@(wEl>*|g06Z}o z=!9BmRy;Me#t=54QgaKH`pmuIwFAp#MiX5tH3S20l7@J9rpxm%!Az{Ta!lC~#>WYTt-wbHxu})rl{L9kz-1{4 z7!Jv;1f99QkWmyN@Gf#I(#yi)gA5-l6Ih?E**acfw#s1rkYU_=E0iJU%9{Yfac_lo zO85}pU2^0=k87Hs4xoir2W+D>CiEj%nadI0{uC!O*b2*Wp@zgD4}X|JZj>@t$jpZt zoaYY0d3}krw{lx!YL0^zTQxd810-Em9j35~=;57PEHW}Z+}ey$TthBLh4CO@7xWzF zR>_i`_$yf!jhLr4Khml5j)lBBY`5;htuY(@SVn z5#DnGnsc_+B8jPr1o+2VEOp?_D45cV&+dA$8GDpTu{Vk-suOg&CdpUKIWR%qUKx zc`}>-gk}`S@uSV?LTK^^9VnG9{TGJRCvxCOA(h8D(#!bOYQFrYjuhYTNXysj)7h(U z<4AEgx-RF(?WqeouqXB)dy0RwJ>7-zp(fQHam=Oag0OjTso5-vmk)5Mc{7f=RNoEX z(WP#CYoM*4JHAGyMb{{ED-?N$d*s>g7vvYR7FuNDIlaxy;>VcT>YTB~S!ZT(UxEK- hX7}Rl0e5)E-u7D@oicVk&gVXF`v;|Puj6M8003;iz{vmr diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/test-outlook.msg b/solr/contrib/morphlines-core/src/test-files/test-documents/test-outlook.msg deleted file mode 100644 index c975c0c69d4199f1baf05afb5117e66076c993d2..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 19968 zcmeHv30PA{yYLB%Y+?fL>p7reRUio<2#5(FR1`R2aci|FAqfzLoR9^?r3o(GTmous z)!KmCYS(~TtF>)nt=87hCfZtURT97|wA#d7v@&-lh+36Asp!}^U4wmTdIx?a!$3$2$q$a~8 zU8dX7=rb6D(P8f49fc2Fg~NYv$=z{%@HV2a!ZcVO{8nQHcjcs@tSK-@!OVeWDGyfp z%kNQ8enfu|EJfG9+@Yl%sw#t!W;xM>gNOCwFk^h zm|+OO0$>h=xhKrXq`hJ819M-PkzM-3jOq=9SH$CBn1{fO!%V}_y6`d@Sslqr~7}0mvzK}fA{|mjr{*{eX{>W zc4_=UV*rZzNMjHB_Tzq|2!ZB}C_R~eL|5qFn}2kbKBD~~eI*?G#c=djJZAK#;3$R0 z;Rio1=HXY|1LHp$(;k&N#{G`#gE)_uAC3EoU6G%G^ohpmuF4+^anZPr(v#>%VliE! zkLY%!_Ew+#Jr>?Za{-i|Og{|8u=q3T{O<1 z^kn*Dy5xVzm(lpuRr!%Oq4_FGPom%M{au@XKaKocNVhBgAA?Qkiu|tmIGT%hRsPWs zxBPAZe7yY0(EDS%)c-nU?<)O~kbdX>kLY(T{-L!9vS(N6KLc?)_kTnm>A&;-|FrUt zfb=_$KZrhB19s|vuJspMdv#TQwD#+~{~`LRU7_zufN`E-S?X3SHt#^DnTmPc< zY=;ft&rTDHmy6IM7pBMge>?rLU6J3l=0)A9tMa3!aqnDnA;pQM^LwarE0Y-|hjO*&jVycpCd7(}yGCFg=bwq8!=9_}f9d ztMa41^5FdQQO|!O;n>S{jlZ47pL?=*l|C9PJJUaU{?s1-I*&h3D?ifvgX7;neg21@ zA9e76+U_i+LxflEB0b*zsWAGX_}8ia8E7nd8v2tUzN{1a9V;*7hn@L9dY%?^mxsH} zKJk(U@1ijPCG@QANg2(p(0w$&MsrLQJD&FWADSOO?)e|$8`%%(5Iz5U6fdLhmWRS? z7|f{evSH@H{3!hBN0i6IvT-mcz&svi9?X206Jbt*IT_|hwbfBJ`R@G;c+GqSZioH< zuU#`_VkNDj^VI~YQ5Vtl93o||LMc_L2&BdT{V9{yluC;TjapVDQ;F14jgpW^AC~0_ zVq~V}CbpV(GqCrun4jfOK>TmIB~a25jaVunlFEtuw#bkbOUd*UVrq0ahZ7zV3%?>c z5oq1~aQfE>-lS6H^vR1)Tzp~ObG^+E-|q}Q1qbdA;bbKJxl|23e*O@5__SXsRupg} z#w?0(PcBp!7jfM=Jcc{thOf?Ti+htBCNA9PYgS3s9Kx%on+~wtF@eWa?X@dCf2*WVs}<~T<}zn$~+0C5CzAN z%)u(>JtK`9>&c1lmmD`VkQ3@YbD{6%#JIvWbg4&WID;Y5;O5fAk(|-5$A*hS$!>N@ zb$KqWNcYo8!%oD8x}~Ou7P)gcb0Z_A+}KhNOs4kHMT{LwMg0((&yj@-xiR4Z9P!s` znHw=tD)QruoM0`J$_jea#&v7(Q4DWNA@LzuNhEn01bi*;_bHH!5uO>&g6_cz35njTOdmqd1DW z@gIy=j1_Z73+6_@n;%nsUG1&#s?3cq898p$@KHw7Xw&%TIKy~6emagtC(gK?)X%Fz zh1fqY*G)JvAj#L9&I1`_<_s=pN@MPl zv1E2htn*8?yCRu4PB39=LjRadQ6rJLh8UXoyf}3^C#Y=>%bdf`>*yQZ37;DDsY z>AYE%7k--h0!4hgR$2s<(5Y~=WS!^Gy$;+d1j*g)a3ISwg1^w*@% zX?nN?$G#u}*dy4(Dg5=2M)pwV=pYK;GA2Mcj=i18`>mi~a>A73M82J&!^}IC1%A4O z0O52@G5=HG$r@WS|I=wd&fis5jW5Q6-f-tcn!T)Z{B)8rc+`Q3-UNoJ3Ke*EMdA!U zol0F1R3TEiSLY9QXBBx1`ib1CRHei#+~}Ca<_PYXzV&|w^njX;-@QExpWa9v+ASx`_CqDY3pzzaY^O=7r!;ZM_ArN-h7F<%!U_62OnLa zR#u%_4NAX$oyLzv?HaqHCWjzo8=Aj7FO9-ISQxl0~>;R)XuHiXnCEv7T<`!%?2N8*Uxz8*N}z{ zz2+-F#*OUIg0aCzy%Ytn36$a`3Bk#XI5xXDtCIGp^r|GSf+F)R43X%Rc9 zeN@RO$`E3-TUrTlli`zYH0i=hWUuw#|-TUhObN? znGR{cV(!Q5Ktij!HZ}JraFY3nb}#eORc#TUI(CE4m^Q8b<dZT=Wu|p@XmTC?Lmx#GkG_TSCx}5QriS<2r_3MR#S1$!> zd+LuHelmJnz;ECJ^PQgu7zP?KeKos=2R4zLsV($7+A4gTb~TTnKtD%+fEx+ZVD|bg z+jp;T1>q*PC5Q^9hiZrEhZ{nSV1(%z%ShWO#~5}PGaQf7+Q#U)fMp6$|J^bJ#_^ z)T{Z9o2&!oYwEteCn~ZOjuLQcvNm0xdXv>RQhuuH90%Munql~pugddgYHXPlNrNIf zORuE{TQ)csn$?y?Y*6Yb0}C7rftIPmE7;i%AOusGg(jJ0w(T`X5m;or{mnD9E!~NE z-CDzb#qlZt>v_xZm9$=4sjo70b9e!;k=kVNW53JxCu`{U=v4;oI{ik&huZzdz=&(= zTd%8b(~G_t5&1FGh`+6+j6xF-S@LXR$2fh0A)hRuJ~3!Nq&}i|Iz9olh9OaZ=*n*F z?72($g{u*qc-_n8u?1Q{IY6mv4*3SCOd40*bymOV2EH(FA_7Pq^b*Jf~#3`^_ z8f=Y@ZN{X9K^3i_F8uf0>S0mfKc1_AtTwhJA*wjQdTOdEYZFxOMoZ44Rmbo?^Wnamxg5 zTYdmPlRx1n*e7}JhF>UfioWeQqpvsEjF`=jYOs9MXlwfQSAK3MWExHjCYo#h(`OeUdXN5&AR!?VB1)T5kPequUIqJSWTL?|BN>xQ`OO=9F zv+i(OM3Jn%S1irW{?-=m4tuepueBxFw4--(NuTN+c9rgPySjxBn(O*o%a2_%mp6D- zmlG{qaMsyZ*jyc`yTfV@QdOMA$j_aTtR1X=i$-YHafsi0x4v8@-6tq|xwfy+tNMEQ zIRo>@_$@nMpish;eTytCpH<^Ee zTWls3rti)J7Mu0LPCA-&(|TDv44y{N%`_hOw)r^5;Q>s4yayY^Qz^CpDv<7}4c^}G zN^{+nB+lOIQo46#OV+S}9mB!E9er3s@&tJa!X4Rts&U1~L`-e7qWtLCBu<#A!IBZ6 z7Byw9s_9V=~BK-s!oLoL0S)GmSCFJgjQq?l#FzPD@ftVn1h}OMSV` zb&kc`=hSRt^bZjATGiW1G=Z#3L(8Yo!Krg<*Kw9)6IRPyPIJIekDVO3ruU*@CNRTN zjz8~Mfv;p!xRCv#5#*6#Nd(jv4);!DsouQjCqTfclFN}p=b>}a{1J9KGCKfBSo zcynLLnf?KVO|@A)Uvb8!~<)w#!QZ|E}9a!Vb)(s4i!4jNYBtJxd%uaj%3H|RIDV4ePL z?FM75?O(Q!Oa@0>_qOOC4abdJnD0^VvbXZKk>HH>efk6Kamz=BkBvstFDC7;yzP!1 zAQK<7de`wmtH!n!EZ)A`NphC{@L6Eb%Ja@UgIJpW0l4y0`vF1!#FLe0vYTRwUi}k^ zz=$cF-BFI-o&%BwV!ghS5WzVmgM|Qh8(5vAJ8j?18sN73^0oa72MCICoL_x%ZQtdF zOI0e{d8b9Ubj<+g39Mw9U|Jm@8c5=+qpgD7k%CwI1YD|mV-R6BwY-ztY4Bqph}D z6_XW9jW_V%2&m6CCD~3K&b%5N8#6`^rfMfL)9^{`$-HzjgUT%aGa|a?bQU|CmqX@K zQ)q(}Ow*tENFH=UAW0ULv_+m@dPVq}u#r`qROpa_PtQ3%vzy!}$^&XVi!XAX4S9{G zPJ6>C(*;ia2Jb_cbt0)i?IkGMZN2dt*RFH3D_bw`s&2dBrUxUKb1h?evE=g(ZwKv| zF$ydYsCYV)hAO4!YZvGj8o-N2t;xssf+>NQuPx9Q8nmx!XPYW`H{32nh!w?#bYR!1 z*sGZ@TVAoP=EXTykYFXHr`J%!E%T^+szy7C3Ep8Ec-7=KvP%0V9M9GmO(!R%96i7E z*1_VW*Kb6AXaFA>H!^3@FK8*f&>%8q>H%qxFs1lT_WO+ou4w=3-7){!fhrTf_e|{J5Q*!q#yP|a2h0@P7VJS!BqT`Q;JbyzF zwJ7?c^=Ead&05yRwdfD;-#7n+?Xd0KjT4s5Wy73#be{UQKz4*PUO6eQ*&W6f*L6I%5uBO4h7P~W7r@2{eBkK2`WklNT;-Y+IW5iiTabrV? zKmmNQ#)$fcrkeVxjf#3_vH7%G^BJdXQv)WuWv#Oo%*}4FpHQe|^@_$yc|Ps4eaZN7 zqR%TEs`I^~IB%E_R-+YxN)vcF{5ByzsPhq|aKewQ#fp0hdY)HkHnV!#kA?4b$|VP@ z!r!Q?SM+VrH4qKe%03)vG)2|38hZArXxQ{zX#NRZQJmyzc7A{Um(`K>`S({2wyWgFn%eM<|WX;-R9wlHF`0<@%pd?ti%d++{iuToLwK-RMX>_%~5W@ zLqxL*1L(lgWeRotaq~c;ESnhk5%ZEl_BOkrI!Q3Zn!qXIHB>H_B-pPtXhIaFdm0q* zIZjA%^^W3>tgh%>|+ z$C(l=#XO#44wb|N$@qz(Z(yGZ7g$nEsg`o4W1=Gs)Q@Z`DQbGwUzg``<4Zwf;@U)c zT|kMOg)Kt-k3gypX` zRaQ8EUy)Q-C)<-JSu3b_R^@~CSRpgcHqEgV*TMzXEiG!(oCB6QMD!% z$aF^nD~Z9?Z%WjKfz}Ot&hVd{y;;@s7IC>hIWx(e+KGu1Ql`xe!b(;sdX>x$3XmAN zFA~&k>oIH9%&pELoVxuA!0XR5F;ZXOY=X3bF@G=Zh;mv|x%8}WjmDcHwZU-O4} z8pf&;rKR}`Y(Y&mj)syUtD9D3YQx{e>$HcB+70BV+GCC%z;WhJy}rI0Y-Ck$V`Z4D z0&<54)ydZl&3PwyC&{0wrl#}}-FLRhg1MsUXWDGIo!!7|B(+VH+&~+zQfH07n$Dlg zyB)y#y0*Ca>N(p4&)`)Dum!T>7I4FGbL;O^8#Dd(GVTzke5-kuxd%4m*K?KU3RtyU z8zkYg>Q3$3#sY(qR;ymA&E&skUv}2V>9f7{d2TU%u2M*>s7h}XO>dL0s9asO8b0hN zgZy>fg<#kA;Os$|o82#9a@I;#W!3E-$I@9!vYMIgsqTT@Yy?cfk%q~F*T)@l7}Fd! zrvqF9SD06EK<+VoX6%iBZUKGCuPnQ$p85gI-o`$rAx6;8M%aQJgP4Qyp$42skl#GR z8=1C|H7+PD^zur+V{2_j?HappgE-?@=;cw=mC^Z zf~^O$E6yDm8rL>9&KPg{D!X;aP`88GAr#)PBgxZ&Qu5 zra0J46EBw%8%a~y{`rebRtY4hs;?`SDT>+>w_aZ1+|8&XM6Z5qWG%}BS<4$Ek zg}u_gh`+bK#=WZewBQ$WeQo$Q#kL~$#Mx)V-R=3glN^>+Zf2`c5}0oluCZ4*vH26* zpWB?woJ^GPl<5njO4Lg>ny)4pQ>(A z7uPDve=&dP##>@-(nNaWKgnaFsMXcj+Hq8yJgyEMKCd@t#a?@d4>c3Y1hT%=) zI@5ZKX~R!t#)H@zO8&1i&eCo|umX#%TsbvX*j&@i8D{Rrn6H$6=oYll@m+b6EpfC6 zyv$j4)w6M%TUgbpqaH*zk>eDQj2Rc8~ru!xs9+JKF6`IxpFSxM*E|X%Yu`ezQEt&Du@q zZujJ$srbNX`#^iZ^3l3H{>AFCl~rL?z?YlE@?yNtJ;>gHPqQ%r(1JjGq3y8cAePP> z2_$+DtUU>SVdmp6JLD#~Xv+bb+DyVDe@-%GontIA)dPo}NjnsvbjehYQe|V41`Ezj znVy?V3V;g#+@ayM7=PROH8s~%ZUOUb^BorUaY~E-z^mXbBEe#63B6SN(jt%iv$mr> zUNf{_iz4%Ux!ZbW2G!MEa5^UOeL07+dsj(d*gdCrJ%LSKeCEkfXoZCjh zwsYcaYz1~+|B2Crdzsn$cwZ&YoS!o#=@2ED`Xx4V;a8UDt>C6&uj3;7OQtXBM|G$D z)u4~g|70Is4=rRvYaafzm^YWpkMst;Y+whO;hHWx>uByYr zoIM3{&nA`Usy6WdCnfB9Yl_He6+5TKl@PPuidPL?-4}MQR*huN`oKu zDdZ{J3GGRJCjN_2KmxmQ^6KYR>39Y^$4HXXcp!_;);_1rHB2#1HO=Ck+j4HqjO?~r zN%_p;nV85pYeqoK3{jMFW^trMx{y&&+u7?MEIl#1ML=feq}ZpDIf=QsDVez`IVqVL ziMeUb*(noJGE#DK5}!@UPD@E8Gm~@E$jr>d^t9ZT`{gOQ&t{WT1SvTaQYNJ3Bu~l7 zfk??@h5$8`n>HZ@y)J!hB7ES--x4+Hmde#q6(N-q#Uiz= zSt&)Y_KM-VNl}4RDU-I>LzPo$xmYcu<#JJxOx@x`xSI6w_w^w%?qL&BHBq3XH44}w zY(^qei8U$}tmJAYnJ!kiXsHRA1d@f#sEA@sky@rGlB!%N0;LS{2AL5Tij)Pg1r3o; zD`5%X$7B$sPMG6=hHa?0i6LM1InZz~~nqL5aq2r=Yc zI=6$F>}L3y1S+7orAS6ATiPo^aR@!^vJ)J2KDTpa)Q9$cMcYU0Kp_(30uSPBLwi+6FOQapl*=`q07ng5LuC2b*~Ea8v;2GL7_yz z{^IYOAytZH$i}5K^c8U-4ORQV>*qAs?~o^>y#PhD5}FYHe#m8rXsCnq5zv-0h4g{C z52GP+=}SA2Au;cvD9U3 zc&C__z)p!6xo;2o2qd^dq#~e`qu3#GNiQW-uwz00gIz&`^oT63k}6AO$gA30hpsAu zl0#JJQ{hl4sB`-JPNiWFgI0iET_|#Sn*#MYh^t0H=RP|yS|H!MAB_l6sZ{Lhb|MY* zPv~~i;yiSPSnWdO)8()QILps2yn=DT`?Vo2pmX@N`y*@L?z znjwp@S*Rls?1@1rjv@_^D1fxH*B%M;W#aaI6>?2+OS@|-pdC@fDuOPA0<7$w2;xFm zbt*Jy4l*b)#kDO17Kx$V9%@dMFO#<`2XfYN4>AU0f=n$dmAV21^gb2p45i2}Qh6y< z;a$x^y+BEcmLjQGjlwCyQniRR(3e`mi4G~+HJ>(txX-*ug93qMM4?7T1o@-Ap#LNl zt7s{|xMLFya^a_IEhi&a@N_l)ijC^WW#o`4{EW2ZY%+&T%?*cx?_ZIg0Hu%kwv!(G zgvxhsRgA{Be7Tv`f1?~b!(isRq1}yLXxAnBKhEx zA|W}}B^GrrSWtHZ6Loig6EA@n_jlss{e5%kI3l=9+lqqyeRDKviX(ChHKD{9B2l3vA~>8Ff)g7T5go^gCc-!|91aZ1X&EVDu)Y7?XHrkOB55pCjg}_ZX3BmUzhy2{DpoBv}zEl~OA{WEJDKCg4N;I@u zDhX34VV_5F(%*M_*xh|bal}(?LqcJq!z=FT_wVCF2g(z#e>cGs(f@YypZ1?F`03wh zK3woW>Hq(<|Nm3_r3o^^GofQWtdpmO8FwpmTs!LXF)t1bC|JkYIX>q77uX%U!9@2v zk~?@qx!mgs`f`4$8ZfxAl){!kZ;=u2F%=RH{Jg3)=1c6tgI zjP^&j!wX$7Iu-;NE@BOLg{hzIj-I3(OubpesAx2SLH|AL}vg{dOUr!^Bmc~^ZoBnD}NTG+xhu} z_7fOgp^x^BBYSpLel+1iXK+w@vi#`(H@YT2iZ5NIPeNRDUK*vx(MMFfc>eCG?LPt1 z?U0|3b?brR$xGk=(eKZZ0V11-Rf2`fdzA4x)$r1;tZ$te%Hp zBMrY`g8b+ZjLrnK$IAp)yhJg-Jzn;Tda(WwY_Kcdcf?oBm}mBaxc?w^u=<~B!Zm(p zl2fLpq!Uwe)6&!6L%5VF*~Ektf}8?h43LwFl<9(O_!REZ&39>{`sk{z(tkRC`ls{} z6OWodb*)_&_gzDMf)3CvcdWN-T{f7~Wr9a`{f*wuc ZpZNbv?(H)zb9>XNkEZbN(*La*_zxA%J0So7 diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testAIFF.aif b/solr/contrib/morphlines-core/src/test-files/test-documents/testAIFF.aif deleted file mode 100644 index 97eac1d8e3d94c4ef139d3c8b5e2a3ca16a856fd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3894 zcmZuzYlu}<6#n*^M#sU_EDe*Q98wf8yqwzSP{1%^+V z{$NYX%(hm_7>NR?+lmp*!#tQK)=V-+#vMh>p zRhtvA5`zJf=Ei*QwfKp5Kuw%&T*$J~cBX3aTAEC#7*K+qVf1DrY-M|^shk{?)K@ZGQgKndTdpa@TT;Kt!w}1C;Xj2okpi+r3(3Fv8dBH-{WA)Be{6V5oVrb-#Y(3%BT6uqf_8r`gtEh?Q-{K z!~gckHtIWRAJ4d(nfC)@*YJ5jM1JMkLti6(gDgHUg-mXu{cGyy>3fXwB&nLoBGq2% zPtbmb>viTdb3H)3?`a=RnM-CA^~6-DH^EfCB;I>GIU~46v(`nf z^~!_Kr?{?X_)@Ma{jE}|l52&%RJ)1sJA2%~XEX6Pv(8x7`-l2U=Ksm~`#AV^>i4kL zZWNV6C@#JN(?1oas6$bVfVn_^=Q+XZyX>b8=zo^{rgQBTsq`^_E@K;EE^Z_CbHHVj zU`}+P7}?3^S(w4(C(D4o)AY^a;w<`mn0GgfSqF2jnZ9wX!CFVAz#Q!Y8Xx9bAae9B z;4tUex1F^XGKZ`2C7}Ke6xHP@>PL~sHzKEQ7dh9-Bk&7(ya{vsQ{dPH-~{)q{}aZ~ zLvfZf>ESH#mdKDM&WH6IZiBgM3G-IK^!C6UCiah?iR@b^GG;k&bqCB2o}Vp?VNQ|z zu{}Wd3gU5YgZ2T(yHN~VkK&3mFkSmO_d_B#w2G`^{t}+g4|f7hYsrCi57vu}s)`I> z%|73NY1|?*nEStfyvVWTBE5V=SG^3Z-T`ynUoaD90-JXMAAbRC>J<5HHZb&7U@G@~ z2xo&wMaDI8A9-H(^8GmZh{(0`fqQ4bOzJ`L>WjcqVy>A3OyrC|T8v^TPyT4;OdBed zUe0FzMnHLvSp=NsIbF38hB3OVojxn8p0E1{0povy=~xD%FlwUrxJ z-PQ&dcj|qYH+d0?pI?joOMQ-g2k2T@?%8uHE;RG3x}IZBXP28nFTV}iXW+8_s}jED zrF_0S0Nlym74vVNC6^O&V~=)@>YFw%+5(0?wJ*E6w>u2Qo5Sx;z*9fvJT~NOUXbl` z%)f5N2med<53c0k8<|*- zv*zfT-{oc)F)pI|%AsZ`LcNTvn#Stg{YP+`ucrAPGkFI@WKnL4DLdhorHJdhwR{yP zTNzUogC|P9O4J>%fRH!@%O#&F+t;YcbN&JK!~ZY4lUZh!|1sRzveiO1x%>WWEU9Dq W2XLuntc5(k80&1&ZYu@Nviu({3A^_I diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testBMP.bmp b/solr/contrib/morphlines-core/src/test-files/test-documents/testBMP.bmp deleted file mode 100644 index c0176157039b25d71e93232a93ab16989add80c1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 22554 zcmXuKb$q1hc_r%m_u2!EYO#u$nVFfHnORbmn3>gLsX@&>%`-jI!!!1H%rmw(abi2a zY_2zJJK0V4t^+sspZjXOtN!%;s!OF()%nhQp7WgdDT%(_^{sEd7~qTo-!bs51>gVI zx7xt@Ti^Pf^IP9K|DXQ+{mmWVq;vU&CIi=I;45W9rQGClg=2AZ*u!xd_)ZnsDx+Fe zOqbCY_scyNe>6PY-P;b1GbBx=BUQ zif~FHSFKQ3bx;}(${@kmRJ4MF(FlPI0A)4hoZ5) zC@2&ShhShm2q*-HMB>nFE{DOU3nWsf#meHd!B2Z2JvbZ&ha~qw&^SC9hDMTbWQ*5I zlyXU8CXqrC$b}Sz$QX1=j4F{)>G9YldL>&g#W5*(EDqj>5z?hzsXl4+mEGxp%5CPW zIdmqE%jNS06f&*1qo=o{4-dl<5k%+}h}GuTn|)Y`fTh?tKBAQJn zaq9UF9n~yHOIRe82+pM=c}%2?O;GXBB05IOhH}U#E*V9_;8+x@ghOP}5JX%b5{e~| zkt{NlgzQE_pac{Yi-4eDP&A^eueS#dMG#PM5*EWCL5ZktD5Sf$Tgc*~adf7Dhv(6- zd?s1JqnngOxj-Zou(eXYTC6f^WO9|uY7=TzR5_oh;E=d<3WT4L^KxL+0)zI(c5vMtGm6YyRBDYG)T>6vQb0S$|(j7(`}NO^?(?3 zwb+%73WFAj!x#w%lopN5u8U;C3WtuZmWag?6putu^GSLsNiW8!cmx>_tKw5M0-8q5 zFiX%1CPTv)801(1ooiIGEh?EqLsSVU1|dR3g9+&{9vP$LV%1!rh@f{{^OeLV;~S}Bpikt8G; zhvCuyQHW}Q2A`r7kR<|!Moi^1%m!PtI-u06Spt?o!Xxq+1R9yg#Va%$MHZ9LWMLb1SdD_~u^4<# zl2&T;+8G)NLnqNVOb9j$Eo4)4;7*HHt2FDh37;PTCQ^%NdJ$5^MeyiY8HcJDGxZX> zL5$b&i5fOp#UhHC3@M)|VzJ~xoz=iq3$S7;TFL-$A;dJif(PeQNFp{$KqGNzXacSW zi^S7O93Bfw!FNF+9o?P1Xjm^EiKOHE@Tfil5>7$G2}l?b0ahrQMSv19a3UH{WFW8v z3>}XXF)%zTLBXY}g+xA+q~Ow&Laj;dG~1)G7?Z;kOGJ8|TCG!%h-95i-HgYZ(P+}* zk}$GJe$>WQf zZ62LjL>HhSsNT*#KAq#&nHgk0630ekIV6h2rnB2sCIO2tGa6MYtuq>ty6wi0o9{F! z?N+bD=?Z!HehWaKWRyzmTE1LlciELXmC>q;dV`fjN~P8?)gq#jhu~50Vg{5?C93!g z36m^lV}(p4mrRziL~@=;DbZLp7#WiwWWbsDZWZPVi&ZHSCHKq~H)K zQIkF9@nr)MgVe|&bATT3Xj~kQj>k}NC?*Mk!NbrvIF^E?1cKpYH0APytaiKAVRQIh zX0yo`@bc{%yh1<*imesU3=)M(%$Ep628B>7cDwBHa2OyU)5&Nm9?znZ8x3g<<_MstWz z1_8+?g1>^6GI3H4QNhED*Z?bnoQ)MTStbdIL8tOrXar15X9<~XC11=T0PU0yW{XCd zP9stk0tvs>&!KQTF9S920on!Y>_PWJkiAf354^V%iiBY`LBGmtcX&N1uWzhcblPlM zyA`Y;h1&rB3W`sKGKok&l}o3w0C9161h{BwDP76};y_lgV03IR1rG2L7(_S~hah3m z6kLLUtuyIOdIf<)b3ir3NLV-pkE|Bb3^I&_jo{M( zP&fhW3_T{X3k&bTA-ahO7@Yt>KnN*l2_4NQqa+L{9p6hr_t9}EA(N=)BG@Dthk#YE z5E2?%!o=%D44ayu7ZcP%f{IUb$oOUjfk&q+1u!U-O(gqGcD+=i6RRY2ZX=$n1>-TB zH*EK~wN|T4-Ph6E(+-p!Mnd5kBnFwtWKczXmfWm1X&qLYQJYI=Hb&!DFrktfkR-6=vWqsr~+`&AP8tDvbPHl;pv4CCJx3S z5`;{KOh{tWSYoC?$e;l&BBOd~$X+rMO2sope70DmG-$B^G=P{+M3!*KS_#c0$0_+F z84oN~7>kVL1M9`b@~9{Y4JoFQlzbGMf?yEg94eASL2)T)Ass1ap#UwpG?H10mGRI5 z2H7kj8Kji6`XXaF)MUMwVv&*6T(L=x5mK3A9?)|-kz$gmLKb&DnRRL{rC_uYh?YWe zuhlOT%JFa%3J(6EQ6%8M2skncMWs>&Lbgn)SEzi6u+VN$I4#X;ZFg#RZE|LKVsK!( zTAwcGM`D3il&s|fh}C+bLn&vn$s8lklXZ-ZWrad6oyTU7Xq_FMJ%}DWmyDLNnNmJQ zB?2!D!>3R=RH2x|5HNbEXef8Ndt>I1C(Pah0E5S+i#Y%*1c!_eQPEn&`1Q83@6|TA})eXBB><=m6Rj}&Q@kL+kzgq(P`EN>|(Rl>@fR^(bPmKKAOx= zqA9JUpLUYB0w^Eq^Oquytrs(E~~oFNx*lp?B<52_)B z&z8tVT&W0LQ;vie^w_C%3Qa)4Gw?(TK_?bQ-7beG%oB;gf`zb%Fb)~XqmcM);K!h# z(s9XsWGpC=IB+MBK@ki10ThRfS8%Yv(Q(Nvm4K-faI_M-lt+_skRmE5G#s_S>{5%3 zB0Qf)lX2w=u|_M&WXz?qr&M*@Ey|SBHI^&1a>Z;YUW%vFzCb>j$VcK9r524Npg?EA zqT>WiIGu{1Q?XJ$RW2l()HJIBC*asjrhFu-P$^tqx5%g#+x5YiS8Y_FsF*GS0?xw4 z(#B*?7bse5%}{GNUTFpbNvpx2qDfd1jaaLf<3$v{f-hDJnQDPRFBR*gI57h)qylG7 z;*n(%fl;l&5r-*aX2B<3-^Mq@D)^O}IuvK3+(zr|6BXZ$9IL?VF`NoXDm zLBgU%9MEpy_-wL5Off0hRvlKsC+np{rQ+R{rOlbSXd)&sYgJaQ+$IQR40tBC2i4aL z?NM2H1O4u?(L}lKjTSAgxL&DOvH2{mQ7(}Q3^GXUCjqC~(MFevOAt<$HL+u2GZOQ9vpO;kJ$jlv;8!vcI1o=&_1?e0MK;aD`#)1fFh z1PKMOmvmbSyCNa13x+1U#L}67ygbd=DPc3GeB~K_H+JU=mmYE|W{!7_FTwj!Z4p9p_TDHCQYhpPZS-ssH+>((w5VNCQ?BT$ z40!6zKqPP0`Bb^Fu&v)FG4U#OXSr<2mCVVs#uw6POe&*W%~$it5*kTJ?js@LG!#w7 zk}5^(L&c|?Q_r?%95P3{sI!t_a(#Qnt`*uqHfVrP2rrGoCD< z5LlpogTzJ}urmD?Z92ek87U4BBN(|MJ2z-00fTLlv3ypc1@z8Fu0?6^*@-4O#ViA% z6BT4EDv3F$%hcU0hYYP`WGmL%aDZW81rn-2LK#V`(|KbqZ*({nI5HY7ATrhL9s;a~ z1V_=ZR4&844090Zkl&ghtcxcor8ZJh~6k(*f`8LiDx6 zdd|*Wop3Cw7vu>@STCfPNXGZUATSI9MnF^PWE`1*CZG^xVA|0r3Wi9-F4QwGcNhNa z%RB%4`LpSPG=W3H0g+^rSuCnT#Pb@}^;o#-bBU=G&?HF|BG8@-B|@NR1gZ!{6i~!+ ziOb-2IFv?>!DZsg%ZAnEm?-ArxUHO!MHp~sgFZpjM{}FRemmWv zC)?EAsEgw<37rN>z)W}P$u2F{C|21eDvQz@Gm}I#f5PBTYFIKFsAxK!*lm%(Id~MG z3u-&1P(uG{5luDA|C?voU2#$#1hzKmuGvYDrAap?^ zVF(zSh=V};h(vrxFNlk}kSGKc4(R|8q7iz9WH1woJ1rPI4o4u6$rLmi-@DLC0i@QE-AM7SR_YzHHJR3yXbrWzwg}hNKC%#P8s_EiAW@HZIDhZ2F4&^`ue)M zd%7Vo=#`Eu9ev%vFk`V85cl=KAaDfyEV)9%b9UpwSnY6ncyFZP)v0tszL*7!2{+}j zYz$SOY|j4b*-fimOA#`73IUEnq@6WZe34wp09g=^B~%CmDiMK40nw`(h?7~(0FiIl zB8`~MDNnYZG$lN)giorMsZ=_<+3EMVVGrBoiPHtJP} zyy}2M=(EWJc6r>x1t>XmB)gpE((1uer301W^5}fs5VmnFa*b2T(9nr8245pU@Q64N z)`6^mf`xO57$pnFCBpe65|1h)Vr&8$!i_I8#m&5sT_Wz(Cf=rjbUH!Ts_6`zzB2!xf`QAm_1#Jhn{A zYs7uqBaNq2A659)}Rq70<}a{2`jfH~$BIdn3giQ_ep?OK{gM|B%WW|bl2w1nL>gMere@bprNK_=4+ zP#}P`st|0DNm5C2Hi&1@G6sZCh5%m)hxkQI4x}5v)z{U5!Xk)l3XFlr60t}$9H@FP ztQ!UGp`j5jrL-3FjTe&}&FXSJKU>NA4Qd1sE~gha=Z()MlMRtq`dtRM6ln`4ldrkBZ@%Mf#}O zUrf)>j6d7o+n$}8of>wR!t$(LRx!5#JhmPejl0beg;_&Tq zf`*HdQSoPe9EV^M6Qm57fQl7T0bDo*6UV3YUbvvfqx2*UrMC;#cCq{71uP5-lJoZ7 z&R#eKM?k}Xn4dd$>Gjv)ot@201IV@Wtdw^?k_ z*~9io)$eVF93j_@jm2~+6DcLk3AZuqum+qajRn#RG7P#!XARiwBoSE9bbOk}cAv|IHkBHE^vwUWWL!Saon;b&V5Up+XPZ4?9|KAA})GRbm-f}@cN zG~#-}nauf|sh~aTk%wK%xJ{XKT7!1JP60Y2o!gO^KxlkAdj=&8!hIYrt zg7JthYsY^uQs&_J^znt%%RSfyjZXk@E|F=R_{P)vVr_9eujF_~Uy=cUPy#QYMJ^ zMZki)jAE-UQ3`_&#vAi!LN0CCDGu4$E{)n_(!0z`qtf8A>YWa~$HCReKvE-9>nSub zg)6k#K=v86Mtqt?z%iH$3|C{#YN(QP7E|U*N)U7~J$jK-%P}dKAO$rkM0Pz@52~sX zY&2k1LaIr@(n>&bf|s(%N|Df}mpF}V;4wTFdB7&}Sa@C=(JW;cWfYT?sgyui1Q5QF zStJ~^2XgTeH~?I2uf2ZZ+h~z-pA)O&XUO;7QR)w0^tHs#BY7I=@>0Hc#yurOk%7$cc72 z-Xz8$`rwx?LoQzEK6n1YcV0XH?eCoX_IECR`#aE!7o#e1!|$vIodB6;BGgKTYGH3J z;v31usuABn%-rz2?E*9)BX6xl7Xs#``e5DvD=X{g>rRAyQRI4^SOb0yiIOGnC#A)&- z{0_g}>2WAcYLmy}i3VgIlh$ujc`PD_p6=9AZEC(lD+V^zX;gSEQmamG)*2nAVltvP ztGO%mkN)=G23tguZeGAOpDo;5yg&#v5&Kyx#uiIUyQ8=(Efe6H|rcNH#W> zk4=|Sllf#N?5l*l1+RTD=C1^tMYp*gbkswxX4KzG1ZIoL@my@RQQR4>pG-85#s?0^ z29G9(uFs72&qr~HnTSLHJs!Va-s&NYI* zPA%~mXeKGuuI6|Qs$xhM^V>3EZ-3rj%@(VL{(M>}5!1QsXf$N9+idY*B$gho52$=L zQyBOwkIZh?dz_h4DwYVlz+N`!5G;C@NdfkbJw7K0sZ1WT!EKZKt=y1_<=1P1R)g1K zau`9nDpi9RP%3ul*-kypts(n#e5ZjeWTHCT5p9?IE?n$Ae;y##bKwG`wHIaursM*r5|R)DpLWVN(GXk-$cr z&m?u~fGqQT7P?Kz_8LSfudbdl4&*#T758vG)+_}Zc~?H}NQRVtL#Y&M_7~E*lqu>M z%;ZnkmgH8Q(`7X}tV)N);C0JgMpw*l@;F2`Er3fJFmtR5wNZmf8~pMm*u{%|7cTUkI}g2d z5ty!~+pt_t&6TpPM0g|}8B2#MZfo6R)zHYu_V(_J7l}Pxkjs~^y#6LAk%X=eznHsF zjIT8cS6kJS@qvTUfz4)ZwNYLltlgX$xidd`Z+ZHi-PKR7@4P(Pw74xYvx;k1(+pCk zSIhQj!7zissiCTbT#Zx;1UBlFr~T5jUzrP=26MjQnxmf04OCLqVmKXlB>drO)}N1s zYe`eY?sVGSsSrmfD#p^=NE>HMz&oK z+Cz?A&2;L39!hN*jzO*h6C-Mk$e`0ZEJCXea2{MrV8b*)JHx9J1}scI4c68M)*yfg zdf_6by#wFT?%~o#L(cVjezK4lPJ{*`o}f}J$D#Egq`KI4<&ATfUwggx{KfXy&#EUl zx-XoUVGzT9%V^Y_1ddxRS*>IaNBXa}>Nh5b-ddb|xIF!EW#;+S@Kj)c!DhDDY$l(_X>?k30f))&lsk+- z+vRqn#ODN&4$Gkjss)HAcj}EUi`i`Oc&tj3R_(BYjeZ@d#V(Uftpr>6;0p2;0uhyj zZf}QPxB$O=5pfwHc7YA)aWN^=QO`!BFp-H4hFlIlhtS!fbZ_s02E+(#xHXhE6JzSW$y)gdP>g?NB zS3Wx4{_g4itK+MSvm=_QPoD_baxq^vmdeGG*?2e?x5R^%L?lwqXG?`bDz!W|wmvzt zJ2|yBF%>Uo+{L7OsG1nAq=#yOX5P_AqzCf8Rv|H1+gjcPw!vG@s3RUtz^(E)41T-9 zWf3_|YMa>*Z~&3iyY1jkHK^bLr@(DzIgEmUP3|^oJb*}Ay-lw)Yt?20z>{TBi`*ui zSvQo;8Z>IFPVWU!+rYqt)#i1%fa@@64Hl!pZj>585Nx%D+)A5C?lLLtI=(>$13PJSHUfhK ziGY|7cJf6sjZCZt%@3Z9fJ;y?N_QvV%-Q?BbdlKE;p5WgVy>3QEXShI?Ppc8>zj24 zzyYK>cXs|ih`n}(*f&1*jn4BII^H;U<&8JHFI*(|bb{!_&!cY?qxWV;pRdh6U7LHf zJ`dn}XLs%0{f*CV9ejLxxIQ@+PllWkufky#=oCDwnq`sGG$IZd`0-eVnj@O9UmNkb zN(px<;V338rC7Y4v(?hta@;yxaE;g9BUMlt9mSZXnl$%kEtQldTEHBqwTdf08l{0I^R6kbNfaB&c+Q9zai|EgzSodLzN3Ek^xoJV=E+$)vV`i zxs}V)tpt#-R>fRN+Xjl(fgCsr!vpR{(bp`vfrvG$-fA&5STDv>^=xURJQ(ptMrzH) ziP@!z8JpG;^#!92&)QIPqL6PSlVio~bgde+IwMx#P^AHbc{r0V1!9d@CSY(oO<=Dr zHI$6kBXJsmp%BOtR=0sp%P9CB29XWvMz*zKuUsK_b;g9Ox=sofWB2(B;ElJx@y6xX z-e`OMo8z@NzJcgIHp;nT(G2V2XpPWFEK^!{+O0j3Z=iI6c8GG`*%d6 zEa%A80-Z_YrE>LRp^k$kWEH~gOmHz|lcV5(R7xyzHQ~vsgxJn@u#|uV0iJ;J*Y?_* zZD+`wfddv=2RPn5*Y)Q4&ezX=3XBAR3*2+YLjdnNz^e=Jwkt)A&pgup?*ccz)nVGn=wYIvtP#6N3R3i1VKNC+B^QC%uacpFMYII^|v^vz-TAD7` z%h61-H8~uqm1C`HVrn2TT8j;pVuSVYKs7fyRBZK^8r4j_TA3c^QQ2}H2=**Cl|pCN zx(!Z)Odqg#>ZzPrXEmyU&k72-Vx>gMp|kARvd8FPlerd!$*r?QZO)`U z094Om0I^NlPH0>GZ}@a(_mfk*AVcqy-yJ((YWvNU}(I=I1k7pF>NgVFx-=*&c8d?+#6AMCH%G-iteR5^!Qr2#g;WHj3J z=A^!PkxPk*))u)88)E8XQUg~AJs!J$SumrsUlPB~9)mKngniN|Ek`hqH{*=-6I zLb+P9Qj8Sjd=07>itX%BV$oHN3}}>-M${8=%TaGeDS%$Qe8#IYZ~$}80ieL*0zjQ% z@{Jq;tiS_$&RqoT0xUY?*trXTW)u)BfEBs7d(3TmwlwkH`uy$b(cx5bJfE7#rF|+{ z(xfeWTtThQDU~ghi;uRJhZ+@AA}I0M#6hR4oKEM{xoR;xPyyblIzE)I7em1y=m#B{ zNW4)3SjEfvOt}yM(=g+snf`iXY-D_9vJ_4aTv9fHU~5b7!PtK>Ie*{)bZoj~6Gl25OV3=v+QIl8iN@p+^g2U+%BH zzcyEOIg57Fe|`Dr+U)GY)O4^_%Z@cG1NF2&ST5w_g-p6x3+FQq;N+`$S1F%r*0be8 zx>73S3+ZCj?G6otjp^#Z^5|qGna{|8dfA`zW@u1Cl#m~>AT#{TBs z(eEB!^9y)7Jo#V0eE+Nar}N{((R4Bxjrs!d%0Mk#iH$ZJOXDN!D+^1j^VLiaOsiX! z=5#y{-rZDVpubR_sI@l7rnjc&D&Z6e4kKpw!zVAAgF~K(U$3=B{1J=a=l29cZjaAu zaT*Q%sYu)p#so}Oy*=n~%Y{lIUG6p8!8W(YXanOv0y>*UrXaeZcsLr2Tod3p5(0&2 z??9fBsvUWy1K-)rfkF%ftcyypQ3wGr6N*M)zCjXtxeam&9G9S%F85u$)CbVJco}-B zt?zOhqOBcZ)eQ{T`7;=MF92XK_g-r2xpcYbQd{rkHaY|VkeRFMMus<_gE)8T0 z<#4K58<-!S-ke#Q7@3+Mo!p(79;wv^b7fHOXBsV=%4m?Q?Rv{w?|(S9IA5qXtL4_h z;(Bv*+;8=IO*W6&3}SmQqHQu-qFyf;b8+iU4y_LSmdmKKY2*QmO(vAFDNGW9gh${B z2s{Q%FLm|6+OB}(Yyq~nV>&wV-90=c%E@FT^adAS5EJtKOsWtDqxSUSI=fI;IsnJu zm#;tpmoB&UT?7Y+2itlswsl{)+zrkEM)0Gv=WK_zwS#oD_wtp#wsurU7kK8Si2Lgs zSNSM3rn4K>-id7Q#CCMc1RS$UCd1;GFod4Qh($fMfakw|@N{K*tTkF08LclZj+MhX zhsBo(i5(D}EkTW(nGG#Y9hw{~p$!VQPqm}BcYMa&MF)7VqS73GfXl7;G z=Z`j8GwF0a0sNWOmGB0f7DvMEwi>_&h{bC$n^bzQ1tdT^yGrTSDs*5fO{NwxcwzPjDVQed?nQTYVLG~qUA;m!lSiisu~;qwVW82aY*x%<{JZD3?;WhqHV2lb z8qG#A;R=~`&agKc_a%Z3e}6JxjOKjya55NAx@xe)v7FiLDS+-V zlxkFmMyj6_RNC?NvHha-3h6axyQLE(g62%!(k#^R(jmYOTHYs@Br!X{Bxeg3$?>=en} zGF8%K`N{su|M$haRig>t-G^%LLU(lGI=hM8edIo9QX`)Uc`a-n3x=}M=zAlzk$V4N zt`aNf-GzkD9yF<30c$Akj(T;rh{Nl(`2E&^-{Q@H`7^600zQCb^&6G8m^T*nC4iyu zIfJQ4c6f3%nJlK$m1MjawA(Ts{bTX z2Aiwk(-i_bj{rrVk-xJG-_=F!>8ADeGWsA28lOWH1gt@|NT(KA4GNdXm6Gr@LblQ% z)oLWhSU8vTMiSn{Y;$sUXuMhKZEk<3UajmL zuPkqlj?L~aZ4BmWpg-~I>$}TMv75yG#yC91M#RgR*vWLkyJ9A%tX`KWGU)T z*o_{KEfDp@GKoqlUs+n-7#y0LTUe{sN5kQCqgV&qnnUSu++ot`!I-YK95h9&#*|A_ z4;#usb0r+k`kW!~5d^(KE7LO2I7}yCeh>8wqCO4;BEb?gB2BS88uFxVT8CDmb(q{> zOFR=T8BP9ZVxU|bjz>#{+`z!#TxD>2dU=+?&i@zBuB((=`b*^So7YO6IfzjSqS zW_@dGZ)xjjd~AMhdG%obWMlW{?%rvuHXH(5qwx%wJ=+-?{rdWzoXWuWK+#>j;2`(G zXfUJ*gJr^y)IJ2h8|D>DTj@NQp3SAxt>OMcsgm)73?dr$C(_|$Ia97?fRz}lWGcmU zdAJUOrAochtd&8?oof^a%7sQLTdI`mGgC|TfvIe=0I-b&68U(xRm#oRQeddSq|t^g zTDMjiuqY!IdBQ3$_>F*FezP&^u!4_n2q`oG6c8N1B^CrCMxcUbcRoFA)Y`}Ax26~N zEJm+c?}!9*`9wJxD`qo|a=sM|CKItrsn|@WhGMB!Hr48{j!aK2Pfl#Ctewov@6K;r zpIbfN*tmZ6=;7Sr{>u8!(c!Iy+12^wjrFV7j~=`;GCsGyurt*h&w1k$)#2TRRgX%i zqA=J=|nN)421Bh~&&s@y14)1l00W4MxQ zl!^l*gA?U^ZKyspJ~BT%I6g8oHPAoSYK#^Vm5J6&IoDX4U0s-69vWX9o}4LXD%IjZ z4UD+w)6?ZZ)DHMy$a;-Aw-#_QldAmPdTPbhb*ZjFvEEv_9-Ot0)6+&$Vk8J}3(-aVe*+TTC9GqN5-2|!;O*I*^PR>RW3B=#^xqQ7r;fCnOOej+AVHx9!?C6uFS6w4~}hZ?rm-C zwwlw;1|VK-pnqhbe`sZ_u}}?V!fudNM;(@o*BY}bqjqD`p$}V)ew#NPaBGA@P^L*e zedL}#78It$zNqCx$BH%L|*6Q-H4f^UGJ~=eMu!-F@`p!=0`mBbTN;DJnC9(!P?HL#{)*!ivxr6 zE32E$=IG-5dNP`ynOZ652S*1c7v{GID$T{k-KCY?T4ktS8#%k0Q=4-On};WN4{tr5 zo!#8ry|cb?Fgme3yRdtFd~0WOZ)|9MW@Ku7cy4xLrPUap8d=Cj3X}a)Goy1GlZzRL zw-C)0qJ{antA+GHE?ioi-I<kAJy;czXN(i-W^k2gi3FKKtO|qj#5A-g~sRvNt!iGS-?aW^099J(sM_ z&#aD*Ew8PfoZNgeH$6YpoLQLN7#f&7IJtlO_QS)2+sEM5p4=a5&aPd(F*mm}Jvu+t znw}h-njW9q+t}aQI9i-tpP$|s8JL_Lnwc4%S)W`UA6*KRLTb^8P4bBgg$LFRu zSC{w3ho+||mKP`2rU4^>46g1^k1p+Ro-9qSt*stkyZPer%g^t=_0InO?UR%HH}Ads zlb`5-Kd;367LFw%iqcCBuW~5f>FP0jO!E~&a z$u={oW;)qFGQKc1vC&^02CwsA_jGUn_T=bdCYo=Q$7|)0O1@REjIC~5pPk)WSl+Ib zhKENNPj5ZDar@Ey^7irdC;JDtcK2@Y?Vrw!uZ)h&uS_j&E^pnx_he^dcWh*4W@c+^ z^>B7LO|C5M?(W`PnBBc~``KIf-nn)A z>8;a8M~4rNkMFO}ZCyXS18{qA@5PJvKD~SYozq*7?>&D1lh1zq?8Qg7?mYSamp}Q{ zU;p;&U;Opj($yD_o;`p5{*Qk2bMSih4z3*^pRTTNEzB-V&#$cvm(xK{Gj6**TP{Uh zrKBg}^o=Ae0}<;0h-CSEQXhoc*T;gvt$cpU6Pju@1}pXc8VIQeTEnyX@>r=lHZ(kc zbnSkte=46Jxqa)&!~1V9&+QG>M~k`IV106Uc(RbGEzGPh&a47Ws25vX8;46vSLYWt zoAr^SlRN9n2P><);Lgd3wYk}=<3kI_2RAp?cZQnd1LfvWeR^_YsXyNwDvvFWuU`dD z;`)Qd`R)6U-akABTVadW4(~6_Y|V_XKYZ}cotsaO5AIw)dF$Hs$Fo!G$47UUXLfF$ zKD~STXmfdgYwKit?c}W|uTE|~`!B!y!xumP`3IkV`S$ytyz|aSfBCE5{_>Z6Xh0CJ#WzJGXfYIbwGJT^Nru{g7^F*mgLV;qx)a{;OC!y{?o6%{;T&s{Pb`C_U}Lc?hpU?hkyL_Z~pehlUMJ( z{lO1^@RNW0w|~3;=-tD^oA=&&0@$^ ziDe*J*Z1$uPA?7*%>m;yHoP=7wYN0Cdv*I7P^zQd{x2h9I+b3J|hc^%IzIycXy~i&;c=GaKeRp?d z|JlPA_aD6b?EN1tEgh_{9v|-CIoZ2&aB%F?-$RXeR})mCooE{uso1L0Dnd>h`-61V+y+7%6xdIMHF$AX6 z{FPW_ZlKgmChQ!xmCguB#N|k+8g&koEA>Kca%6sCZvE)sHYlmnv#YD?$2+^XZ=XK8 ze)4dA^TyHst=+AIv7y=V(XI9M!@a$?7H9UZZd?OkE-hZYed8U#oIAIk9_-w|b^Y;M z4_<97@9Z7k-&i}@UO&0Id96_zJ=r@w+P!&odAE`4KiR(i@$(OF9o@Wn_2`TDKD&4N z_``QTdU)sQ`>#HI=hgQ=|KWc=Il8~Mb^O-dmoHy@^5oIS5AVPH^oy@P{rs!P_g>z( z_V~u#ci#Wti${+?c>eas5AHs>^WeRYKmPL5?|%KshoArYuYdddKmOzY{Gb2p-~avJ z|J&dH@o#?nch~n%KYsPeU;OyzfB5@9?Csxq@%F0+kDje;?k}%yUftfC8E!NS;lY@F zs+tA^%&X&rH`gXMflF&PkLE`Arv_W;^gzG^V%tVCQcnbSHb7T0Ju|*KGr4r__{P%W z*23J@)&0}0%_ATVPoI8#eE9Iw4}Wmu=oD1Ct+k`MnX6CV`QBR(Uu~`J&rWV^tsE{d z93F1p+TXc#xC7vQe&@!MYX^6ZZ#>#uJvi9EegEF`t>wMd+3lmPlcmwctMl7;_HNud zxc&V4gTMOeFTVcli~LUw--3 z;qI*uKK%Nx{`&vEedp!z^`~#$d-$So?~@O{c=+JmM{m9S_~S4B<~RT4 z!;ilIo8SEHAO7K={^9q3{OSik{rOLR@%-+iAOGM-fBIj4`t2`%b?5%u&)@y%;P`Z7 z_jq$>e`|B6ztO+ExG*}{7%CUNPSLumhMVKts~hv< zQ!}$Go|NQk2e)jRp5B~1gfAgEa`1K!u z_xmqC`rbQ_-~H*AU;prjzr26z@rUnz@0Y*$&9g@zz5D8mci#T^lULsZvj6bj^Pl|i z=XY+tczE~eqkAtt|Lh09`K!Nq`R-@`@ZbOG_kaK2zWUMEfBMs({`Ft}1mODU)o0&( z|D#`g|A&A0yWjnf-~ayf-piZ!o&%JQuASbx`{3l{+Q#O_!t(Ok_QB@W!$2T7H!*dv zySuc!IXS+(JTrcJdT;0G=HmPc@Y28^t}kvb0dbyR-`+S_S_RAb`kfmOL229FzqPY{ z0~owd-h27<_I*I|FFyO(t?Lh7JpJTg=ho=(>aFXKfbjnReOZZf+IW`!ho^RSrnYxx zcG5{7xg96*0S3$^E(r ztfU}rG{6dW#Pjm`^QV7*va{>KY4wkP{_~4BZ?}y$>#oz}z_yLnoo&mu)xKjez5M?5 z=7z~=w3#i)hY#k>4cqJ2=B-V0Aye@vtl+dd|AzCjVM&u&Y z4}bgaxzA3luh|eAjo1n6E^fDyB+4s%3bFbrzsS*H>$c0jgZlma#+z-A+2ywfXoQK; zN-17C=?pr#((s^D2gIt4+uPfI^Ppa=p7f564+pc!+30Z4X?9U78St>UgThcrq1AXW zG43B1^n5rNJM0e6rn6YAq)LfIELY4o>h;d_Y&IQFR`bQ=cy!ooC)8L$FQh~Iq}M;p zl{{{mq9mu&=cfgN4cguAzy9SfYd^n!w_!Dz?EmrIcduT)h3d}b^4i`0jSYj%;X)bK zXtDkD{o4B5O|#if;MB7xYoNUdL}SM?@f7O8U%%OQA*|Wx*7#saPZe^tkergGXgSl% z#=FLk#NYn(#BOw<2+dFmN2wl!vRN>z!R2!@A{)ir3^0u*RO>eAn9Hq@OJ~}+QyS=-*yt-VjB4TOl=`+moHx?qvPp#yqcePEA_MS zWHCBH+$_gv2!dh+>+&*!V<@|Q1P9v&V*kaN*G&wx=fG$DjlJr&E2 zI){_~D4i+~4v&XNqe3FrDK*R4%5u3}%$J`&-2+r#e*N|O>T0!E++M@OSdIsS?%w{z zbndcYyGEDSLx!Sh#Or_i?%mTLet7ljwGX4+e)`4hcON&lU%gs$+T0eSb$8cgGFdzh z8^OR?YS-z*IL?k>1WsD*9+Fd~P=vt)n=qoYxI=u3#HDVdnNT7J`~6(50`=jbJ3D9| zV-C*e5p4$h=DL;i%Wi~8M9XQtL8EFVyYF?-MK=KJ^&^PDlYCfG_A2#8Yd@*yEoP4*ml~C0r-j(ugl@ra-*Nu*nPyl&FCVe&ZL zAwlZw_Y^g5*mBBDM&|PrE^K~4a2M$`Vjer?*bVS>N@mjmOv@x%`}Kjtg!xg)YY#Yf z3DhrBUOA=|X-|-__(VS)Cxqp2Jl@*_OhA`D-QPoK_rR#@>(Ae~0nkBcw>P)tTr;be zafHEbXtUDD>A6a}S}RoMQ%Kc|@$ncSg)46^FCT9KcAcF9ovvo%@x^SiJn3GY4yV1| z{B#PJx!nwo^In*QSnV4hHeUYl+-b2U6WO2~izSom@3(M*d-3!8Ag^q1nJ_QmG}#!2 zr&%7zCeoZpi(yU2d<+!5BCF}Kbfr|;HQ5oH=k4ncDB?r5x9vNYW@BFoMr=EtPHR-k z9+-{LeGnloq6#{RhRmC8)E!dUOf*;yQkk6I4ERKcgWh^;wrqHuHq5$>`W>9t8juj$ z@@A8BdPl`-Jst-Z0ZNcEK(M>}dk7ppf`}at#}L}5`%kUfKKQakX#&O<xm>kt)r(30=JM?FxHlgTmuEA`L7Sbz z36>@~)Z=^c<8%9tgT!zx80IC_k5ii;w-E;_b3vM*2@HS#ZqsUZ1qjCDCTWt7sL(^@ z;^73eRH~8;3Xx2vHXMu?Mv8=Dc9YYvZt}a)AVE&grn0EMe`#!$`YMz3ngh5uq_A3u zOo^E4u+Ux$ZQJl`Tv1mlh4dcpR|7uIYH)vezGe7m1u!>W8BA}Co-Kslu`4!rR@NGE z{d#o)k;3}zx8DGz+v~4N^(Z?)HqiWD>oBes(y>CVSnD>Sqjy7fJ9$&1+> zBnxi2oXwY~)ARA^`RVCBc;W5MvCN&+j;7-2KoP?Sl=iUFAZFf5FZ%tpjd2$+wGgms!0Bq?mOyTM=5;iM8u)<83l zM(JoqkLL&v>2>(x!N`88*r>J`oPGA6uVo>Zjn}{bG z_6PNZJ{kyumJ3qYz+5hqdj4~e2Yg$4QW$O--!y z-Aj`X;h+3<&Eq7V{}+rsU#~rTyZ+Z+>RLAh3oruNeLUa5Lqsm>0&qgH#; z>#b(9*=P(R-z>GejU!#kcWeFqMsI#{)-E@vC#R=J$DMrXaytJ86|nO61s?&u#~)Cz z$Jzsk3P*r$db*@32?C{j9-nE$#0OZ342Bh5l@o+7uxm7nLM#}Bk7>V;4ALQ4Qf+1j zL(9>4v0QF>?07O(ibb-CXr7~${q~@csRg+Z8xUzCme4XuEz+viK~G|;?$|-Hk@n_G z8|?|xE>RC9cvL~nd{V1N_?*I|Su9q{v~U-1-f}Ty$nX)_-tfMCYv*Yhpo1Bd-v<9^ zK#|o!vpy^pFV1J!PphcP;plyY%SZ>4HF zO4sG+AYYjehO626e}nYDem^o3@Bx$_!G!H{u}CD#0w;&VQPNL>CaaQ~OO{ntS3^;Q zVOOBVU?`(T3t_Dwi{XHmW^lo7vhP~lq8J9bCJAZRjEQV26R+S-CX>i#^n%E$xX&Mz zOMBT|BVVi*imS`(a8QHUk>9KG7<^ruLV}e<-e({}RJxuU6!p3g(7_6mdPNIof=q-D z1ZgMfwxE7zAg<~G2ZrzBK_6F+m2yH_r{jy^F<=FyJ6PeDU%x=KUqIYHKY;r`x*^m~ z{l;*$xQ_c{c~K8zLOYh4?X^xjd)JrO51+snz6#yHS|EUr(g#3+UR|v&vxP<|8kfT< zfz@CRuLKjZXjTiwlpwtQv8!^Jm(!x0^btW-f>cZgX*m_i+Kg6)5Hum|bK*1=3G#Y6 zTwr{Bt<-K*T5cDW)HII?Yi6^@-HwElgTW{kPTD`(Cgbynm^wN*QJ8o(S_mMLOeT*G zP7{$zCRJ0y`BMF0Fg(Q&HkYV7wlUlyMDScxNJqWNn%GT~6>c|LQ~D*Lb>2O=xmZ9r zcel3?3xo~X15NJz%`IScHab}@mN!>7(}P)FC^lrh8IJdospCd{dNKx{-(Ft9W&bBo z|Kt+v{Byw{5|kt-gz*3ujuqimg~$d$ZF1=XCq^tgPQRPv*a*j|O0cNtTU@Bdx!bGqk5~8 z%TN0~aJ1!o4y_U3@c&>1ryiLFJ8&hWWF(#uWR0VlOk8(aY{gXepis-42or<=A7rIQu5`J$pb>2S#cPkjq6WUE z_qcOV*1{*FlSnFsdnuT=K+BI<>2$tPt?pN$;HmC6dq<<$@@O*e^v*`Z`Q&8bGI1HL zA^8%lD;8$55*sgz2X%Rn - -1795 -From: "Julien Nioche (JIRA)" -To: dev@tika.apache.org -Subject: [jira] Commented: (TIKA-461) RFC822 messages not parsed -Reply-To: dev@tika.apache.org -Delivered-To: mailing list dev@tika.apache.org -Date: Mon, 6 Sep 2010 05:25:34 -0400 (EDT) -In-Reply-To: <6089099.260231278600349994.JavaMail.jira@thor> -MIME-Version: 1.0 -Content-Type: text/plain; charset=utf-8 -Content-Transfer-Encoding: 7bit -X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 -X-Virus-Checked: Checked by ClamAV on apache.org - - - [ https://issues.apache.org/jira/browse/TIKA-461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12906468#action_12906468 ] - -Julien Nioche commented on TIKA-461: ------------------------------------- - -I'll have a look at mime4j and try to use it in Tika - -> RFC822 messages not parsed -> -------------------------- -> -> Key: TIKA-461 -> URL: https://issues.apache.org/jira/browse/TIKA-461 -> Project: Tika -> Issue Type: Bug -> Components: parser -> Affects Versions: 0.7 -> Reporter: Joshua Turner -> Assignee: Julien Nioche -> -> Presented with an RFC822 message exported from Thunderbird, AutodetectParser produces an empty body, and a Metadata containing only one key-value pair: "Content-Type=message/rfc822". Directly calling MboxParser likewise gives an empty body, but with two metadata pairs: "Content-Encoding=us-ascii Content-Type=application/mbox". -> A quick peek at the source of MboxParser shows that the implementation is pretty naive. If the wiring can be sorted out, something like Apache James' mime4j might be a better bet. - --- -This message is automatically generated by JIRA. -- -You can reply to this email to add a comment to the issue online. - - - - - - flags - 0 - sender - "Julien Nioche (JIRA)" <jira@apache.org> - subject - [jira] Commented: (TIKA-461) RFC822 messages not parsed - to - dev@tika.apache.org - diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testEXCEL.xls b/solr/contrib/morphlines-core/src/test-files/test-documents/testEXCEL.xls deleted file mode 100644 index 86b291606d098e2c636dcc3b72500c5c237c7aa1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 13824 zcmeHNe{5Y<9Y6QI*Y>@)ZtIURewVwBA6vIB?duq;2uwoBC5P{^W|cRl?n>((3F?PE|+j9_~}-xlXAn+|zb; zW@d)wG(2uEx=&CIe2BcV{Vuxy&l=dF#_*d^zS^utaV@EbIF#q?mSPr&*4$i(X10ik9=UU)r^$IbiD{xD+9$Wr2_Wi@J9rt8S;bH-I7kXbw-KX|Q zKfF&H=Mgrjl*bg!+A|kwJwQGKI8SXt%KOxQbqDhBC2gEX$n;qr(+B24%>#5G1Xig& z-0#D$C}Zz7jKU$Q6Iv$JFz!dsBf0|Va~;mWGY{v1+9;`iHWr-8H~D5Q;P}MyuCnks zc;1*#XlKTzYUO@0+6Ea0+_kbVu=@3Mo#d13ncY5rmU zBc#t%I94u{n{WL`)8*JS^C#y3&Ly<5j?DPyGspkMaU8AbPscrmp?#%86VKb7#fnoU zD{!33;hlQ(_?yqKxw7u*XR>P5>JNU@2HPokt|^S~h37hQH$2yiL3sAF5qJ*I33#q6 zUxw$J@+dslpC{nC7X2KanSTczw+UDFUpnpues#?P(c&{T>5|tbBtk6v`j+6p@JPG6 zEx6kbRSO^!Vj|3tfJcIHe=zU|iX#IDMnosv(f}}bcfR^N_1O3R`oSH0>t6mA>hFf1 zz10TUH{dxgS$|vMnXO)UmVG}w;|{@dP5v%C>**DE*4q(yX8(0~Q*Z3W%s2Ve@tMDB*lOVOxF>^_ES`02 z22Z@>)!2xj9&7lFEvYVvCq1nzw@jTzES;ym;VlN&pxFm>d0Wx?=u)bXUyCu%-KZ{3 zG$-X%J5@JWI^OBLDmuCnbP05B%jt>s{d>4;b~rqfy;4>4qMhhE4(u1?9gZeLe}TGu&x zE=1jEQO|{_!i^5~a)?@IQ7?z6j)>Ng5OtG99SKouJDt|?5VhW-j)$mqYaMDjM6o!I$K9QV=r()8By^WX`uR}Fcbe>gvBeyf+3=hJ2EN!RDDYE(g6~ywiHYYQ0c_YvLXEwB??zz%}uX+a>!e zmZ1wHl;gBqo=v>t>NmVb?9$4YcN%R-r?m~Y!Peb0=7@6X9s`%iB-hl{v}fw#+&pWp ziFfImx|;S(U6kXRc*iw$HSL+YD91JNhRYgYuW3ZD={`?AcS~i(%ge(>&8@G%B{I%! zw%q0lTq5J#4$H;v=Dc|(GS2O>+@1mm&iExgyo*7z$MbG>}Xgxt;GF^+ijNH zFPL>-?*OQy-}23V!|zb$FI_{bGQG_jo_)rJvl2d%f@EDZ$y9 zG`7z?cS?RcTnT)5)aFjwe3!xx8NS?j9Z*S}&ic#I~l7rzLCFtM@y9dOxBs zx^rtFmqqJVSE*lLbG+v06mHQlZ{wZ&mCR9j^Zp|!yK%CZQtIGAHnsoXaov)KqMR&G zVcxg}dt48;w0a)lA~lWO`#bl@_r~<)%pG=}3>zigxBKGF@t=x;Y`!e{KIcD%KW?gwLXJyIQTbL^P1iXgW4G%zh;O79Uz0HbkK! zw^gmeKL=dNLOGj*uxz24HCr*4wNv4@z`xI+t*o2!X-C^DvA=I3&Tr=kmN570GWOi8cJ5ClKU_-1>2$Syxn7zhcHVkwgNhgNYDg*ifh`=aGD{2XU2 z1zK9U_!~>d6WFOO!`yfj-+hjvb2h)`sWa+3vVY?q4&N;1|H%R?w?6;JjXuEdj-S-o zGdQ>OM-R}VM`P~LD?fu%xvbg_QJ(w_WG{hdS+wK-ROp6ybf*`=y4WJPP?m#QgoF4< z$)%wlA!|OX&ivpnzumT{cSFC}Tg1Lf!mo*C5*XvIL$hFeqrQrmqnRbRXVPC6^rI5K zsG0@AjjTR8Q|g;nfA-9W5C36`fZm^Oz zcB`Zf^>dDm{UV8DiIO%RkS1-c&!mkXUXwO9O47!Ult~-Qo3yc)CT*;Rq)i{L#OsA0 zMw2*x6iwRLN*VP5s|RClvHDOBd3Eb4^_p}l4{ga`T6w8zsd0_NGY(@pd6Sh!%Pp0b z)~Gc46Ytm>i`B!aJnlUC5Po_?HTpsourdx2e|m%NNwum6ps5B4@x7gFqr)sNmZhFs zB*Ro)nwbxoO`b|qy*i>^7h<3eD%2T@I!mK-$-bb~Ya;4dr!GvRUK3GocIu!)ouR1n zI9jVSk6Jw&Q8)QQ-i2w@vk~<+rw%IA8H#$=s$qqQ20ob4JkWiz4dRy1Wb1sB>Fy^8Pue4l2|ciuxj}POMf(OR+~>cTYYbxvLFaq6H#ouR06meZQ`5zQ8-DR`sF zRV>W0A2>}=p~+A*IZJBIhKS~Brzv=&$ycvXb8@fK1QnVLMU%6v)@+Puu5p@zH=2Br zGx;2GnxI0HVHM|A&eE5nb~z*Si7m!9`5iV%OEGor^qTO8!ma9_S$;c}{@t2tP`{9Q zz6(0WT@4*fn3VP~Y2q&Z9zvyVg8vw98q(>s*({iAPh))3z>;VuMV8yH(87xpSw9=# V$GXVKswt|uUhpF8>iG1A{|35}>oEWT diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testEXCEL.xlsx b/solr/contrib/morphlines-core/src/test-files/test-documents/testEXCEL.xlsx deleted file mode 100644 index 8d5169f84108cc4af5dd66e2c0957eb52e6622e6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 9453 zcmeHtWmH^gvu@)SEWx315AK9OkU*eu_uv*9Xxs@7L4s=&f=dV?1P$&4cWZ(K3+^uA zbY|u|nPKLvb=UoIf1Epet=@a}?zgJeG!JQ6Yi02P1^003wJnGWYYz6byS zH7Wq`0Pqz-U&_JW#lqgjNW&9i;cUp^VF#wpnMPpF0wBWI|9ku|vp|2Mgi<>fp7fdQ zSDHg22K&fhycaO(00s?29OG@d2NUJ&G5WAsUO8)IX5=K7Rz_8Vf!hU#F5WkLAfR2V znTcX=S5LUKah_&~ZvOV_g;!0#!XZY|&omS(+3J zS2@nTo1XWa(pk#QOG;m=bP(P~$!HtsJ!lCQ(jWs1_~rqcRPfkCaB*|!shY+f$CB|4 zIZAr5>G6dmx%UP;*TvGRI5qKaLT$kn?_;yxe@$FT8aE|Q!n_^9Yk!V}Q!V*jg1H|- zvN%EJc+m$dZF(JVZYNwEX)dU1`SmFRZ+7qHc`7lEH^t@539n~`-_47i%W)0XM{+Ef z&R^#Ek;fbJYjA{5ydZ$0*oP|RbaMOI{>i#2g4o8cr*+X^wx-M>p=G%vn`hkXUmP!# zo$fU!yf3Yc~S90@R+;T zwI|ykwQ{v-FN>a?U#}orPoqKELcdE@6`e~jes&s+6bLe#9Xb6z^hAGtcg;qFio%pr zslAR_qMX;%d+LQp>02$_EK^!58-i}K*Ayn&bMs&$kGw>v>C{!xr^raMy2NWbQ$Cf}752vytuN5gywTo?T|U0}RKus};K8}(Ynq}I zC7$^}q|a?t$`Y%1XtpR_vQI9E@F-&pzvqh93@MWlW}tk0TP@^BEB|U(cJb)(LWHyU z)>Z(=TkGQU#=*x+g~8tx&Cg9*J8?CP;BNan){jrnM9AS;M;93YAOoNycz`*7hr7Fj zldY+PgYD0}^9S^ie?lL=_W$prA#n)Q!G+hg7r2MFWZJ3!X{UYq>gDT|*%8v;ocN z1tFr+PZLjR*K33GaiOw;qUBkpvi_${FIO@WyOh*Sz4@}$bf@H~B(;y8ze{)@_uZy$ zb+-(afLGCt8KqY0GlCn%<>Ga7b?G~qZxxzUK(d!zfiHF4X`|PJid^`;5V)m%-h<=C z?_EFcze(o;Rd34mbt!OhS0TrhVc^F+k?0tAcy`E;y4}V#bHCqBkz{n#6$oIPW^3V^4u))>^Z$X}p^&n-@ znzQ5MuVWnA-Jk?z$aCFlS{v^bUM)(a``PWkkSK<%fG@$p@6XjzYa#}lo;wI^sGn1h&fU_^f4+5$i8 z#N9f%M3rR2P#@Q)k~Cz%c}Rl<@M!75U?H#~(zGP&WFf3Bp2EFXcD8^ZNEBP>wUth_ zuT`NrvnmUja)QYo>3GvUmNyG@xScaeUT>-9N6R{n+e;Dby3bV%tI3aJrr8M)YvLRW z_ag_&ro%bGKbX9II%RxJV!Yjh@(tE05;l|Pk(DhQV&bshGF-up{r4!}C}5}!g!4*u z@}J!DH&?h=Ti97}{&wd68ROf!$_}#v1kJe9?$j<0c1+deXs!{{<YLvQ5CDdRD_&H6nQVDlNWd$7r?J;P2Y zD?Pr%N3mpR?M?`Y0h$zXY(mo4_n3yeU$Hj)W1u#^Mi(L#!d$rKlt0D(A~u<_Pd;Jk z>pjp)9@{CHyniz2Rx|6ek7|A}z~FNWD$#X6KqcTIQ%)I%W#9OLM8gz>X4yj=06w|t zZN$vsp^xkMA~9kn-swc{`*K_YhQ&h(qvO(<%maPHmpER_1ey*29PS&WmP<7mkc~FP z9-}B~%wBP|#^Pgnm1VxX-Mmmg-n>Xy4jXz+^~2?QedM5>ik;o+8^`OQrM8p>ztiti zRJl(w{f-U~h*v#yJ8m!AnVPS!J%#ajtZ7CfIp1ENZudl%iQU}PcQfqU1w4~JkCl0v zqrZX<+(&oe4RvH-qH}B_Y{yf#Lcs}e!`R5g1TK~eTUylOI7Yi`iytr}J%2mrZVn7K zEwIM;PHwUSI1{o0a)7WQQb(*h9zu~pPHU!G9YA9rov!*wv+k-!T|`Y@EKv99T&$4k zE-%GHir7@b#Z=cRrGX0aiadFAOtD0fZZ^n%X(=1`KDdD^ z+=*R<9*%j1QPj!GqW6P>sqU|(9z4oqp-76OSPCb^_IptBQQ=Fg!MU`7GV8vJILWg{ z?zm6j0Xu|Cq6|MIsYX`UD@y1y4mIXS?75x~oS#44-M10!P*n>B4hylFK8aOFSB*OT z%o?=kC@6I-a*ojbaLza7e)n7Ljy&_Y(oMAWwR>l(YEhQ}>OA3c zbq%(@x-Mutlex77a@*QgA7phP7NBD^?=e3y=k7Y?y|uOb&Opqz%8)>(sIkuTqi^{} zd~TBBVK9$?inEM8XpBjInyt&1WPGb1A#NqCfIhgpvR0&4+_(Z2A} z4Q+se9NK*bgSMH$dnu!n9_wRCqND=HX|0n;uO=v|X1}?LW&p9Bcr5!c`Y(9RDnDRk zB+XnVB{CP?JnDa;SH$aFLOP;Q>|DFb8ibpo8PxhUeK>@A-LR!sQ7FfD5+cwV6;(C2 zgPQTV)8iYhA-&XN#^RXD`986Cq)Zm&rW{DrZXln$Ai1H`oR)ow%`z!S zP4a`ErRDVp+a7aCA59lGlc%$TO$GUhLj?Fi5T9TfFe46Etx*{3I&Z;!dnN_xVNOU= zL8)EA*QE?b!vn;=#cd%=x$rTk5_P^lb1Xs6j6UOe%%uSP{7>YogJltN7g%cfBwe?> zn#X>KQz%ujAki`Syxm-;$ts;5u$^Nfq3i?Al4Y32IMXH2bNRO?QBK` zpMqX{u`+NNfPr}xhlxUSrIM_0{6_M9@~&Yry*x%*IBWRB2MUw*XX*tY1tf<7jkUGYV`H$Jk(YUO(qwGd{B%f~ZANp)tK#vj zcVxu+D`}|a&?m%H!s6d8i7ufop9xk!kEPtJ#dToq(pzo@ zMXpvWhJfr7iqyycD>W1YqovLN$R1vl8K=B=o> zzCY^b>!4Lp&*FSshEkOhOd?ry1jG6s`s#EKT5r(C0$u)F4442b*#swNSeoriaAL7|feNsR)82{Q=eZo;2;8{8sJZ;#D8^55t+a zxm&iHC}q8xU0$kEJzt9GFa`I+uuS^2)V5>Mp+Aa*Z5E&A8uQ6})Xunc7&6e#HGJw* zY247vexW{7O5=OlnCEnJ(8>9K>kQt1o$>$384#f-$%aS(z!2vD7#Y7Yx~OEu7$_HB z=w8w#RcwqQzp><(FvOA&>j_F$%^O$+v4Xff7m7pm%^8q9C_qTnfy*z=$`XuQ(R}O7 zTFPq&3)Dnt!$Q-o1nKB$bOUa#56&Hf-X|xpGBA`_`~MgvnhV=o*jdmy8eL0t!g{P> z=kGMQ^~SD7PDwe>@MF5x$N2KKr$P+^&+1FbL24Hx#g&HL2Rjc~SjRo5`EQqlEOg_B zk>sB%S1!Ft2u%)_y0&{h&W?FzgJ4TbQS ziDyk4h1aq=e)kX!`qR_EIs~^+WJE7u^@#ED!DPz2roXJ33hDc`+)Gmj&rqd zI1v~vf`~q8U0;hoTE?d(gccOd9!xKEa%8Mj$JVFwpri^|Z{HY1gd)+-g4Y=#7J&=*|_ANOxWV!4y{SW>=%1b3zhh4rk@>#2QtPmZJ|AT4&aKFr^rgDRevVGf}Vk`j4t}xxT(U1-da;KktD= z^$N2w?KRHUcaq52Un2WH9I@si$5D!zv@4c^PGIgl-!kPIUw?perXPg~-9fYr_rr;e z0JOsP^(RsGSf``e<`h)Fb1fgx2H-Q85$=UjMc$U0?3d{Dv+^@X7|AKVKE|?nEj^*f zFwoLW7fMjFA(OltQla;nW+=+pEj~s#%AdfJ!0MZwVXAvIIc`%g$uhB4mdoa|&oAOW z327;#3=wKyf0Z!9&9y#v{<$|A z#40Ppn@yd2vOZ`J&-j6o=!!OO;t+B)OtPt1FWbuN4bvL3g-kFCjtIs&Q?iC#TI9>qndT!XBR5Q@A`N#3<8zGWH#f{@iuT;mR+ov{ouj=hC@6g1;w7 z?`_V~h(x>b z&~Y6(KZbKQpuBtGqS2VexDV-Kw)!Jhk;w9J#f(ozlU5D>_&6@#$4m!wi#T5qx;)zb z9)n`E&B?53CDvcqRGbh=C*4YX{k2nk9`U%%mcn-I1(h_zvBIl2s0C~ znUuAOjs4FUGLM`4*?SAwLs_8dZ-gpM?0%>;lx<-eMnkvzN_40mPkVVf7?`)Q#*wzy z-`^wWeNi6jyM<@9_ozh*MWMbmrlQ_$|Ki|>3nG#Bs*FX$81Q-%J>|ziHZYNNCC{?k zIxrHbuvORKNEs%lPl)R`^7=~(u2!iFwl-x-)&vPtskW_$p(b_>AK=o>I2&mO0PR8A zzYP?!?}kPLMUZ$sB{Ji6EPZ8hVOxW8On7@VJ!5X~vC=);zXwhhYZGZK%I6>&Mb2la z;duAuEf@f?VDzj9nV4Ul@O6Td$Lhj2wPx};Y!HRNZ5&Y-sw4 zG}-~(9aye{*l&OP9A@t-vrL!a+??S|1Eta+^bK_VG$~9g7ePDRK%|x;>^0A}Od644 zrT6jOWNa>-5Fh7-&VNo?*=_BG0^3lDTmg##>T_iEZs0Ze?fZ~vQgv`V9Ch>UJJ153TSI$}L92ua? zT#PqLj+`@rXG@D>Lc%1_0=aZYV!9QkbJvsF-FIDTHT6sW5eksa{IV<^c?;I?U0jNH zvg1Arp@aKbHA?mY7ce4R>KhI{|H(z~Azhm@L+g_u}wXb+8UBneFM^g1;Ftd|w`0m})lm3oJc>gkSg>qB8e!ZEv+-GtoS!K4iPi7qnShd7f-2k4;ZWSjU3 zPb#Y%F{(DLX9Dt5d0d)3z*upOWWkh$cyHtekxk#v3)qLkYGmb_JB~y70on0j5~!5$ ziIVM^+yV|N$A@U&7WB3@{-wz{v}@f`#hjOT3rerhLgU;!JpCJwj#w7bTw;lCQ)EUr zibEy17pR_=Ojj~e=n82s7<#l9x{>c;w{G130DYkjMP^u-m22i3ryF{5A!e=#{TOt( za-qNew*Tmy^Vo;G&l8(-bzJ@M5b4+f?vvlC2?8QJoV@+>>lpuW-G8qC@N$NV{9hIP zb&C8S!JpR%czpb6rhHfMuOqoX3%bEu`2X{O?yjD@^Os+mM&NIp+?mGQ6}~$$_$6!r zFFWDFcjpIp72NIL{8E66|KI=qw{Fh=xt5Ik|ET4+HpE>mcPr*!&%y#{>wj3{&(HE( z{d`x;U6TJxO9Sz5TK-1)?`pV9czA@y0QORyp8ft;(u)OD)K0Boc^Tr*Z>e5>d`bmzx@}<063Wd diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testFLAC.flac b/solr/contrib/morphlines-core/src/test-files/test-documents/testFLAC.flac deleted file mode 100644 index ccec94717a47661d02153d1287cddbcf0b76db23..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 10604 zcmeI2{ZmwB6vv;tcU^FWc108zF;>F?CGbMN;RNi4pn@XGE);^u%OWTsvn-A#a_uWj4$KO_!I z+L(D2wJlw>##V={f)C3_=G86kF~!X8*#BVd#E9zX-99J7!w>otYqTjHn%cs;iEZD7 z{9a+o^irFREk34$PPdHeg_^R97d^as+&db|(x+C>JelAWxAUi5ufj%SpeAh1{_1XP znXcP(wZ^BXY{l`oit>x`rFq5!)&nN5r!k8b=#AISWfwgv_NAC&d`#-%_-1pN?qp%r zUQPY?6I(Tl<8=u}b#aX@VY1PG>MnD9#N?Js9gFWr6)cFoZ>=iVjMl^)hzTz;*Lt-s zR(GVw-uTk}xMr4D+Rm$5%?)dv*IkpT_(-KW;24hFU)c|egbM}IFu-%+2m(lcKOBkSKoT)t6K@34comX`;n2a2 z14+n32!0ie4m3FuBoe{Hi}76GQ}G`FcMgRotMKDII{y1%E?`8)@kfP7+?rrZ2KdiU znBo1fCLkwR@f-}`7C0yq0;I7LcNT@#B(HU(Nfpi;cE_nlt%W?V}4K=9*njYZijEh;J{ zu&G?xAem}Aa(%sxxHa*=f?~7K0d4)E{BB4d`_#ov^oZ%U#_U?>PO_``~s=Xn`!u zo+&(kD7@xY)cZ%O`YTJP#D4y5M(=V-O{eD_RqXaF9WIAfR9vZ>=IZY9)|{~$9d>k| zneMUl4d%eiqp6N{&S|qoZGFW-w zO};-wcOK z1uC(KFEOwNe6C1Rvg1OaVPJ4KR#5h=c;NyeQy z6_9>ycy2LpCmZ1#aUiywW{HH`odO5&!!1M;cPcndFv56<3XJ195*K!WRp#XW3oEhs A`Tzg` diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testFLV.flv b/solr/contrib/morphlines-core/src/test-files/test-documents/testFLV.flv deleted file mode 100644 index d35e9bb6063f1ec353d27e0cea0e2c59de8bd3e0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 90580 zcmZU)3p~^N{|Em0Y_knxYbVDl*@zO(w(d8vx#U)%3>CJDl!P4R685nv2d6{Ay0VmF zuIUtV8;2AXU3AigQ7fsK`&{>bJKx{q|M)*1|2^{PvCrkb&-?v;U7pY9>-E{O(*st6 zAV?kjL1YLfEc_2XC`cnDc$a_pLED4j2l3zl0mA#83Og8nEF>7R8Y#lBS@|srQj0j| z7k(6C`Lh=8fgtQr|6>71!y%UM|Gq<7r;qvhhxmapz^(oe>yNXK zk41x@W&Yn_*x$qb!~8>FFrEMI{{Q_L{>ZT)|Nq@Hv7Gh4yTH=Gx?l-j{NEs^s!^l~ zqF?BI3mw+aKno0*4XuHrYiJHwq+=-z(J&AVhvMy27w|sVtSYs*<8yDV*I&j6i-m_F zJcN+&kaQ)cq3U*Ge&_Hv(^}ueYhNyJGw}Q2jT3`^(KvC!FGAY)t(^nP8KlTn@5|-e z2cC3oTjlqo>Y|XJ5N`N>P zWc|Br#E)#%qYg!GUCq#8mYKB
  • L zP|U-Gl)Ri&eNF0JA9t@hKZ}4ya2O4r5d}{TMouQTeTI4!W7RKqKNYyYFu&X`lWr=+ zz$4`fo-wjM#*!L6-&07|+3B}*RST1+E9ZdQ81E42M(A|B<(5AwSk8U|l+V=Of=HWn zvz(L>-S1^YEZjDbAwolRQcJRIQmhYmf4tIqxJ0H;KzlK@c}sM3lXy}**N(54c7Ss| z1MAMbX*|W^q04GW(Ar0JaHE~I<=7c9Q=>Z@3vF?WTpf=gS+Q@;CdU5cog<)+uy}Sl zzo*BLIH7lc@?t1hG4ECaa?nD&zN6smORp@aa~dxh*EQz4VI&L-S5j~g58bHSNXa5V zh_(Kifm$iMLNd4`@%ag%g13bK78n5TN$WOfcwT;WDOtadfGQXBcCa?aCtG@?w0_8k z-+Yet-agkTfnjg*R75CzN4JmjIQ_9XQ8edL7X z`r*hii)~+z#p5a12exHAbU!m5<*>eIx42S$!S9~ti3AX&VyvSsA5ofE*d8>Zz*cBHfz;1z|I6zJL{U@lYG8zfCELhe#oWa(c}p6QAP!203?sxk)oe+U ze{(P1(UWacX$cMOyZD;F6(7NgY-KUEY5K5O4}R@ThSnymnRV4znc;FTETrYn_7i86 z`nOUOv*!t@w`m0Qu~3{k$|0SAP6`!i#$(WoGPJ>Iw;%131HF%AJiT~C8w=M7Aw814 zSh~?glFOR!aV3xwK5^+41t}6;Zck zmOe0_(GhyjDD{_h?&u*Y3wR0!>pYam~P04M=5BQx0bGL7}Z~qn~1rAb%O>K%FDdTO42LZ@7 zcj&A50&S?f=H|0Y)|CWIhm9Wv8_HW|?iKB$$y56c4VeVd@ju?3!XR=;KfW%Cc!T@XvUTth4PAr4^aq3Q^(q(=V(_sNu{@hYT!IF4O*~qVW zo>T)3$t>wuSc=vWiI9{0%rlMqi|6+H-vRcx3=2n6U<6DKjA#n{R4<{F*w3O8CB^l1 zs@nly^PYLXUjO^2e;bf&x*r8c<1L-PBEYMG=X?YnJxA#LcPsFA(pjrqu5zr<~6ilGNES_FsqpOHNe#mofy#Z#) zw$^PozkR-YNwtT@kPv87*PSCi$x{>nWpsPg&AEXvpv> z^e%(MvJ~!|m{?dy4XIQ;36g3+e)tvB#%?CDy(>nNtetA}Uad59_u+(}4?{yzyxIuI z+Lfs_Ais)3qu1rU>voSEb#RxZqN8Kz!BRo6He%HpKQRGym4^JjjdET4 zo$fdJ6qv?Ci0wj%CMz>mU9k4{Yhd;BQr{=<^sIPf783zrIgLK#ohT zUL^&V@z8~#2Cu~e68rYt4lL>q3Vg&GwMdz+wW$vQVL?h4LR4$`l#;10d83blu%f?9 zii}rAyG1fJ5kzVYoRH#3Vq>r9s<^7sZC>E$!a}SB=5J0wC*M_AbL4>uDY5ec`J#0h|lJPM1QmK}@p%)w20ZrWsk`l4-KRuF`*5W#ro-nOkW}b#2Nr{2 z<73RDvC!WXtb(^Ra{-%K0NY;Tfw#OkqATU;382h@n-l9xxR1WwUrIppn3_+~kbPz} z2)$SbE%OyK)30Fwa$p(-FJm{*9I_H%Mwt^m3zhPt#mh`-0?ZBz1v$Cr65HYtA~ZZ* zdd9FGv&Mw4*=go!>q8%ovo90i!Cp#zEvd~ssaAsFPrBzj0y;47wB4^^kq?g#!pIUx z%Epk+d6RI>4H~KaSoPm~sG=?loItxXtl8t`)O0S{I;x^>#!$wi#3Lu@0KctJIMLH8 zYxq;&ILe2*^QZ5gU~KItpl-wgpEDLcQdE!ZN*~<$=+DPO*cwa=i-y(`(24mtG^FqC zO|nfkP93;wMC%Y3$2evBT5LU95}>2w2?WcM-wR?gPx zvgx9qjUz*Wmz?!r0gI^x{(dEpe;m!WQb77 z54s~`=V}gpx(#;mSKxJfkw{txdz^|{WC#d`6`97$dO5ZDnJ4E>?9&Yw{VGUgoQJ|ZTi=;^|{?qP_*;Z3OH96s8Rs1=Qgh^v#VXub*J%BkOS%M{= zPgjML`tG|&QOV`_ZZ^Z02TqE6|Hb2(-h9C z#H;pIQ>?n71Te%@rgO|3f=HIi{+Fvn!q~PQ^SdJo!tv)PkbMO|t1# z&~&3y6Kd#b{P8^^H`Ce3J8wHjil+4b05BN^&^GWrQ(sFm8#zg~mo2kBYuA9UaYoB- zw#hYa@8D=xx9k?+nV4J$z(lq2fenN>Qd8|zbm`L@n%(2ml_f*+*|eTn&F-c&c86BH znuLH+W|jIMAr9`_^JMW8@wFT>4dd2gFF4i9vYFoCS+Ql{L;(_Be<=m5y}-*Y#(eM*|}u156D-U%VyXdD1m= zhJaJDwl%(8HQ$Yg^t^ou=s%bm%~NvP4J9tx_ig^-U)zR?X9C{*yHiB0mjU44- ztK9AFL9$03BsgLr01V?-NCoiNVn_h8VkWeetuWB-WS!=OM&98c*Z5T5)3XRSk5!9! z^}g5K83V7h6VEzTPSAKbz=7yusTN9&;ksRk)17j;11Z=K@+-Cl7|cK{iDp8&Dv3um z2J=SO$M%gwd|9F*!CUfS-o@v6MQ)N5E#_y5QqA?>uPe>hXBz>A$)kG?uavSNnNUH{ zPguhkXsaXLyZL@j624dUsa~~tOv>9T-3)S&4LPx!(dm@YHnmn#pVyf+Rj_7METst1 zEwMq9$Ywx@Dcj0}7|_%adNhUtDZoA;zbb*ZnBbYTST3^Bl&agFWVjPKchj;o$M@I< zh=~zP@d6kmah7zG07^Hq9$wACGpjs@TD**&2L61kT$w$meI1YHAOVf2a`_>L3x@aI zt4C7X?&!=6)b+fXi-p)^ITk_;ps@0hpI}v%f;1enePY&uoTx1DzFP6|<5P<(MHG}P z4|^v=%EMTzRUL58Gdo?O?TQ>_;n6Yz0N$}EQf9A_6Ba({J$v@WPN-KUC)Zkj-~{+> zn4gXchNJ-2eIP)8q!AzmmI(Gqo26%LmP3ve4*%v^lvY30w!i$lIPpKgKr=!^j|u3b z`WRCDu*pJNRHMz-)R1N^1P;@pB&W>iN&e(UG!zIlB4Z&$_Ze2=FpNTl4K(hlBBvC$ z6MtL{Dh*7GDxR6&vrXUR z{;}UmBQWrAL<6#>W|9q&IQwtD+sYt!J^QjO8VnG~2H89}6!>YJxrSA^YHrg!a4|r| zj$+hU(se?p)k-QHw{|LvcTPkQ5$UacW)XASRG%a9j5jo*V)SYe^ml@ zgw^*i-(pKtSSYNSx0C=IoQxUIWigivhk|$x(@Dzm+E=#^?aYXceLnL=C3CSt;YGFx zB1r*_lpw&7d&hfb?fP6E7ugN%;l7fr)iC6LCwI-6AXvVV`)lsEQe@6P#dI7 zD?Ct3V}fx3;HC>veR#ZplA*QfJ07GZK>N}Fw#@+HX*d##F%FE6*QSb0l~bJL+wmlTAA19;1X!^kEF9NCB(F*+6BrX)n#cI-=BG~c z%~tsWRD^{AFIYsQ0Zwa4MItlhb~5S4S!Zh8M|a=+IEmBE4VN6G#u(aJ81Y(AW7JWu zqcXtA4#vT9CkyV-$0wIJJOjB?*A%cGR3} z$so-JuTK@CmUQ8qO+TT zrcGmLyk8S88@-q=ODG9lJi>eWw#p-EFMDSPIb3soy!{%$&J^qm1qQZy8B=C+EqRWu&YGBrxOQbaO`9ChfP{isIOe*t=N5L6fCl~;=`=5SR5g3wHrak` zDb10lg(MJA>{FlhQ(NAcYpgL;-UntTN`TEt_M?9E1%JXMDcc(`=vw;@_4!)mTE54L zP>?l+ko+hUVC@%xMQ?532Iz8_Nx8$$P+4|1atu!81YC%hlM?9bx$_xS& zI%|;HDM++E@9xe0HfGl9Q6SN0dyp^6K0!fm{;K+c|g!{LbfNqh$F%u4daLs=>;%glV^ez;Isl}o!KzkIl6z+3n!UJ_h zG!!1I>t@zVi1Xg2F!(wVaSO3g@3ksvc7? zBg1B66qq4l&?}_EsgwH)fA59jDxT1~dwPL9iC+jeunV(>glj;{^hj;JVRubxy)N3f zev7U}E9!Ke1D@OJCV_RxpG_UeIhnGLjL635OUUo66JuTt5%;0zi{Ia~+H+d_w~Rm^Ly(EQpbO9lsX- z%WyXvF$50;bhFQ-2IK?=aPg0<&HQ1dGRD^egObN771ceHGK2a2+7_Uz6ag`Ki#kh+ zTMZEVdUYC=hC@yh^V8r)&j)k0^obu#1px+nB@cmHCr46W$P zKyY@aV5KZzff##%9aW`3H{zyAD7{6QKZjbHI$TO(cuX`a9?0){Y~c9$I3&85DuiCG zudLIJ-4jojZm}2r7Bcy8EJp~fdj_yOBwat>!TX@|B)j&H&Z7`cX?H27V=zZa_jW zOmjzi%>e^8c8e3Kg5q&MDsC2(I%82W0ORddr!_`Hk?VW*h?mlZVL{MTjC^^ZWclxO_*!{8RO~0jc=*rPZzQ;dUF=30h_=~DDVQ&zA{Fl zjJ_>IE%wEI-m_#A!1wAh=@t~8O%{+c@F+V)mQQv^V#eaNCmPHn|3sHZMmF#C_oCpAcnV00d(anH&)v>QgA43jBz-hv(GPkPV8Pi z`}_1AX>lbQQ??-6^0z}6{kE!|kPGIZ^a49b$#!6wI#UL-aN zp}*mHs0Q7D5Xn5Y2iL^r>_0PrbPUIgc}5&5EQ@_Jc+G7g&f7VMR@UqEuoj+ zK%hYZ1UNjqHz0AIJX`e8d%X5Z*XjX)qeP}yC=v**h;QjYQyR}=wp7?Rvp6yiMAhu} zpz#oGcAe6dryh^)b1F+50K|eOassMqNy9L|<^zM1(=gt=cTERi%4moQ$72@q(Xuu% zO+b_>4T$kOykSYFZqHz>XG`Cnl&Q6$uRB;EVOJWMlJl7;v@FpCmp~+DiaH47wocP* z4{p>`Pf1<(wToD&1c_v#q|%KjVoC!k=u!xb{nl{j>QpGKf*OtPRa!oj4Lmq zV8>$?v5{a5Rk(9fpMZMIMD;=dJ-;JBRiP|$Jgm7s`$wn&MRXFdD7K@T!BmlokMT$ z%;D`i_@LNRw++KsU7zh~dQG%*Q>p+Z=9$MDgk!O6pXJfJ?s@zbgoPZyvpzLOL+g!o zU4)Q+z7w*fPQQU6pHi)EyZlZ01p#x0KqM6an8+FtgS@!K)!M^yy2t}=9%vhOnLFK} z%Ka*?JvsDrOOQ;u#B`htt>??u`&yCj{Wt)o|2=cVB|z_Ww!_F@lWaj4V0IL)$ZP?E z0p10|Djo?3rB6WRxj~gK-no9&~{X#Ng2bz^*@7;kV*x@rDv33~ct1 zI`S>o|fw5pUOsddg{&WWuT&oH3(tt)4IqyoG#iFH7RNYk8dIJ@%dFH$S^uyc>G+i{b zg8=>gfvi)(52qz8<9p6w))k1Q8-%b7)Qhce_Q`l^)*5-CHI;RoLtZT8_y9Y1ZaWtJ zn1byFxuyqo5aSfCtFrhnhlKKOCcf5qsmeqSFqvAQG>w9QDMd2Tv`BP9s%{JgXKG1q zuv@37sW!YMdX#2D6G9z?#g{V40-|`mG+8X&0)>Jo3Ci4O#X$7pkOg?9!HhL^{71nhFQfUYxb0=`!7`6BU%O+iDhWEy6=GpsAQ*T!sxe2(V3CWkNL^J zEzOql2)LfUy8vTwkph6CXm7oWTS6&l9uE_7G%e4nk;0zxT2FMf&q=KfqQHU?UmLdkyx5iM6112g|yVJnQ|E zM~+4nS|uR$E(oM37^1*DZE{MZ9$NltbUxd2=H>Y5L#Y-~u1vI8uOQ#&SAE@Roz44! z97no&w6cF)d&`?-0*=On$9CMn$y`a0WtL!(8~_7rEl@)NZUA-!3qSEu;E>SyewM1@ zpWf-rLzWa0BE=<`I4vb$j#{J9n^&~YfzsbQquOCj+2uf+y$ z{$yN1I83>G*6g2G(t?IA(vRrf=2?y{xqEZhiQge_*u`)$=2y(G_C{;>?74diAGLcs zLx-BUqxVJSt&TlM*Joiyq!ULR)-jEjw5Qu2b)0$6zZJjK~=Cr zAZV^vslc%fJu2||@%Jm2GuLZBrJi|ix@qBn?vV$>(KOOZ_bl~zCK|6on*AqK81Y*JuRTf)>Tkdx$FCuP2Jv;mAUa+}$)Vk$&vB4|O zaKaX6!?L};VebFdymRhh`o!){r7zxg_nfK4F1$ni@y`1?)SV936XV~N%`K-lw#Mok zdcBKH{$$)dvaTmktL9b2r#7f#KF>p<W|NgoWKl zsgGVFZjI{|ZiRldC<<+Qg>3p#27e!Z4;ug7(%ne!wB6rFum5CRLw_0^5uylIFDp0% zwoxn@V=%oMIrq6LN_p_SH*j#f_a99q6m- zU*XQz;djqni)hLDbNhDRwikU{xk+meFH34STh6q^>ipoi$KKaj{8D(zx@^rD(IDjO z?b|oLFFM$D_~p$frx@GjjXJO8{H#%16kh&rYtH#{>~~H|%jEc3mXjBvJ-qsSaohD> zirx)nSuda6D=*qUKq$PM>SCJ`pcWMzPIUC(Tuzd2)?eP~xprjN!@K)&t13+1>G)?J zuI7|}7Dra@Qx^ChnbGaMz7w?UPY$8 znU){Z$(mdzB2Me3)(46$C1&Kzkjc?>`bw7*XZa&hF|P3bTM@^i>z@vk=9k|)?`*&r z-V0+44qWxeAO5oc#ao^wq$(UKMJ0pC8TT>@ zRa|-Wv{iMk%PaqE)4y)uD{CAjof~}%m0NS}4V0M7UVJmSCVILk(&u;izyse_+=Mau zeEO=V#V?c>Pft7G`OPnV?NXn505!iWODZsabSgDnqNX19kZasys=rSa@uuhTHjW=- zatnPqUBQOr{*xWF>4xsU8w^#>wcJfbK`)klpDsAM*g*T;>l+yUYlAC!+#JT)e_qZ+ z@g+aqw)nlD3G3eXbYn?s(~aS9##lYNOk0%1rB6J@)@*-R@_vp9r#XX>$?lKyLCvUT z*mXKTxi&0*X!nK;`|Dvbd2hV$dzXLJoNuA<6H~Z(5|!ORemZIUfYnqa?yW^my%_no z`gH%rO_!P7#HO=hx=AeQ8C{9zrXGyJNTGjmzLMSBq;he(Q<3y7$0aP{OPysaPA;6d ziSm$StrPfkU&q2XrBaOptG?D7UmV#pYZR-tpzhzf`qIRPZ+dRff(sFj%*SmC}UbaoxwQ4>%fyRQQ0X8lUh-CycFaZT!R5O9tj3LKYx)sk&s-m)^Mct&7lHv?GH)I zNTINO7ni8FG22)-$uyG~uifxE`N57sRpG9CBR2Bpsi7GDTFFh-ljQZWgC=KZg>B;x z!_lbacvlCq4r+O_ZKj*YbiSwkJx6+#O)(3{t5$gM_(Y1}a)O{%hhq zEJt_EEkfoUuIjBAxH}v%o$LSC7}fN8_E=$3L8O5*&Ezw^p32dv8XBGOtzg`$B0AbX zjdY&wu3WPm!>Nb5tVW4{1Y}a+@zUS5C(IxMq z;c*X8Qg|k>$C$sT4;%tICHmMYEc}!Nt=S`Xm5Ae2g%@AFN$oUCmXk{>x|2oc*e+b4 zKv7^<1W;IQa=5{C0@D(%pUWA_6h%8>i=^9s^%+QMyVG{>rHl3O_~9^a!SUPHBqjQ2 zIv$E2AZ04!jGQDGvP#!~Km_k5xFv$pDV8ZKkgdcd;ax&g7>b;+6^<8rs64HR2WZb|;?Gf`VSJi9Q zMjUqIHtpkI0d;iO1tCR;jyH6=G51yj^=&XIDyXYxDD(S1BW7Xq1hHi1uJ7r>;N5Po z1x6RkbJsp;Tz{#gMGl(17}V4s88rrM8cj6kMMcXJyp7`;<@<_#rUk35NjCb0wMylg zc#Rp>%Rg}6_z(K-j(-C8VVuwY(>o^Y_x90|aLmh_t$hPiXGWe@SAR`1ZTU1+hs97? z%I}?Sk%Pzi)lO?i?!HBibh3RnH71rd6*}%Mu;K2F^3QU-7#YDPR*xSj zA|~XYoB5~vpyhHGETDf#s0}#=$YI82tt97`*6QFxDdl-pt7!|=R>O@UTcSw7b3Lp$ zI2pyVeSW`DYU%54an4FI8+r&hA*j00(tYz6Lx~|RQ=0MYUip3RS&{kIokHlAj|_C; z9NI52DohqEI>~PX%?!)UrmJ-Rs?C<<-D%KH(0FIeLVQbz`8M}PCrm$7jJssU4;6M! zoGz2`92+&t+}>@Mv_wpqUY&nu4XDU;ENj~FG!W%{zMEE8`oi};yzX2Z+;H^gCzTND zhVyzPwf9B1J6;*cSRa=C4f3t zk~WP*qFL)5_txn*j?6w-vo2F7DpVN@X1qe7r4Vsk*}(13dss>kIp_Gs;HXn~ZZ_g) zTKg{!uNvqL6VxHgi)^w*c+0-lflz}@@*hZufBT0qXn4vA@d!0> zt^sVag8<#~D1Gfj?gWa76*b9al9ZrXRm)=kEhD?areP9t1kz=WfG8uZX1>n3cn!rUt(%|;SGJTaFr`9t}r{< zHnlr=Q_N=t4+olCK}{AY??>4TGA*MhwwV_(ncQ+Y;yivfA!1g+$~rJ?>=yI6BG)ec z-Q6-CmW_Dg7gJCq9_3C-N?V(h=}7(-(R3C)ghDForX>RVP8YY%V6%5c>89wLa?c6x zwzXa7+q~MM=(8E5!GXk9m88hsx3E^vG&ax-KuSm`{ciO=X_91 z(k=}8Geoku-4QwgL--9(WQkdhA+?e%{Kev{#~P>eb4Elm;$G2 zTsLDLvzg1*oX&N=vaR8InG+*;bEl-Zj98{DH^MxIc=M(y>djIMEMnA-Ms-Aqn&)-B_8%JW+$OZvUaO#xYiK$pg2Ao0*h+Z9^WmvYuQi{TjeVy|UmZW|8IPndD;jk~y>|K(Iz zmu!2TKI)w43tsZ&!yuE*aK3xu;=QnOy?1QAZ-y z^Er{%xaT3Mtt9?FOJG6mac?bp+{=g!ewsDXdwG9kYO+P%o`<)IR;pnOLoCdqz-2_x zrvh&^GGQ?Cq|(h8DA9RV^75yLZ{e%DK*yP_J?(B|$#M7RIt+qBHxpwBa6&xB2}3%I z3#Sm{4NDgj1q4QoLxFn7-YpxHOm&-H6LcpUlDnI^7dg>$ok~TLI(JLz^k0b+>CnmO z@}UH#Hbd3ccfqt%|AJ4&Jei@2xpD_|D^WHR5hGAL3iXP^qjopT4C2N>H%I7MixR+e zOQsL75nZYoXJ1Z#|oxl(3X}Iuc7P2wT2dYhJfsT;rlfph_<)( zi@r~VO*fyizY|ma;|tl0emUy#c9X`U(A2!<8O=N@->_JIv9`7=*shlXw42m?91@XG z%1z=@F**lvoqOuxnK(;y^C*AFpW9BxYIXK^%lsI+2F^h2}hdNX&fj$Nd$aWJTR z6$@X^@UL@BmOkO6>(1>~`8}B8^e;}n^1v&wu#11;zHe7|&%Ptio<+w_r}qM6u?P~A zZHqfBOZLqS|7!hhZfxSqfe7Ls1Fy9+LsVBR0>6GOJaOj!z3me*i~XDbxgTMrat`?Z zyVu*9@4{h2Yh?*Mb#!!e{`u%NyXv{k_`rDrW=Atmc*L~%Akm0==4Zkh`Gb$|rz#>t za^7xB_uXgT{NkaSn*38QUES_=O>{114b5qopLqEGp7%f9=;LtdoE;MU<@rp{?8O65 zFNM439GEWsSwFUS;Hmg@&^@EzJihpXdDd)wP}W~vx72!3skhrj!Ri`)ktOdBRWYV4 z{P?{_RkjL!#@Ef>W$ROuDl-V^P$nvePW~3kM)#1Rjp{6YiIb!t*Y3%spoi_&Q;dN> z%XK|Exf@GI{KSOzaAig%j8>1Ohb_NwVPv$LPthNM;-><^4zDE510HG5HkESCxmc@CF4WyK*$HiS-G+%YP@5aa4 zrsKQ6Ts$Fpc*DJzH`qmoV_uh#WlVxpx@o#Olmf?3=F?(`h`*|~pQJ*LtH$?k_p0wT zhy_1`S@aDw-9Qm*YV33qAe#6E|#J3Sno7c1W6 zw@1A5vFbYWi>DB}Mu2-GTKXJlrT|FEkYXKM>!vf5N$e@#9 zY$3Iwe@6aKWmQDqS^uohYYC`OCI-<5x*;AqO4E%Wk0aMB1a~?;e z*2Go;lxCEq5&Nxg+|5;FVHD+AZ=&yB>|W2vu3hdaM8;(TAj2~;*1LuyX$?k>N^3GZ z9#1FNstz2wZ^{`w>^*t75uZ-7MYI^0MdnVIE%8>Wlnloy`3aXklguHPyE}i9;}RA3 z)?h6hG1^cN`jhJA59NeNb#0>gYEePBgsJ^LAvvDlKrcTQCyH+mxq0HzkYV(zKPG$T zB(>Ae7>jX+`ZA!Z(j#Ciu^7PglwE2F-`B-(*O75HiM=o=P534N{VV%mwwp$ zi{!@Ox4j*N!AB$Aeh)XkDNbDy-g)Td21`zGs?m(j)}8Ny`T|g*iDYL#7 zZnOy#iM{8xsNgR@CpUWP_?cC^6}_?ew)DWKX)Dju?>&!v*nh(8-?H05Z80Uom}B=o z1cwg>4;F9TX@C1u`MsfAZ*nzHJ6udSdc4Rtce8!nlo99ZQIq+1=}!*p{QIVjQ7sA` z+?jd4Iv{$cIqv|kV7u3Rp(kJUhkW1+Un8uJXY!?o<>qlD?A{xn#XEFNQ@&pRQFMr) zOG<3NA2d6VV*c#;i2%%~$xQ=%YVwiJ4RL2a{Pca16LEL<;dQGIeBY%}@$1P3s>!4| zy65NQxu<-3J0SJUESNxHfxs3$0L1?8qq{D!qy$k0SG(tSk z2vIq5%X^!W;%zkAt}~qiMNI_TRz08O2AB04q0&_6f44A`dr9qOu`<^eUuO$;%Je)n zRV4Y@#ImTqzP|p#jQX=ZBM))KC-&DBwp*p^{(P3*9cnX|GJL{vNk(xAeWCm-RF&K?~h(6~akccH*cLIbi#y{^bwO zSid=_lCOvB3>iQ>04(Y!08S1?M@0*cFYhUZv zbRFK5b?YxW`SbD_lhm=NNuE93o#<9$si}DJ+lh%QE}qlAcOLj~z8!&cf(^cGoAJ;2 z)0-MjWSJty5iw;frxlv{S`^8U&j zKH13;qoREyxRXBbHPd3>;}fDHCU)(u?0ztnJiODR^<37_#$ms=CtNb~ANtHrUU}nf zkD2c_YIV8BF#EWkAJ@e7m76Y|NH0o8RwLiQRhLkOE|A5epmZGmT&Qz_&g-N zZu&$`HIc9(us8IAO4R-SJKgbFo%?pT%~iGNkBzD`_dQ>anqNOxrg(m}tHM3bs^?nz z`yX?acScXGdfV!@JlV{8U)Zwomq!jQ-?jyI%CFII_K-i$n%H%!GII6k3-7Co-_|l1 z^8Icg7k($_csYijIJTgdJ9nE->h=X)(~ox8ZN_WevnR_n~W>^Ht{ zKVNTd{XOgWFK4n--o5^?b{)_pRjr?M$_vh~a_dU*_$%bjg#RKxTkcxJjKe5MNi)N< z2?hwitu3WDa>T)y6X=&l3>f$`JJUtA*74U+Hq{OE7ozaiH#AkA+aU{@dfe41mg0gW z<4}29vA2F7U3o-4b$al9>{Mp9?73*3ci~j- zY){d{_pBDwKE&v zRJVmh94l9KZFc=V?3MQv?0-0^6u%MyON$}Wd&pR5wT z&9A9-i?pHZpTO^%#`E2|#vfF-k35J1t*W3$9Y%O$Hu4KOupVPz*8DF(ZJnGwZsxJy z>zjGEDxyK<6Mk%m{uJnz3vd&1ikS?!|_;>ZDS);D++xzpY_S{|7!?B7pCf<|(s}n>r!-xe&F|-6mJ~f6oy~)zzTliZ^$iJImvA!i4r8Z8#sYqO3>-kp=fg0-#77e`hgL}iYRElgbwb5=AZAN#ds;p$%T^Hp8cP>-o z#~n?+*@ncd8@#rp&oOkiYG0@g{p{N!QvQ^u>UTxGT;7x1=}iBc=yd37-uUv6qpGoo z+?lOK)#H0K->T-@dHQ}2!MlCt-EDR6uvboo<4uoVn9)p`bN-obidkulIvH}$z_7E| zg-VnSnoj;k?o2I4ykfW`FM6jf!CM@b`ZLY4aEImG4MTm`ga49^e8Yp*0KI%7kk=%b zMRR0~#7e)%l=CAw({!>kPBhQa9S934UA${60d*bETYeOmkEc;}(2(ACDn^U!=Wg61 znL16pMKyi#b2fPn2_IbkF3M^?Z=}OQ0r`wogi3?hVg)3xIJm^WeGmWR- zcbRDOtF29X@ICR<_x|ENrJ^p0bh8t~&D>oQ>s0u*P*R2aGqQuJvG!$8oKPv}fb#m#xE-Edv>R{>n9cIO z+8s9Neuf-CGqg6D2U^GWMPCDQUg!;c?Ok0Cmv8=L|5#u9U63RC?C-S6Re9!Jbw#@x zN;}OaZ~f@7rtZuN%GS>;V2$&HpsNvvc*_tA3ZyD4_6@2<_jK(mJ2m^PEF)@FTjB%G z79HcO(9_K4p)>NgS>4m? z+DdXRi8Y#KA*Yg+q=aH|JY&!mft6xH5~=pUwkLi)VV?qCz5YNoy<6sLdhZw82>;6w zn==0ydJo>rXY>T+Zl8!db-ny)@Jl=Q*wvvIVlUeMSzYt^j$c>xnFM`_g(d#?O{~eh zdmCpTs?I+@1Ye0Xkj(3veX;IcaU?$J+bp~A;04fxM>Xc2k95vcK0~6X@4OqPIllEy ziyegR_D+gj$PT+p2NDs1#CtwdPp?dCDQS+oS@_`d$f2Pg=l4C`n*R?? z?;g+e|NsBbc_PzZq@y|JJkq%^hvbwTXOdGQB{`KtPD50v6lPI5=CnD4oDXwMs2rD& zV^++0%no+=J$rwCUl)Jua^b?Z%O20i<9WN^Zr9s`*9$s)^XNA5fvp%(?60{a$Tj9r z@U61yjdc?18_bp)^8IU#_P2D8$}XG&J0&JSzgB52*Er1(IK%LvInY7Zfy#%;fd#S*2bw6Rxq*WT-hV^Riigf(tC+vuKI)vbMRHY2y(!4Y)^V+N~*oFq~^QihTPrZ_1 zneJY>gZWhS>-Knomi!!MNGz#*P}tB>rtfy31J0&4vOaKp(p8-I;8!jt9%gkdNa~B; zo-*knEWniipgkx$*!TDsj7tUuHB2+8sKv)W(FxEef zqeT|7!O}}&qY0>}@Y#)4$*{kjQiMC){!dr}x|sPY(~hQv7n! z!pm45@~QAw)}4J3EZQoR_qaD;KGb>EMnr2EoWF_u)abJF`C^7sX2IcAXk{j#x5*qC zFaSp&PPQyI`v;WPuQ$umB`w-8KbNJ~lL%GxJpA5oiwl zJ#y1JI$AX*0jGpvx*XB-+(QSw?Dhn3fmDJslT5t z=(CzT9ff9lPVtxC_QiRnUeS0l3Ti(SCjbZY0I@sL!v^#`i(lZu3=PB8pDEcsr+2Z> z?p9hSAg*?ZY=(b`cCGB+U>A~);7IjR%Y7Grz446{FVDB$rJ%R{&TtkxfjnmS$2h>g z-NUJM+>vo3Mi-ERhS!yCdVA6`h5F>jbPgp%8=cU7yjBQMbYT+jh_zG#otowTZAG&NiCz=-INJ&@v#L2e>Oc%)HWd#f z>AE_$f;pbtG8(R$(NO8%x=g)*ak$6_nu^+>%d%0q$3TNUUZZ1ebTQ|8X8v?&yB6@Q zunP7ry8uo?F=h294u`7#Hkfkucpfm5ISCLf#&;q4DbKPKfc*qBNNXGP|2TZ)=anlN z6+B%lLjT74RO(h3w!g?#6-jdWT)AILDnK?SC{{Ygcke-cr@p~muov2b%ly#LJeoGt zvl{`26Qe12|Wve>Y&iqfBoxPfz-az<^edIFnGB{`iG2xWuj zo+zr$VNDVijJZg_WFuJTAK5bTVd?Lp%=s&5_8)fe8+Y*{;j_16sfJfm{dkl4>(o#u z8`e#nz-3&X(hrck(e7@?R%F9CMp)OrGn2}R-D$Lk z^L45ws@clkCpt3;QMR>?IHpC;z^UI8cJJxCl(=Li&FB%46#0?pL>JH|h>ZRURl7%LA%)VHLpJ`9dKDrVo*QWoHm`56P0|M<3kW_dP-N{M_O@Kn zGnxu_El;@6wL`t;DTv)sM9zCiL4*PW4^jhy|Y15<@hVkP?GRTZ{(Lx_9Z`gUlp zUmL_EVV;kL;?L70ans(>_kOSbssmHj?HptEu4+82Y9&6~PwmnXFJfBw?dhr}=X{wl zp7p!`rjn6%t0mv_$pz1EjDxLkS3GHSd@O434l9B{c%hom4_B@J0|WgO?;SBSghHwA zqee<~Cu*ZTf1F=HsAq<+ku|~Ix85! zoJIZn&3`(pD4?@SFjqpqG{>86eVZd3g+E`8GG+vLAxz>vxY(CKI8vUo6!k&37Q1|< zF_GTi8y%Y&ic_?KkZ`W`p$a$dCUKEN#Tc zjolCqodwV#3*@@f)27Zz?B|gL`=2nkY2+i%;k7c=J&VIultLei?)XoUmz@=-HMyCT zBLgUNv%jsnREKLgO`QGfF-%I6^KCTvP zEI&f4*%SmDTP<=k1y#1(h{Jz4j$p4}-7BwHXe|rP_;`4Me_h zaj1y!_RVxOaGA0WZ&%uHq1aYiy|O{y#~hG5CSg#&PH5`$div!dvckB}Qc6_+I4L}QtivVp-M;?em(C4ZKi&rN3B zz%#nY;;r{v*Cq|KD-7T_ZlX!9t_zOylpO!dwV|FbX1@49_;Cu8y1SaiIzHmo!?<@0W&M0$Xj*GwhqeEnr(kH}pagu(m z`Tp{x%!j9KyRp?f98vs{4;KCX$@JK(!{sx9Zfn={_K+so@zm@yRT1n z`+q%lj(b6c0f`SH)-r0QdN+KJv^{p$? zaFlA+l^m+-Pnam-@uDzZ$eZVCDTq}DSz&&pn9|y6;yPPoH$y3WIjoL0cJ`3h&tkh>r5U1Rqco(JYfL2KAOr)RH=b-BVZT*J z)wW2hZTWxoo0{PCeA|~{AU4Pd_`A_{dh7NE zc5E(T-^tP3B2@dCfyeV-@r04VC0QJ5Q04EN@bFqWEBln>tv6tgD>t+Do#4-B;8S|P z;Qq{QHqnM%zONFZt)>Unn!ktPgUG#UYt$E?Cv&Fo`Uu4~^F`xM+mi}t3zdgsn{<{1 z%A@jb0CzSssWfjz{FiVjeM=Gt1Fkbd8VNSb~)ZSk;_1YR# zW0=^z{M0`tq>-{Bm^^J=s=7ig`EZi`J;c59nr~MwM=g{FMJKmRwqi3gb;=xbRM4Dn z_5w@Nsj+-*IknaF^3x}6W_i>sfXl+U83Bz}fwR(9@7y@P`+0U^K8#um<4g4!Hqi&g z_hy~ccpVo5oo=pyiQ~^cL({WOUbJZGmmXsm<~(7%P&QkpqwbI2-DTfU?XcfSxm;%m zmZ6P2ylIOhR!=*>rgN-R_*eOFW^G{9MlR8%y6UYPo%QnhbXor7P`h`^X3J%` zw0&1qhnqtXp(j#0e^I$emuh|kw`$IPc`d9L$O%Mot{Qr&;kN+JI{&$oB z#=QSGvVHSX?mszyt1Qpe_gHP8 z!E}B04>c>32$vH9S@VMQ!EzSgpz*+SEfEGu0Hf>Tf3ND_u}1)pFX5N zv|lhwy3|gyL`Dib_!U>855Cp9@MLj)eri%4i%-0a#cvv7KUxsv&R-c1QM48Vt!00P z_X%;?+p4;f#%C?(H+2DA+bZ9f8ML?Oo4y{959QWTu1(;N;;MxsP64@!luJS3%bgpz zUk8d(NQq~%wH8;S3$vd56Piq@37T?*8N=(s^)B8i1qWQ{wZR5W2&1THuo^0YZG1)KwP8@!R%6 zS`T!BzdipWRkJnPzHZ4Zlk0tY zeM8oep?JCWi*zA2dZ`QmqO7P6d@yv04xPP^+E8F_$*r{NaV4WWludD8#hzC;qpH=G>L5c!v4ZqlU(KN)f12AG$uwPxeZ{~x4-z$5;T%_ z9{PQkN_orY8`^l~7G)RiT+hX7dWHL?{Cw5IMoM5)^v3@Zr?{)b@@T=c7AS))?}c(r z`%*mB0CMgHr0PDY!h$sYL(dDv4)t#J*qBHNIYq8t{KYW~qN#^RTVT@^$?=bJ1F6P> zZU0fomET`-b;&&flAiHsp%r06OQ)0zw#Vi9hhX|?`~_4uu+J1&|2ULaI7;x1s#I*> zOxFco5xYlQe{7UxIf?Qtsag`a(_(>q%f+Pv$)1wwfx6LtVxk-J+|~`gvd2(8RunQ3 z{l<+~oKml&w^Kscu-K@6z=1<0kj7koSdW~HLBfc;1k6ut5<&9nyygqV4EoOgHZdU& z5H_Y*-zcqiE8LzSTW0=*{MG1%!Nbfl8Pc$ZtsB3ritSOXb4fVjE1knVvE4KLX4?AT zb_mkv>C~((fXqGhjoCQWc6g7B`MQ+us$0{Qr_jV!r&V%;Xp9S@7RPdsBUJNMdqw;% zEud-YKQg}l?#GMb*FlsWS6Elv!r!O*Rbl4to8k%jXL> zALeV?UX1Wu5=vBL?t3fr!r3z)Dr`Lam?=J{Mj;6EZOPpYac;vcrAcz|=@R3UE~ogk zL_ve-F`2k{i>;9OGw$JkxsO+3!rdSm(tXXLAL4Iu%O6G~E|-a=+x25CME=fk@M&EP zFUtT`UBd4@4Nu#6usaVTkYCFf}lBU`C9j7R6j4=a21^mKhkkB;7VaFc7i#Er7= zTp@gInBq@dvAm8h6S#^+kMEdNf$P&IRNql5O6@D0kqS9!nvCyE$8zuRi3_2(9=w>F z9fwDSEpknT5XjjfLzMd3J7z_foEt`JvExMF_nU?0$wViGJ-v#^M}f8}*X^&BqOz+_ zPaRAFaUnrBABd*(c8+Z@6?or>U-kmE15}pTS&a@x{+rJRSTN_9cgenzAGjM_2N9}r z@HdME0n|wSZ3A#~yZIHy^{9MP{ox(39$0(sxV-`&NPhFN;+S0!P&QA%EQT7)kukuWLK3T5ans zkhwUaWDKi8*sU3v=->t8-H-JgvbIurPcFyNO$(hWAxI*ybQf_Q23zyt1a%?*(ttfi z4m{l?JiMVy)}4Xcre1}&V7aKv<&4nRIO!hDk$)Ghx69{fbz&=FukEz~cz~gUc&^s| z1*`6q_>=0b*7RnogYdri1XmzR?94awc8{T>XOEx`s(hg2qr5^|G_p>!Dt7U!cJ6mqQDP>4TAgeOmuGepHeIN?gUj}9MrETl%D9*mgg43 zZN=rnpq}s3qCPH!F&vq~id}n>BP;e{!AL(d4L>`X7(F;8H%<>=rs1V6e;r?F_>^!Z zpqZ0&J3KnYfA{a2`KJpTr8#g)OU*9VB-`ha8$s^lSs`!6`gc?@mLHLL+JalQ!9mb6 zSV%5X5`8V;`5_R~Spgdl!xA6#6LJbx9?j1&RnnqtcBVCV_)lse+iWi9Q@u#rGt`)9 z8%}gY0%!2Ai6^NONR$|YACQLak=#Ky-naS-4b}PED%_mxd>3TeEZ^*)X#rw`;k78s z&%=n?WVND9%(>!rBf+j!vHJp(injWfSWQlsUW}>`>H0BGkvt6j)_2Ak!>@#t+m;wM zL1)CPyKo$YW)iJa->}xCO{krohM0-=CM}Mc^=kVf=`a0w(^NO1&s_gMmdWvZM_bf1zN7XP=Fn6dD-C3h-vnjCNiG2uBawVnX7c zas|K!xUs#_u1$oV4$|W%&ca!|$;Dz9i$z8!v?r`QUeS$-pidS?-n|+s$^0qw2Vmjk zo95z*x+mQJ(e@P<9}8+QEEl;fwJx(11La?B|pqA8{Ce$)`GB_?5!8ZREN00BZ? zHOjBix;`ToCq@FPnyHEA+}j-Vp6}ow!x`y8!6juo7bpU*v-;7f0`9_vy6aCnrHZ8Z zHP`B-REcMI>0&ChH|&|wMhr4M%s|?$b|p;j?a7Gpad!;2<3tK*3AQQy?i+CVra5Jd z@N&PUQf`P4s{O|Y?gdITZK(i@P;bhfiN9BkHMYHdd-A>H6FZI~7P2_|twY_sp0X!o z4p1xu+B|h7H)XI=e=7^2q2wt4{3_MYh4nAge!a?o>}}wWTaC=ib8uPql{3h*p?m6Z z%euK|A>4k$V>>o$P;{7!V!`lEdPE#T%{eGc`J%JS3S;=hfL`U$!+baI07qRukINIj zb_i?)Zmv4f^-bwJXfO4?&7A$C%TC8~UWxa9^}%>(_&81P$XOL`>mg8go+}20qLW&1 z18MRtWv!+#8*Y!{=!s}c%6n|`CoVU1_eR2k{c)AAbt!~*b!XIy7YUiHU;?Ix>m~U@M5yBoE z+0=*rEt@Rhk1-gSzTI#Pgmh38mM4ZGa{U*SHYY=6u#t++?Ad`e&Y8^ySv6QZ@46J= zH%}*wmf#1H+ju61;`LZgF`f-sCMD2Lp@tGZfWH+23V3D9W42%%%h$EA!VAWH{R;MZ z!%1Tb-&)v^Qbw&c${eS86+kx;zMRh0xU00wc{IgsYcX+b2eVnz+hpYtPyQw5F4puz zTkN+a;>_ZwsHu~p-!4U$@#Qy4@`8ojCWVB?WD@tGZ+u6Sxj}aJ&0Ko~JEI@pR@E4g zJ5nh9n4pSG%e^!^FdJ$O7Jr^+8rn7@20hu=PL$r7$g<;u*tt@jWqo*NvqW) zzR^I~jj2L+#QMbVq*&(nM9L9RI zg#X>6XRk+N1H1S3^)?*tg>A(*?$OEu?i_Z5aru))s#rTW$dY@)?10Pfj{Qg|G5+LY z;ShmL{Psh=rSAopckb7oYMF>vdl(Zw?sj`=Zl-307K9z9gBf?*EOaVSMa$@d#PBjTm2MEJ76cDOmVJ zZ%7sX+pNhqOimGO7Hd`*PTjA1!?*iK^w^oc1#4Ms=e#-^;k5}Iw9Fg(-U@Y5AHA30 z0|AhvV{xJ=dU2=jf#Z_G=IQ(w-2!bt#^}TtGMvr4xvHpmFu*X?;ziTSQB%wJKl`qT zwwS~n#6cgnLH>*G+4OKBiz8Erzz9LWl}2b|Hm;=J_#{J0m~rdZJeQK>PDi8HI2(FR z&yQO+3F?=Onc3_0$)pO!=rq%KL`*o@_~@xKI&am-LoQsLbXh`lk^=QJz|mDLupXPM z9PjaWxe*#C3n+{R1*yak=guwlW@0Jz7?M?o?o@g~#=o~7AB&x>y@9Yql_GiQnhpc8j*>$)Hr`De)$HjD?-HsQLTSGC7T4lYnuq3xSf zRz~aFGIZZnJBDORa&KxJw$r5YAYU8H;Lr3K`?lQ3roVp`CG|yNxXsr{Lxvunv4>qN z@5#>bgkBEz0J1!~2!>8Mswtm-g;MEoy|~H*$-`xdcb6c(<8a9EZ@7tOPwVFH1~>R} zcsB1_UaK=(R7Z0M=a2|VrbvjGr_(UELzfLxIvKp7vw9fs}w>sUaKs6o}!RTv<&yo5sg$Y!gNSaPlNEUPB`q#SnFb(n} zXB6(U$}ZTG3j;()6SlXHWtSSeN0)(x1>6&wyhHZnYJ*}kZ!ccflm;q)Al=ExQm9aM zJi=g;1L2gu!LM13qmk-}B~_l{C@W;ndZL^qiKH3TCXL;j%2K@E`lq`(w~A8xAbeww zK`(d0`xkjGsmm=0z{!xCND7e+sYM0aAr+H&!kXXiqa7bf-9o{te;`8kFAc zF{10T_X2;Ve4FsKdU^^|>G5dhcT=Dn!OQO3`>d`}mdmp4sWmaY`idkg*Y!h&WOmv_ zE7ya2q2II z1duQ5|M&er`7)sUplas)qPT_h8*!igmu(biS&G6kV@IClo2`^X6aVi~a>GNQQ=#Ml zU%tvZG+tbeUq3PM;`KY(7tHzuWEB>Nt-F;v&Bd&>W7Ci2dw6zhc1|7`3Q8>5!AkCz z==l_vV!D%-W^819vj0w6>cXA8vL+9Qyb_-y|Zc}`WZw!0$jbCN2Hh&d1@bi*=<@l6mV}Ukoi2WXpocdMNV|u zQ*~fcEQ#^{VLuc7rwF52^LB`Z3d0wtqhln31jvn zQdMCp&B&{ET-A2Ntx*5&2TKidV?*45WK1HOujhy1UcEZ$ZF1*bhDB*40lO)7eVv-w zzr$Kavm-Py_HUQ&pkImPNopZ}l-uL1{A&NI@wH3uyP1kU7Ff|i`g@?Mle>*+{4yy! zT#fkn9;LjAPK(w_lKQn60D#&7k{3XbSfy+620I9AMV6R7-W%S;szuWtMeW(szX-+s zGpox1-6J4|wITl@9`eA(4jQ4OL&0whT)t~T&|mF2S)3>nQ^|qG=P9(9R#|5#0wI_v zMVP60cclRh1AJo5TLz!~IBuoM&m+@fmSd|^Ed}|R@g4627@D_~O+iHWe*inu90H)= zLg0d+8}BL4#CM~tFlIbgKCfJZmau^AY1ifxo-VGiYr1R?noEKoXwYl(g$%|LQveQX zzFT(+grx{+A+38V{2zW!m>YAce|3&!XPuzW(rp*{}M4P6y+eRF;NzrD;j6b^L z!x(9z?m_*nfS8nf#J!JbDJZ7FjQ@xNzveJJ%)AZ{>fE49NEhx3E6!jke>#4LRFjVL zKHymNN4D^#M>z^bAg|BMpKe>UsdFQz{3U6>e+)_m{{yXlk-=%kIkF-Z`6Zo^1ItVo zA>%g-!?mYjo=zSN17@^+# z;(|cck7O78>cxs1jlU#a>J%T zSp4v2hBMDA01a=Rj2VoV40D!O9sSAA`ntVufu1K;I&)r0wxMZA06o3+UVu z;)sHt6VfjDp8kiCrIsvHo}T_4bLdk`7N264AZAGPIHQ7y zdVW`l?UUnW6XNAa_ES*JM}m26@dmMB1#2iJFD$tQ$E;D1UT7B=1IYkLTTuX_sqcOy zS6Ma;;DLZ*xqz*xG{->^i%aO&E5FA|?d2CR>fgspVcA&Jy0KSnk`{m8`tbGMKr0V# z2@p5Os6r*@u$LR<2*NT?T>MWw6mQFEaDQ3gLrmQFw2(X>`t&!e?X@H2MuscpWsUvy znd4wrU>2nbGh=B2-o)=W5KG|D>ME~_!j64dtQ*fTrO;?&v!RXQ0_)q<<#Nu!*Qln^ zDzlv1%ATTeFI?+>oH(1!6Vool2Z8}?Rxuu6TBKRW3x1^sHYa+EJWc2ysMAEF#c*I( z6z_ph%pV*qP740@V@pF*ZuW@B*5=U=Rv1rvwDIXfVuSoDb1GXfTbHlylaE4MF7A}` ztP;!ea=KfAkp7}8!V5r3##yI7Z|-N8=xYE1L)+WeVw@r^w`Ix1$X0;+bkb*${!_p0 zcFaS;H)3iy9fj+IfZ&o-HcVp&+n&R!a^h9X-9_s``Q;j*1OrGy7rB8YBl;h?G0rqq zz(YXHR-O;c5Rrt;wc(q@JCFD?4jltZ<9Id=DHu?sEDpZ1FN#wvXJz-3XSl?LTrqT$ zhriHpnwbJ|>M>{Adb3r0_4MGdVP>9+3zZ6NyjdSPQs>h2e-v~b?86~uJ(7?7UET+@ z34s2(zXJ2I>$`@4o=bkLYRrYZ==HLKarkk733W&;MpRJboAuZs9vSz;t!`O-Rso$V2teJ1)awiaj>caV6`c<cg;CPKvLhzhoI1fMvIdk^(m>ekx38WV4~W zS8ypkN7DQb6i)PSsPPtb^5;tc=*)bCYo&IFrTb#Hr zW-cAK7dnf}Y_anXcy0zhpwEh|L`DEixbsmisOp;muSr26XFt#|W|u3ApxvpR+i8AR z))PVP7xr>Xp#o;V=T+|rx=f?I@?_Ds7w&l{W~+jSrMLgwPiw-7#tHJi2+- z^-XlenU>%eDY1`t!J;?QH$~IYd3;*pxk8ZF-4JtT-h1D=8KYDQKG;8>}SK84JhYu zn=Enu)D4yisFN*V@}V_AOCS$OioO!f1DocyUx}AvL!XXCSQO%@_Q%Jl^~HGLKZu)b z0UYI?S=%xKl4nVdN#&T%<0@x_-m^)BE~SzFu{7X*0$?q=jfTfS4XdXv{V+BW=~N_j_IZY*U^ch;ktDYX2Mczn89 zl(>u5uFLG1ix}<4mDU`6VHUArYyal?7-h2X+U>QoeG#rP?agsQMW^OxMr1bu68RH7 zXk~}VswOQHMetF|2;zAW9-hDU+Lbb_Ae#YMeXjS1S(9-`T=VLjHbM}x7@ag#XA7`q zpulcFa8?RuV|S{;Dtn0$1r}f&Cg=HC!W=*9j|K3G^nPk-xsv^OurIk}wcAF7y`&2Z zU&~VP)Xtv0pLxHosdh8g0RV%DtMW2A#>D(D8uv`5KOK^~sVcUmR|}9cpz1i};_XKV zLdki-UdPmexV*Rly-nu>^FTG9K^9N@WBG>wCk<|PwC=C0GPAQp3z^1UIebcYWaS+5 z-5c|drc*>fVz9rHGF!-HNqc<-{Ijbljpvn~GEIADg?-{Ne6=^;|3w9Sc}`g8dD7(f z35*Q(+Wrq@qy%?}#=9d^iDx^uZUdDh+s2gpp6#?IyPL~~M4 zE_TgGjQ1ZF{+J%{8E8t0uS{7E<`NgG_gI9c2N{26!rda0r=uG8x#nQi@B59Q@wm&4 z)b2t%sME?I?MSom{cjoH7vd%Xy)fHjaTKW*P%Tt6DA^s~dF(+e%zme#tr}6o=-*tsL(EeSJRD>UOxU zG}2+zHM6`(^!amb?GI=!5%e@VR-FU?TK6qWyh?U3%ZYa zkC`pBJyrm3`6}=x>_>BAlyk%Ob*bHa^{O#vtrTSAaV5cc|Gp2mRCq*QpZW~tP2l^S z5Y?>aWr$tlNbjvXed z)QOMZw?xIcL6JdCHFjzRg1+6PYLWliCrLvm4vb@Dc2I6XWj6#$=lehDf6z|)l7mb> z_rB&FMXT}nSCXHYYubShD{=fm*BWn{Lh28)+l6G+@v+*z`Hr3QuXxMeBshLvxD1!) zY!O0zGAy1AO-v34jC^-|S>`P6h;Nc)B|P~AX6DkpyZz;P&P+a52Y`JF%>E!ljpM^SsQ02VzZ)UFt^VFI%UN(yFs-MzCTzYf!^ABP?NN8xOy6`#gP|rqOYo< z$3fjPD$dwf7E*bt@aukb!Ls_M5Gb_n+`QUke6Lt^e}&c(_iCI!q4IvC;m4_?8`^C@ z75hTTHHQcWBL?c`sfmHsX>_5se6xwS(%Ju@XktbHcoV1{u-COmw1K7oH)&Wqve=*oLcjwtKt-^T|>|1heCi*&#mE2&X-baN=nPcrtgt2Z@`i6c|^r; zq(({RseAWo&fj#Tjn5r%(fo@^G@5RsMZ4VJ`JwUSd2!;|l!5LT-7U>c(;T8S->Zn4 zoUe8(t3lP&2MeakNK?SX6Kwyv^+Y$H&0^neeEHdmGrDPza6gxKuGn3(_ggB*IXrWR zz^btzGY2=II2htnFqaF1*x!s^UoS^n2|tuY&(Z~%#_op=_kUk?*axGaSH0H~2SaLq z`4K}usQM7XRo_xpLgiYfu6JJE1j>41#9(19;&=9FkG&j=_ol5x#P@} z+rAHJPT&=Ptw<}2iRk44%C!WH=B!he(NELO;nFYH$W1PXlivVfJw6^O&w6rQ-{pWi z$J4s26x~>roC(jq2ZpoLxrOL+TsDkTdeb1#ant!C@=>&S;H!&@XYATzE9A)yW&53e zFiUkU~soa!ZN%D_|z3xYH8m#_`H#-n+PtjgxrQMt|5&HyubCO({3H zwH#EZ6`p^!iIHeW9{6pN@6-(1xRFL?#ABW&9W#|qrH ziF=kz^xLbrzo(nsw=9#QryfNdRG-PxL@~4;&5xVVJFAYS$>Hyv{qdq0>(22fq~$DH z^1I4hnovji>D#!4=$~$AFw-26S!(W=eD;&LxMeDdgeU&vK#zvo!f)$CGJ2Ic_) z>60=LmM4NacRIYNWXcv|{Da$eT#AQq+8KA|;Tv=SM9TG{gcQZ>f<7B{zQ$~@7@x5H zlOVSm0$?|&SB*j8O7!LNPMN=lKRhHno~IwR;Ssj_bLYiwOzarFvK;Q*kjU`cx)bn- z$ArWzz$zwHY#ix=vA(XqES%o(Yosr>KlG3K@-m}=#g-Yfs5_yHvg#&5UME;LZ~6z=N=0j40b!R zSQ8u@sx(l2Wv|*kAXZ1>99fdI%CEx04_+^A+Rz6&?@MKloW?74h+XHX?)*hGt`aZ` z0WyZ9>!-!D-si_t9k7O^8zu?V#=Qj05G3V|!zr-*hE(FMJh>^Z6tDD$%oMC3SH<&$Q<4)ZBTVCTOxz*EZvP6oKP2cs);l>ay z=pQ;1?BU2SY4Coe3K_ql3Tr?0TMZ~S%G4U&%CH*59MzMoWW7$wrs*7 z%iQR@4bdIAeseexJ}O8= z-*-u!aSD5OW`XUBC^d%r!`AG|0fGp+S)^{k@CT3;yh`wtpjnBNJbtrVcaI+lHSz3(ZFJ8NE%GY*o>ZsZWf`o^O$zT>~g?s+=| z|1;cDCL&ilUyU0+L6dDf7>RrB@+jSk5mXIu48#Ly7qGO=mrgEe6le}xHvxpz1@K1* zNP=$wz^!9~tVwXMu&Zu$2drig-lVECfv0YO%{_Qd;*z2Xfloo8nSu%eM0kEB(wS~k zN$rFKp{NeeYpm4X8v&TiRt}m-Ivy6#LjaR?kNzHeQ|K;{GRvwLUfW~j@Ho0SgeRCW zpr96Gf$QqNi`yA;IV;KwYWhi9(zjh7n+t-7BwJu9IFj=;jm;<5g}3h^Zer&1ZLv3T z>B$afLuH|bhFiKPe0C50Xu@)M#>%kWw#K^sra+3+sl%KI?O@TL{+%HJ0oRmtV(|}) zER|yeqTB$cfi&ICVAVh`pSougryX*@DH0P6=rJ(c4U(?spuGWp5+GNpbikw|7$$m# z*d~QgR5*}%==rD(PmaW*8y^`JcO5z>2o)tX-F1QKhtGu1HpbN${T=Do>W;fuS(7PG z=Y~5pZY>T(oCeBlcN=P@G`_oE5fP>Qb8nAK(hIZ76=rCVD^Z>*Y>y|M6|vi(8pwfq zT!-0Y?1q`yv_(rEB;Y$e;-JFRO(Z}T$9+5W97r9<*`rO1GBa7=og#`X#8Cyhnm8!p z64M+nI~DjC&lLPbLEgf`GUw39hiRgC6IuQ+qNA5cn?9nTF_F9!zaW1`!p58Mi4nOslCayIm4A$xNwN!{X$sJ^ z4m0)dZ4*>Rko{UIE>P%k8Ekjyu1BRV+KsV){He|S$;WSMECiu0x(h3(KvH^xQjWt} zLG5;R@u%?va=!kn+k*I!3&9sHjATo(&ibZyTQI;|g55=FuKM9lAQ9>Cx)kpcZyvlZC`y`p+P^0eQt8Tfp zPQJ8IvPX*=q2>z^)-3ZTohfJXw$53sk{&>^PErFelCSvhF(x)i@R8&JwPcM6kLgI) z#|JmX~N#l-3fXz za&U%)#k4kbSJLzA9FH4PynDkfyOSg|xl3LcQKz#W4s4SQ-g)9;Bf{xgqK~1uz@bjT z%c=t9eAIxL`D@cxVH(@S2I*oOT zx_t!l$7*G{%wLiBT}F<+G{Oh;v9PCANxtwV@nRS^_;93{ZM<_NRFFLGTxVF`_v6#H zi&467?W2fqEHO3PK8+jwvr%FTVaT(uzDRZ*3UiwQ&&rCr z!4yF1 zPuAzzU4Lz{V!!W!D~jhonl1f)5m@LHC<9ts+*VF%KC2Bk@~Kka8^dQKx(?20r2uHB z-dQ({NIOdY@(10u`mFE=2NP2ki}Wp?5YDpKCwzo7$BG1+Q19;(TMY{&U@}wppYsh& z{jYuh&-upq|D10xQd@(04>Xxsxv~W*fLH$)YYOK1|5siG?|`j=+d-h$zlX!-uA@le z!IIbit)vG**g#;xe=q$T3Gn_K3B1Gv>sUqd%I2|L)1$lnxGAW}QHb&IUSrGgx2K0f zy&)Vl6BY1{{5+9u9vh?yEkkoSO?X|4Ec5L!3B7IF&;wgF`778;2g>Ln3Xb@}1fT8U zAH3`%A*OC6tWH>K4y$+SaDEO<<8w00=nTi{=YERoRYp7C8J(4uwIxXyeU) zk)UcxPwC|cWc)&|e_n-}VmG1Gi-)>4y+gJQfx( z(!Ukm5k&e;&W{-6717`!8=VLlmZ)bA%f^m~f2L&{N%!SRtRtGJ$wTJ#s4Fknh}H-z zUezeGe!P8%FIx&d^6l9b^Fq)L#)U+O-J(g^8WgpT(5L6PczfQjW_oPdUzoM}L-F9L zVSX?Z%^fC6nm<;If=V>0Ys23ooAz!d3|JfF%Z;vS^=Y{Vj`uhu(&b;iuQbF@)-)}MR>?CFr^eG@D z&0b@Eo)5=^&iB&Bm*Me*FI8-vsZ9nNMCidr_Sr|Ari7I&)m^K3zp~W?6-OJv5Owdr zggm5dy|B*84lr#`c*l!wju-)XQu8Li(#1{ovE$og+y--59#{IvA!K-^~`q zZ<%5W|DtBGaT*90%M1Z8k7~`4*z_)>6S+F4;U>PG+$Xj6INN?P7!i+&8q>&V-jUEF zQ3xHBYGQ(qP*fqSw#Cy!Bzqkz`#Q~21$oE@MbmZ2+I9GTH0z5@>z(gpNXKYL06Z&7Fp!xY7xFpb=RpUd#j6;4k0-zL4%B3C;~JMr(mBJl5#f z@~1i)kDppH4{j66yxueao;pa+0OAId7fNp;3p2%v=4mj=%V`O&@QqpB#95|C*^4u# z4?|=eK0C2HXSA7T`mG!UZW6|oP$%{(PDOJf3}xHQW=7YF) zrWBlnztU7twR=esUEABI&v(_=S3d55JcI2N9CsD|WRt6N(xcf@?i(j#;lA$MOJj(51SMLizc4%Iler<3crKpoieTYNZceko_c)DsP7 z%9gWu|Mct538a0*cO&Jd?Sx~<6+qUu7Op%KgTHBHQbrdMRy^*^vO$clt9c}yHL@0^ zKoSg|uH6`neV65)jLeLyu}7QJUnlz8HL3wma|r5%jEK{rpFO9I?LV z2dhwrn@f+ z2zw}hoZq0FdWUIh%e`DIT)?vNFM;Ocbe$_kteoY#k&wt!x#yS4;ierlh21x5WHMzf zzNi@=*bPdYrS65S0B(ygAtRpv!3u_B>7wQ>COhYP-V+Tv_L(Neef@;F7mbk@{?`)*QQ12cs# zd%U-d*D*IA@9~aEj?IhjJvpip=1sh%Gpx6hxDia%=nWS)@V^ROzdpU~$|)@vby?|) zj$l{3=?JmSp_$fT+*SzUha9v&o^j`ut2jNAX7!Za{M;KmnLUGw(xqw&~r zfAkn_WPw51pdh8Il3v6)bB}vKIcjEWDYQNj-=e!c?+FdXb6~KP31&qmR?;}3Jd5}CN#U3`0)3?ZhR-{`o-eHy_m!<3amC+R>UzRQJ{x!UT~gXsY0wtYWy+o}Aw=JbTG& zc9f^yGkeo7f|T@m!+ zuBc%qvPsq|Ur*FsXHY@`)7=PZ4fj;1GinxC{!LY(eJ-U_=4J2HA%M>6X|^p}Sfwi6 zm-qg8B-pK!3s8MK6&M3m6%pmL$^Fe@J=GdA*=rD-JOOVcUT=JRa_e}=Li;5N!sguR zF0WT1GwwTWpL_)9b)p&%{I9DXPSeT3j5Wi)!bq9nj=qMy(9Z%S%Tb9)%{cW&0*r_K zT;yz%+fiP&>e#RJB9Xv@?oO2!^vdOVv_kORT}p(92^sGe8XtN*s=OS3XkmywP=mr# zT**lq1*(Q``7JQH8_*E#1a%ybvZXoboQPS-4Br0LzF@JuB9hX{IF2)}Mq~h0{|hiz zAZmHfPqPe-rWaVDS{K)mn}96oX9hj$7~?Iu$b0{;s^Q;OoKA409AM+|e!lv-tK>!I za1nSA`kb?F6O$R$FaQkjVfOWu@nMp@oUYB=-{|C0q+)Z(!dhlROrM}oic8uhdQ5m9 zwNv*v8QNwlr1a4C4EwCblmIpD5@^NZyY&eYOoDF=JgnRbe*DujklehZ(Cq`!pW^X4 zy*EaVEu`YRAql!mk#g&acCN!5PxDDambs8YFSQ8StlRZg$C_*sLfS%*X!&vm?!6Lt zT`-IrO*Z1WLa*GCaWTGOc3l3ZaMdglI58~&%8f*PNz)b0Th3A zN~xUvR3b(DLn_?_d@^pHfxq-@qybU-5LFOfDcgyJ)YrHXcrk5cxt6N$SwL?CizZ6b zNe%8b0^e=rnI1~suSEO@#LzO_7>|J&vthEc)-WhXKX1bG+3wVQ;)&Gp^8`VaaWk6I%Eu`!>;v3 zIyfMkt+@Va?{U@UaidU_EafgdM58XnIpYr|r#&8BBMPQ~m6~lhvzkr-(*K zSPI@x5b+ABbd0yym|0J(E>O!p^dcYy&;B;3X7H$=ZrfL>&TiS6EZfZmxRse4;Uk(rsf`VW)S>Yh0ZO5FsW zu)C;3p4i=DZOU-cmfw^xVJ8U)&!3%bx>vwQx>wa&K6-wXe^^@0e;XxQCv>R)Tr{AA zni*d=L_+%7+@$<#pdi>0Cv8Z>3xU#nW>KRRvkoR`*H&Exgl&r2)EpABKbsy1U8=D1 z49Iur{eBQ&NuVS*J?ZxQzQgJ;`o}q1>Nlo}b%izz^svo$?jp-VCa>)F!cYsC$TN#Z zxB2evp5-8IuYsmm%6ZR8#|>|uW|&wN^${qB3nub3`~C7+0uVwc$vxaENjh>+bP!5r zOt$9v6K=;&Bqo1LKLuc>Y3{n`0UEAkaHCcS{(gW(fqxoYvlT0IntGFEp<&aOv4gZ- zp%t1VL`*or?P~@tVQ!`N{i#!6DkD(libyC78S;KT`I*VZK(O?Ges+Egz)<9HBeSz> z))$V@4m(MW?@}DM&q2%z$0^8PhK?o zXe8U_xb*3T2qx*`_`H_}0e@@mEhFhlNzJO+d>5G(;@n9CRj-aWDa68njj0e56Vc>i zaU?oDdmLobnWJ!loB@2yFJfBbg?AgQ8|q~XAb}!^Q1mlkZ3a*GAk^s9KK>|-`LIl1 zg!0t85emP1>wE=3?`2U$qB7`^gwVmgj3SK3^)z(;hl0l+v0L%TC`+&jQ%*rBXT|(w5!$ELSPm=2Z_E$%WV_XaK4UdeeC25f>EDX*sBk`2xe70AEzpEn8nsa3H}aA|8bEL zvk`r?!7}#?L%D}iT_YuvYhsXc6GE|K#V3?7FVu+o@sVW;^*F1W|K74`3fvEolxRRt zPC|6I`VJl%_Bx8skhM%Vl7L5&snUq0;_>0rrCZRzO|M+%VbJ^=0=WKI#WvTX<as{%`XD8fcz7or4%yOcw;i)FU4AW`6$M{nN=O5Ez zUR6%l)5G<<@a%*-Lu98K-9$@`8yS<-VI^pm^+T}h7hOVPoAS>?EB)QwfbB`>sn-71 zbk^PR7@03)3!*QY4iQOk7vH%T#_eMlc!C$mOlLxHZQY*j;nWuqH%zHAL$A`AG_ZK) z&-P=e?^aJ4x0(+YAK_j<4@8%z!_Xo;#@l;8zCI1P231=RRvCmo?3?2vfItx5|HKdA zhyP{-|A`;CZ|To(KUETsM|k8jp8Du z>;8HBc=d6q3^yxCyU{Q1?t$_v{*Ro_YjT~wLI;^nqvu}&r|T1f#SD(k{q1cN?oHc*eUE}ep1L3~T zCGckp3$OE)TMZP{ocW^5#e@^eMA;`hbdEV9{j|G1u8}XEKdYx%;oSGS`!F{7d<`pl zF&- zmw{3oRkO^)+~DPF`JP#rEZnNgcTyE7j0+SldUaGW`f?4H4K>o|VVjh(4fnFLI7}&+ zD0(RS2OE#lovComJ}e&H*Pr{24bk^WIz4pi4*P7mE!M_mET!^V`Soirhu(6y7|(Wh z`KYEdr9Vei;LK&=*x-wBst1;~wX&s8>+R0Bt{NYjRu(HaHPKUoMCe#4V?sE1Fwbei zIOWLgb@6J~0XhF1BrrhGv&LeOKp^(CarnHe(K*W7Pgq4|KV3+D*ACRo5IE-F$%C}C zo)vd~^Fkct&F;fSH!XtA_x7 zPKNDv^eFpl%a6*O!XI@-Z^>{syi|~Hx`V18Ui}@Bt{a8CWk{QJ*|u!^4E6Io@Th=J zQOD?k`*TH_i;GtW!nK6~20XVcj0MRE@}%sFAoEZ}{5;7wvaboBvhJW!xCv365@8Kd2?WZgXXg(U@&23F*KuHeba3}oyKMQP)Bm>R|sc~qZlW>EZtp0 z{mylnn^;qejB|2u-xOtAhsTe%o5Gc@-{R`nEDWmHZ5e5eq%5MbzC4J!2Xb8NqiVy-X z98zPPyZO2O@1K`m#1R0V>;5V0Okqivbx@!oUW@z%&>DOIIr@nkZwL5Tx^9UQCsE92 z+)$Q?neaKD%$Ex~V)a{X*VQTciL+wwz^loYHF*<6DD?g ztHjagimAxjuP~z9QLN~jsxwG4(^^A4cdU#v`(LUY{YcrFWiE4QeqqmJM~dsoF0%za zf&#?^p(qo?wz3uFS`)xXmPV7%$ETa`#NF%8E-+afm-Sz&{&tD!?5#ZSa`&DU1#7H&3OHyonDx zyQM~)nR8=Xt1;}@(CvIE6%1k4lYNZt-!17Dxkl`Zf^Z_5f{JZ%J6(R6;S!Xf$D?|8 zt8RnFsP|Yzpy^OUAl@juU&M8Kdxb-&pIxgmN)*i0G^`kZXtMKW$?5OS?$W!tH&#y6 z$R9o0CU5md#~lB44`2f0dRgF@C8(=47o~aC>wOeeJ}n0D1XIL+$ZwwOlpQKz+N+JL zS0n$DWg&JOrCSzX>ypv$!P_lq&^9q2I%D4Ld&TG<1QJ;a5Nugo4_KLX3*Lz}DleS8 zFbbqG{y6QRalC*2q>pWdMrx0L`o*_D_Y_>HK97@Jr+v5Kt|C*3F=y;Z_Nt<5IE|sXNco$Z>XU%((d>oWOF| z?Na=O;#zcfmOT4c`p&iGF2P(FyBKjaSa)FzYYLky1nh?mBicK9mI;Lx|mdS+W1*|xTKe&1l%rKJ6noj#x+C3aW_W2fR9DvFGE z_$ow9}zk}utkLBFCKc+X1IV8euQcqI}6pPVcJ_~v9ZJGJt6lTF--#FbzTxh{_5g*mTJ~KJ>Ifq_b51D zk=;5M+e&T^^X;3 zlm?T(T2uGL)#GkF=g!kidyml)LbHKP*)3n~1RjQk`@Ki>o0h^d(%?(s(a3{yf6buU zRKE(Jc%_6>C6$#(@dBoV-#`j@(L#l_@gnP?X+S2^cC$jOkIdTruTkY)zYLC0<$G=S zuC#L(<>Ra8Zi#qAUJTn&=R4~*y(7tGuTo*$^QQL)mGG(LojO(j#5LmbJv%+} zQdf#zGK4606uUjrTACZ>@N@UT2op?(bCauc%TD1H5sn3lNVZ1j-@9U}YHO>K^YCK<5@1f~c24 z-|t#R|GK<5GC4mLLc1A%wH>hp7jj(m43;IISs{KU6<#aV;Uc^+a|Juh-{vvqtXnrP zQIVEqQNlpF^Uk8)7#5tp>DtOyuGe=xT zmWN!GofYpLi}$1UhstjjT55Uh3z8DpC+jqr5EV}q2k2GFn?sMy&x~iU>Ew+xxks(;l z1ltn}`X!Qh{?M`HpQc{Kw`G5B_R?Qoj=Xt+ZQtM8$5(r8I7R5nkjdXLztV2--^{Iz zNLPTT_`rX1@;^<#O$x!j{&NN1e5(TgFXXcYkgGoefnWXKQ)IS+6%&0sz$Q&#NGnAT zf9TC*4W4$s+4sMuE4k|?2ngNUzpxW3N5juXFV(;OaZ3rZP*I!eerqH8<MwC-rW` zjyI@Do}xAQ#>7OfJfAazh$+R;m4SM?2j61=2j{ZLTdoMcj+Yyo-k3K!b-(=|zua1~ zJ@mD>NB+56-D)eTu5QODD!RJgFTLOYpNjqPEr-q9>tyoJjMI7bf0x}QWXVrPiN9o$ z@85LaA(Nksa%5?VpJd<`iND*@tE;P`z;zQu-vn%fv^21HbKrS$lf({iF@o46!kgM( z7JQIs`?c*7Vf4Ndv(z*t?I6lQ|S*wwwq zyABPV*Fw5YK9@Ayn6|HBYWspkEA6`|1o&$+ufK?#bG-QQd93BQeVQciLh?g_J(bt| zU+d5ZNp)#Q>IEx@`*mA)_LRlb2Rr&pyO^e5=K2_`h|Nj*UM)3se#3rujDI8)e0V7P zd3r~*dr-`?O*3{d7W_SQ{B;Te4I^5I+r-`Vz{E8>`5W_R%=FFgR z=~dhP8Y#1%iLKTWMtY~`@!zqWjkjOMA;GX1;pN8aU-xg0ebOekQ{rexwrn5L)$IL zPe6=@>Ccsp`JH1n^6$lUT7BaN@3X#io+#c-E@eGjh^aT~+4FUa_g0h4Y+AE6)SvJt zTjt`Cjg3J3fPNq550Uqr#y)xt^-LZwOLH2HWnhcdWnNn=UaOWlXj0@kcXR+u+X$AA z;Pu2hWCp{0$weM|MO{1@Jvc)E4@!m z%HYA^mY>Y?R^uMS8XPvMVE6}pzqF_%Z}Oh%JpsbkXxm0=L2i5$*9~0pchfo!MkCX8 zR{}3`&c6wtX*r>-yM1se8`^OuU5U2i1U84cLFof<%T+Ar%A|U|!_8}#qREfAK z43Dye!Qw5-kO55Hk~-R|k}AW3&aP|S%|WhE(BL_OB~O0mgiix8OYgs#L(=t1qddvu zB>*W}hs5LG1{*mIsE8A6jh|3#Ze4yUGzIoAN0GsrdZ{#D=COM8b04KpFtC{GlP%67ZrHZ*OFSE@gJb>)ef zBMpkw2fF(-GHK>;i65chfnbavynde5+sX8hH(@s;`-HO=eXOTwd;n1)KnM>S5P2eD4YY`_ zj3CxDW)Pg~N}O!jrNY_0wv#zGFii-Z1Y?hA+1=Mh#>^N^vj182LBWB+t{ktJg2n|s z*tc;u=f9`cqRQ&Ut#Q^@S~zIwQB)U%*!6iK&)SE8aHBt)>G5d~098625(wYC7{0%A z378mw2YiYov76q(fBgnbT^lMQ2SO zGLWB*B;1&5@})Pk5Ii4cQu^nx_k95$G?AdO1@4+_x*tV3f0H-LVePqy>d)QfhL-hb zET_v0T9hHWi~MFeJlQoTf@ru~W8X`QRk~A?Q#h2UKX<363R^kI?3yi$fB-OwWrY_w zC!=kqHVurc_R3U)8W!pP0?zFxAAbt?AQKS@0La3s!A(Kn1vNhZ<#V}hCStjlx5w-K z+4P*DkWY;Q(Y@?cLJD*owBVb(CrCjv07l8U>!yg~GS&Q;ugSwYAy@T4+5xMIo8i`_ zg1lvlUR))p-cY=4q@3@EFORV{N|e^0O-} z>M?cNew@@{5>7(xdps+CT-iLSm2t4N!@k$EzK2AkJ5#7~%*|i@X)Ab?E<{f~3UWC1 zFY#(ekjFA>3?Q03W*zUaqT7BQknfb#OjVWMVwSh%UaMwPuf+r?Wx$Hig__5m61^r5 zPO2KdIlY3zUB}QiOQ(dU@Mqu8k4U2{*AMo>&$hN*VJ<4`Hz=+gv%S7W?%4hKcNmAJ zVu~f+HI@j#8urX^z-TDqIA|XVQa01kh3e2iE+DoVIki;(E;^o+z^mJV@=j=MZu_VwXf{?1ZutwKZ(F*uNP`9)C-7#FEtvT0rtp zAi6$FH>Nv}P|Z6iq>6j!nwxoOaGS<^=Td16aWVutD)uMrU7R|d!zt^)rAgB&pF=yc zyk3-OOxKd1b})NypZ0iE6F*>#jqN^#b>r`1ctvnNC4J}Mi*AoB%mig*we_&58_Ich z8I-fe&s4M5JW#eHbb*dqEvx~JA-F)vYC10{i1JuWiAarIjc<#C^Y;rrwZnN6~5lahR1F}DY zxT-?Tt(ya4+p^Roz~mB2MYaQ;{2hh#1jn|^_jAEP%9)%H&x;@@W?nPZH;%ziwomna z-L3=v=I#iRG0J&Rq^bzC9mPijLWN~o^M5DxQECJrzGGuG zIotb76|#pUDjCSkvPZG0tv?eTca4}sPOtiGvkm?EvswPRE|>};q8OqP#55tRf6xEt zWSK=ra&atMS^>^Sdia1(Q!;W*)mMA-OMoM5I2 zux7j?eSJZotUzgvZB31Fb3_P)Cv+{Z@eI57;q<%x)f;Q=(a8t8{7%cZ`d(Q0J#eN1 zZoDSCq&E9|RtTYx&(>g$6B34z3kO19&GOWYiZ*+a5~y~UYi9cp%^c_*ByAWjz+Vs( zv4&ee#k4p65o}*HD8EJQl-eLk14xuhP|ztK3V+V2*Q@hJ?)c2oHaQF{lQs@#aH+J7 zF@x8ta9Z*;SD0XA7_W?Y{U**EssVa}^oKNvuF>+Z=s6NR7|OJAW*A{2X zU5*~NPsZ10x6h_4WWKR~P7qOY{TsOkFb?bPBCMlkS|W~bUW@U)vL85|<0{I|dZ>1j zfgsoEI8|E67QeQB*){dlW$0>2}rIvhjc)mFa;QFDMF( zF1Rr|_*rt`Dd%t?qN!uqy;B$ZLGf zzESx11R^*b3(2@A zCa*Qcs%>00BV0E4)8wi4WK7EI{BMO+!;HSHYf@iXugG`kB?OthKsyRaKD;^Woy@^0 zpza50?M5q?^RxLExXphsNP7R=sfeZEjTlz*zXprEW;B^=?&%_HjDkM9M_#P`QS%Bd zR?_Rd>c0KI1!QvD_OqwR#OpS7-lw{(e@PF>qwPtUp~d>wqlhB>|64>R6NH%6&yt~5 zl_Y(GL4%`;Pl2uN75U|b+Oz+yIrFt>JfNm7yuotkQ_H8@@<9UhG@rX}$mnqd-#${L za95IVqq%L+5_(PzG zc;ocZ@2XXBS1XP~v*xLF@8e8gbFua*JGPiGB)Dj5oboOStx}V$)VA~Gy~d0{NJa4~;`z>Pnk0=1 zLS%sK6}Z2IPag88-Ri)j#|DPMKch#Gbn%Q=a<^Gsrq5@}_2H`01l`-&Mj6M#-om0 zN7L6S%2E3S-P~t&W|5gcAc_=O>p9E22;vR>Mf!6ozTDRISo`p<_Agzub)>z%6x~zb zH_+bB3V3~Dgt3YiBPBpsUCTVs(BH-k@~?=%nRwTHE_&rVs8i$>YG*!mWPyCZ8R^h2 zD`6N6MO#k3YLt%fs5Q8HxP0r{C}WpY#c>%;B)nt(>(PqoZtv@|F zZ|8}Lbjo!}mnTs$yNTA{sf>F1hyF1KHK?Fb99|ww>&YSV{NnuyFYsD32-n0lFMI>F zr&9AHl7o8gA`M6*1jNfOUV|f_C%U7Z?R<8uIlxr71t30ZfHIgN$yUV`QA z>rjI5fI*Na2o=Jab9pBKP=!ECaI`73q1Al5|kW$xoHe4uuGbiGdkq&e6F2K?Ew0JU_g$I$-% z*0JwnJvVf@&9Zf;&kouAioF{#DX9Sn>yC?MRzhI#%~LLviE-xl6TC@010G12$EBRE z$iOz@m4{eJc_$+HH^xgH#rmQth~+0)3%lZ(*@i$HkZqddXy9gjEtnBVgqZsX`RW-= zCP8|DTRjd`odzF{(q|xFK#xw0IR;=BLHs{p_;GJ?=YP_+;TiGjCdDVd&N8{^) zDsyiU4<&2XWUKjy7Km#C(Kv9h51rM_;JH)Pjd66so9JcluZ)@pxJtytNTqU=LHqJ7 z<2iu^>&V83s2_esE>q2WNB8KIGU5Wu*-&0yQ4R-I=@%eOJh3fTeG*@tph|qF z;F^un7-n-Tp-_-e6F?(?-wT7tN87)Pt)H5jt~jV>P3Se)JFnaQZI&=Iy=j4m6LMN~ z=l~&?@%HR1gs;VR2fCF-`O zt8+nCR#u2CizBpHTB<9}M%~`#ytkC}?v}(72TlM;5vCpg5 zzs=A-Yk@S(dl!4A+YA7yHZZqEVl=P!XjoZE&cw=0iio(Zh+WU9>_@A0VITkE?L_bw zcU-VBO#4z7APX(MvmT6VG;r2nm8P;F9SQN8ijfNri$piYueI zX)Z4?uyw=m7SnW>3z{BYMf6B>&`dE57j1^6`a@ zbk}UfsXv%ft)6V5dxE#VE{i@a!%BAL7Ub96p>L2Qr8(D7)OH7WZ!e@g1gi*EJRGl7 zo1`7K-P)wwucyAa@N|0xFJoN3Kd5^<<&VHvN^S12m-TzpYUKNq+WFUBA)nbA5naO< z8i!0H@2>75G%+tYdCAU)%mIk`q5UJVnt}s7jM6Zq;A6FI#w*vQDq+&EvWx6nldSez z^lvI8m7tHN|SyZ6xu* z5LEkAX&N2N;Y06aABC|Cwo9yLR~G46I}JGyD__kf@DF={oMIi+UsgbRwN1VS_a2pu}Tqg(YKxF27E`1Rd4HM)?H zn>5R=qIxJAAOdycbcS@-e)D&#ZXKT@u0OQAcL&?#cZ(sZ{yLCvXT;O`K+I7Z62WI_ME*anApe|h)|$wy)aiMX@KMje*iZZxadL0hr8bWxMF zRX_>f4NFT$xm_bl>WfLX=YZ7Y!KpDxNS}~k**Jd0S(+tCU`~>v1(hA%&K;7!dpf;b z^Ztw3GvaMKqYABYHXTx|^GntftBk6yYd_vJytI8^G%@Wei}}7#qcgi4eGE9ocU5az z+@2h5#GI~5HdMTDnbfz8?d&89%dMRao+`WM#xzZc(oeP#Y-y!+P7eYh|+ zAMMKC4wDJ`cxwz3nO=DR9&fSk-)oV|*(ZXrv`yj59b#|v#)UD7 zOK173Gi!}44;yVK6)BD(4Lb-rn8r zB%c5WUk?8z9D%d`uVaa8<(I+GP=O~pz_$(_e{~`%F8roqmuOP_2I~G*Mi=nLNa+f_ zaq_oe<}qLILZHJZMM zz6T5d;19NO{A>?fK?xyZ;Ad?aw2J0W2c_Kkz+X9(lC7Kgp8S%xXMPTLWEZ{pNzkT0 z=GWfG2|s=}Llvs`Pky-HGAq{gdX=Cr@)qNPtKTqRwwmfrnrZr?{0`T~@e*6fk;F&( zd|*>Bv%Wcth+Zo+-$yfS_*&iG>h#{zSw14M3|f^&)6@4o`1w_Vrj_#7JK?w;7CXx3 zVErB`HwD8sPDcsI&u9%GocUkE(Z6%F`v2RM%vJ$)B}EX`X!5iOOK3|=PZo2g@~#*K z=bsT=1bpkNq@9TN zg>n^7iZFZ-CrI$7x0Ku!l(92blD|i-l_vU&1gHnH(Em{w8mZ@| zXG?*OPW^6|xdZwTab0DH`IEzt&4+i6RE9l95+}7TVk$IEKJ<$vDIpJTT#URlDZ7m^p3*z|SqD=Q;JWJcYro7MUQXpeTfr~ zLBRC(a^aX#ZY4zIv(W5qvaC^eVxV4|Oyhp%u6EAXgi|pkJ@RI{0YWD8E7Yo)Bux31 z%fREVl9zjSD92FsuDSDHkIV#ujE3ln}RUAfvKI7Yu>8><>ei&6MF8OCpIKVTBu?R3Qu3J z*f;PZI&mVy@7fy5J=f});gT4_N`GTJPR>N?^X6t=0(YRn?K728r$56Z!G$JB3)tjr z{0e7jYT!xo^GfrKy3T~wLL+uT<$PxiyXR;zzQZEoFHq$s=eGQwYiu{%Sa@Fi?DMd% zPJCnLw?-?q^yY+WtbMH%k)>#olW)zCW>I9t{ot&34`Jgduuh*Q}AfQR~pOz9o6` zQ(a^x@om(vFJG6Z5M~T{c<|>ss~ogl9@Glsk49jq*{dB%nK`Ztw+5?*QQD-R?s4D%bM9pn$wADjj@Z$~n&yff7mOcl~5 z=U;;|)Dj-w1-%9v2M1U7 zSzEh5AVg?$hqB`mt41HuDArf|vjiCN^zIvq@161t@^Ua41&fG0(n$1ZHb!Q#M z73-c@m)^~)h;g_v!y6_w?W)W&I85MQCxRrNxAR&h?j+6hI_r5(7?>4~KVn=q-n<`p z5uY~m3gKhkXvc9olrph3>KoSuj24m67$ zn;3Spsfega>>xTgv3JPAikIpo+{gcXGKXw*i9Evnl-ba7ShP6_J_ZT5hp>UosB}fL zZ#_zg@Jgot=S+!T8ka}Z%7Y=Pylg-qSsO9=Ja;HF(CaSpjLoB))5)DX7|hUf2TZu= zd~&xwR886FFH|ClVivutVEBhp<*)!jowV$bymw!x1M>`vPyVzn3i zU9hmbF)d4Hj-f-4(|CixSD>_c9mWDqofX2L3`#HX*mYf~54QH>h=uQOPH;e-)N#|p z&!h+zf**Sw%Oo+%%2<`+d@P7~zEUz@n`3OXcb5Iw!r{f}WB3}1w8Am4$=Znw85Dxo zPmKKo3}h*uo+&kUsdQ>rh+i}Fv0*i~3w{3`Usg{WQhz5gG6wS|PrAOk{E068f%=hO zuB{%NUdwW|3BJ2qr=(cny+WRgGP*;FHa5P@-*#$VAa}xlhG91tII9YNTXMN2>hvkT zZV~U7VBlqf?`;qQMXlpXoW7~y>X9cXiAbSVJ5#3~JJKG$Uj)a(1-F!mP&d>CnoI(O08pE75I?Z2 z)@Qstv~op}0nY-~pR-fU%`az%j%`#2c})N-Z&MRfd$q~V>Qtcvxd{5$R}Ub0qpu3R z=!#x+Zf;MUutXaI&#`sU=Adp`^o<1#iTy#bPmZ~TVmGQEBXsW&zdN3*}~$aeR-SjhCQ zi4Swpd^6i54|+?%SZ=338@XM$Gerd~EK1oqrB29m;Im7!nil(TKjlI$D>#sX78MQ` zK#1sC>i8Ue;0U}Ecw4%O-A2B;opH&Hro->Wt)b4Cfry4y<>c_NM8r!=8@jN%;Nf(R z{ilg+N3=Y~S@t2j)9Up_hR}TN8`eepPzz%V$!yilPU6Y4+V9?mJEpk4TKy9pOnBX@YmyR^NRWVbxqjKc;jVG%?`6i5Fv zQO^ADL@6Mr!bU?jYSvB$4`s3OX4lh>NL93Dy+v-Ts-E#H{od^Yi=O8cLo_Xrk9ldY zrH^d79a%8%Vu)6Ll~6wL_>?4W!mXl55~IOtL?4|}LHoi+HH+vI`>k6ah=<4T_4*OI z8y9g@EYZ7|t&~9{elG_>^-!cuzs%m#YC48J~58C;W7wiOjV zD2j47n5jRwaj!%;7wnz6S3oNZ=t?K;O|H{@9RwPmGwPM5cN9<0v%S#MBk8so%H}&( zL8vx-${qQ3PL#xWkNTcyuOiX#cRhMi2CuQW+i5~n6#PndEJELX@+io8M1DkCe<{c) zlrAcxX#0N?B{>oC(f|G^!ivG^WJEJwEsh3vdS5FFs0+F38>CO)=Kt_G^g~I^Z7!2> z2irI!hRb2>mIkF5aW*>_d)`MPr2$fUV`Xv5QZ2nWB~F?zw}q8R6K}Dxy2#CiI;Ru$+Gw++X|B6_z zB+MSX$WG6WhkizvyL%i+9j<1X&;M#akexwydS2WNSsdb@eF_Y?y2782qU1`!mk9P> zDORVL5ym|i82fnQFO6MTCI_;IrSN2NeueWFtto|{Dfx<<@(X4T!6Y%V@At^QKJO7G z@+SEql-IR3Lab(6LVcf*w?Rx#aG^XTB0l^7)1mBt(;xs~tZ~B)x4cp$+R-~E~LxIKjHq%{` zWIshr*0kwNX6#+{6B)&R-4Y-A@e+>p9tV)I8q%f~{w#FXpoDgv53&h^==Q{(-K03QSWRzZv;vBlxBwx!i!aGAC(~au9_4UI1n17OPa5pV? zgS;y|wv7)6;TolO>ecxo(^&LEA7) z17z-5)UqX^Z>y(JK6w=9AhF@HR@&g|E!Ctslon3qu!N;Yri&)c2Uq!X8(XS3`-qd- zVTYzH)b4PD#;;}tmPtelNH$u8*a-@=p@pp>rj<^quA|oQW z14+2h{bsaH4^Voq zctj2-p@}jp!N`kG4fEo_lv9JPRI{_BZrfdjj zP*-3p%nz7iwD#I9oNleT94T*!r`7JTd*)bneRY>>w&L`xW+jSXAAucXT+#siL!0oa zuE--L2F^AMs=T_St~6oO7J?cl1}xha|Z^D1M$0r$d88l>6CIE0ufk7Cvc zp|E7sPcTvs(3$~MsJBFe79|9e=ma}V4$aO?3F=$^dmFXWcAMro*O3f=Ae?90rwV`z z*@2hMukw8ACZbb~FV}9BCsPp2$D@EEnPU&Qo)`U2ry=?bOpK(6G9KEM* z8LR?4z3)MZ&^Uo&M4?reY+=>>PXc11AN1U41Fml=4#GBb5FfL*CdPIm3PcKTCa-qg zzdU!^JN0$n?;SF?$hO1`g!x|&J(l24X`7{}3ljKjjsFKk;;atFis<=MPL2!&4ZEj* zj}E*t_xX75u=iX3P{7h*UCT>=0I61oWX~E}4!x7+qD>3CF0?XBtlYP1;-3z#0Uw2f z5{EF1Wv0%o3vLP1{{VVmE?kmXYN5!_t9JBsiq*+X^?}jVoLjinO@ek+^Kk9(gW^#e zxGyY!0}zy^bA45P5{GY+!CXfHQ=e@Adaok5n{JY$Kj^J+#9qcgFj-M@n)-q!$L6Q) z-?Vi{2;U-7HllEb{j%!>ya|{LB(A25i~y27l!;lTo4W7FJ%lrqctE5*Y zR6X&=>=8Q|P@kK$qA^%dPocT%5qm^18S!=7$)WNI9uD<^!wGPgoy!fn)|YcZ@z2*x zeUJcwY$FQw0CZ_%lTOmY(71A+;mL_$EuPHOAEs`BT?aMGvn}k_X&=j9EONVl3QI^? z3&jnk#U}rmhi@3;@qZEJVd|$+o;3f=@ckXCX#omkIk>K80txCbG!K>jO_~G-skTgB zTPYZQXT5Ge+wVN~?aXSvNTlL}559(SlySMBHd0EUbAaUksHsRdx$*3`QK9 zPhy}-0Q>$EIqT}?)+SWSh8&s!pgRCR=l{L@(DHJ=!uf?+K0cQY#fxI<_2xF z3CI29CCiSZN zJApI|T*@~S_u<~=;N<0pWu*N^wLwYS>=Z)sPdg^iE;;h(ijOV*JGP2zj6hHmB~Xzr zJ1@hoc5V1JYY0FM;w6Tm;xz568hw9s+TGuUkP9($ZD%<$&W-lSf$fMAG737Auf<3; z`=^)Hv8&b%na350P0s?38N1^}3XE{tneK4~QCHC5s_DxU=_3S-#D3vZDFsG>byRRW zxMuZF-lT$c(1U1J^ilzTQP0=7Ue~o4hNZojy1?YsA0Lzb`ZDc2lF_=qn^jYV6>+e5 zYUH^vP5-Fw~~DsM5De@!~U}?OS^P=QbxvH=yZ6Xag6OQI?zxpHV`i} zTG2L(-@lPEpJn>#cban}?{2XmIVz7^pYaXB{`}(}w_V|6+M6TabbZ~YkdUy3f8xpD z8dA|I3t@SJw3akJacyllul6WzXLStL><*idc=Nd^5*{jHsm`e7=jfE)o zf36R=ZxfNQ{r~HOB2@Ot124%MFY%ms#!3Vzui$**0dMAm8M*cVgN9@(Q??3~M&(UmYR(k{=Wf!{5yeLDb+@W!Zr!s`o0=0}ynt~D zZ7aCCFx}=E6_GpY1nS5uJ~?t$9N)39ZA_1KY^%$%X|ESE%mlo*WV{NL&{d&tbx|tL z%o!qsiD_7xyANDMIBi)z8TeVFMu|uUw9-QL8TGQuzGcxLsTYJ$*B*5+<90vJ)FxMcHmw6+mt~yzrs* z&J}>ge-SwWCJ@pfG>TxI@OWe}S^j2U!XybjOfSM9xN1G+@8hp6toR~)@YmTPNy`yv}mF`RH=dV5^5wXjs15|-3exe9>ze*)Y_+_KdyU5*2naSM?mxAi(|EMD@PEt0IBn`7>}ITBmb!pU8o8fNVCC$rb8` zm&<>$YcP0k?$y&OHL7!QtEun`27y-1lsT|gz>EQ$BO}RIn(^?u)`SV{SAlzSi;yL7|3h+$iVs5PL~b z2+tGGtzPsZ6BUkPH@uB5c3N1LbbJ~d&a~eyFq%HuF8>03L@7*0a@ftS(O8y5tAAmi z+@?E*3bcj{H|+zoTTw-ZUGf6Q1hnAuWiY`>dJfq6{)pDxKfHFsC9i|QM^D)}2A9!H zd~59`@u3cW@95ezTpGQy;~S{0kcykb}Z-?DXB&s_Z)oZ)zL*2DFQ768%gb4)N$dlWvS0>z7_K-@MW2eNP;D zOGo75AozLMlJ?=e#1?BR+AJYYM7ub4l>&dHa>>Vp3z2h(-@i%myO(;icDVF%VU?fY zA+vcHN=c&i{xHho1fvf1CB5 zg#Dhmg@t@};ZpN&0N9%9bP1WUY3G7UFzkK~*Y1GMwN7WJVoZ7RF{(93nA)hfVNaZg z2dhqAu;L_tZmvE$hfU3WEjszD!hNKGMPkP#D;LtfAQGY@-j#$LqeWcGnzj{CaYGfU zLClS&)d7F6HmrBz7y^xQMzw`)ED)aS6%N?6ki0>}zhe*|KtLoNwrmW+F!eoHUS4ou z*G+|tnJl(oG&#$}A~^zYbkzr+q>t4uzH}Aw%$q#ecR<+6m!el5tjsPQXy!?@AJ}6m zm!yWs1qvH=mdr{Urq3L62=vw|_n%@@E@tg``{6Vf%`5z}z#%wG7#qJi8rYIhL7#l1 zTI^Dug5nz980A}9<+;Y=>Sp67A8T&7SY~E_HyW3=m_6%c*AEVpcc|~AWt#ZoLI~kL zJr9N&xqMrM+ka5OIapd+K564DKnkCE17{;bJNLTRk60Qxre=Rk2id3M?4vODA0tb> zA0Aeq;j&0Is*OKQX^M}sckDmK$NnATBi*ep%WBvu*owPLf6%7 z#7e+x8)IPLVRQ|6L>GYJX+XRcuFsNto^9+Cs@{MY+rY9A9{{eM_Li%4wa=+w`(I`A zh3J2spSxYw$QiHKHj=L^`XH8GY}Q#&`YISzEs5L8D?NJJyul)y)+o97G1S>KUBqas zSKFK8?EF57jKK?Udo8*3^!4?|&gu#=1a_x-ME+dN;sf20?daxc*^<^Gy1l-4y*O-l zLk8@BKQTM!+ezhSuxQal^o9ZX)^2{ebNTG`>(f8o(}LmeXIm+#4=GdETK4vhW<AvZnQB|y@!|@LQQ|2z$93V03+ja~iJWvgnA8_gq4B?Bly6fW z-)HMT{~|`3XoHOS_7;+!gEK$V1pkX5n0)Ec|nP&mcj7@`qWvwVYs?aI5i zr#l~(#Upne@D$677X`IJKNha#vO8_mMRGE}`Gmqv)7=M^n?F}`j5`yyVwxhyuAlPD zq473+2Sc9oQ0WbYU|JqNw5}wFOz6nKUEa-8A$qNBf`4Nu_B;jwi55`p|DQN5>F6!Rnd`aJs;O6ggX(7Y6$kt$CjURsDzm9Bjx@GRLXUQ2G8nAG} z%PAiCrA}T<+^M!x{HJcU&B55LDhQuE8F~3&F2n;P)H6U4cuv82duZ<*gaqzEl%KT#IUOo+WQql$v+5a_~!HyNfpBk`KrZ|6OIH_8R zlSegPD%OIdBO7JQP|kxqzIDqc z?&#aw9yh*QbUG@72}-E>i@SHexBeY``FS|5zxq{7zK&yH;Y+$wNXqlRlq?Aov&H-O zl4+Ep&E8Z>(l819wxMEa&&aXCLZ7kq^NMX2+_uBEeVk!jQz|6t-?sRnBPnnEO1?(d zbsOy$=21U#hfkq4YdE;bFv)ocdEG=t8G%H zwygp>7-UtA;X5!R&gy-Rx8{{NBSG525tad~K_V$Dw1h(~{?GXo{`Y)LPaa?Y-}BuU zzt^5^tHuzQ%tt4B$I7A4tSY`Iw@%9gk$hYGL*2aw73Fn32G_3gESZQHZ{T;E4!6f6 zdQQ>xRa4R))Y*@Ci7wbZp|5&!#2J71nbJg5QF}L9-px0i2;=3+*yL9>2~WRYAW`#) z-xtC;|0&>gf%&cMNek{tPr7O7h&hYT1?#eRm8Ewu{ksX8d~xL$ zlP?cSsWB~hnVidqsu<)?e4I)>*Ye>7>~PCAgEoOuKggGv3!MlXasVM$v$O2-9G%8u;4 zn4CG{WFVnTC83B#)=F`XWNQ=hYpIH2NB3~yPe2buwnD|n=DT|om|BV1qe}ICYWUha z9j}bfAMdhnIVs{2aUiXtIATg|qsqH2Y3e3pM#BUaq-?lCBstukP>vuO$Qqnk*yS#h z`fE)&SFa2RFlWEWG-x7_7a{`{fdUNjR+PSI4g92Ln?r_p5=UYQ8;~dvo=w`fgk?j` zXcxU|Az_4Yhj*`}%F;PBWImhlk##4(g@3nVYvTEA4Z;>n024_~KI zAFjX;;ktsZ3+-4n3gJwCF@hdhk&T8lAfbSPx&u%9@2GQPR=AIf>etfE z^~l}U#fS4pG?7GM&>KtW@}Ebx03>g0o{@?5_Ac5OE@-VIVyGHo&2&wYz*Mqp`l=tGZ7GWPC5;NnrVOUyy zDSD@ippK5PkWZcSFVuJYk=yl*$mRwjLXaN|!Km@)B99*?chiJh!@|N9ZaUXao^u2T zir!gaJ-#RnW4KY^@x8Ikvx%S~Tb# zNcTo2M*h2twFSYQ$qJ;6d>ehNj|!rSE~!wwh6$emP0`2D&bkr%DXlazcAG~~#0k=x zfWh6}{;W1is(PubyYp=OiEaw-TcS-6ZG9(|7hMiglxh>?lg1%LYUo;d=2U<+%J0V>G+XLm{Gh z8luFZ!X&Mj!AgO;oU`{Ft3oh}h_2gjiB`Dl)>b0Jm=PX^C&X1Ry`LvI-s8Q2pz*KI zwKHUf?FnXV5{)E6Bp0O;tpZ1I;!6u-1t&T~L|xsEk0y_!YMbacjcVAD(h2LAD7UbFHxOw^cBYr5YuW#@(K-Xj0EaiH6)@0tm#xTyMjhOlviaNC!$QhV#ejQAq zj%VoVzcNn7^wfIp)L_iM#B5`VKsf&@a{Iy`M-u0+cN%95*DVhX_Yj4iGdDE;U%?DV zvNSf-iAUy-zfh@rEJhbz@$=uUMz$$bYBgKIKh2DjbF`-&N|^?QWVv;A61pl z(2^5q=#>l&HQCCLa2pK{NYE(7JlLgaXOu4Y1cKa^qY7LaBgyuO}% zuJA+y>HI_U32aAS>3&mO3)O?eb-0wbldv67va;!kN8cvZ&*{=(9DUf{Izi)v9SRcj z8`xNh1QV?Dlx&wQY=APgJf&Pws@!P0^pfYcWXwcptrd!>=pcpFWAAYHpMN7G%s12QT`kKw9I zNv}L4m|NibeO~uZ&@J|BUyA;(iOy((~-jY~I1n_#%TpC2VR3 zW#DO*zpLz>IDemOOSNkr_;|2f0^bi!xkpTG9(*=$4Dja5&v(f*YFCBz(8g_F6h!hL z2CiOShEuAHKee!ieFF~pO4U_o$?x^IA>|a9X0&sGah4^ygdmimA*7tg-rTqDAYc{G zx*-^WYVYuVDM#1#w9=r$8_>58M?K$2abss}fR-8WD(*Km7q%|+PtuTO5>UhAHe}Rm zxv}_AKL==SVIX@T2i$%_D0KP>j}(!+0UqOeM`}RS3ExE!Xqf-c_+a{9dnK3@BYLUD z+Qb;ie$01R?~IO4)YyZx5HfM`(RWYH1vCOE_^#~~w0?@}#^q+Kn$rFCWBh56$Qge5 zEjjYT`^Ux;+mq|w&8+d$bFcsAVNE`)MVu?W$$+V(3>qr4yJr=z#VD-6$%0#VH*6KC zN!WirRA0N?2R{_u7c`7bPgIa#RLuicor57`7+MA z6{8G37nwP?&GZYG2!Boy*)I4szNEk4o+_Soi(%_KNnm)OOse@gWcda*9_b=WIMaXQ z0|^m{_kZX6Z+y@oj1Sy+|4x-a^6_4$q+_YwKZf3;fi6$Q1x1@A^28E50t$bDOv_Rz zg=&*`uReX2v*gygA`3CmgM{f;)SjOw?Ah0)+*vw*?*_VHyj(T;U-2lshPr3(= z_i8Xan};vA-goWyXt0s4gkLXQ`TsnIxMuZM$N_d$?K=U03BC?haFUbl1@&$J`unD)}@$K189G)4iPb%qtPjX@+mQKX2~!d;$0H zmF;*Uc^MC7I?dGB%G?^I{7d4}%iSynVfM9F){Dsj16oL#)XqerRbKw^p@`gg4-CGK zs|d@_5XIOj(%Ix|l3SRX#jRTu^&x%zG|2f2-D+U6&-%8nFCHa)I%&Xo{r>rgPUx<@ zX4d<&6ubBii*A;J1C&Ugcz`3%P_rrxsZwA%h$!%~nU*#!n9ArR2+|ZVv%~^H)nRM> zr0i&ch?`q1xl&LBDQ(k<)8Veq%x`>5H3@*RbGx{|B!%$=_VU?Roxt?MfcG&SXhBEc^lgh9GFwW{7g z3py0hYYr-xFyCdM{&DDm;KslaPsmIkjYPD!+JaC`k#9Q9z24drY}+o{3Ve1Q_xji% znH936DWq^I4LwYFi^lj&sD5#jf`n1jhRC>}Ps4gM)WJYm{3y8A9}=ivD>7O(?*J&eO( zloZ(B^aKB*!TSs=Nrwe_cBn@ghY9+935`V_pWmq5((L=7UR*|+14Vk`OyeM`!6B!A zyM;82rHG$PcN}%=QpXOUJcRu;j?%GeIVA9ok1GQTSFe-u=Eeres84#mW_ug#=F6yS z)QdkCXU6WY5%}(Fg+%6mZyGCo3^o7WtY5C zcr8pr6u8@}Y0+FSlo|1W{@x0;g5;M02|{RCVdt^^uOU!NfE(_7{TUppQC-8kIA32i zE>%vvHoxs*{iM&OLU*M`9^Y6zn6$YW_QIv-^PL|qK{evc{(MI8KPYUNycO-5=P&H@ zGN>Xz9nV@$o}e36HADK2$LqpyYqcjol8Y(4`Nl+{oWBOev{_V2Vve&LWCS_;z{(rAyi- znyaqAeM>ypdYC2;%No1#i*xu_n>hplVW(kI9g;a;WNY4&1i<3bf_=Zq^K0FX`>^H|@%k@)GTzRG8`T_YIR6E&%L6 zI8|CHt=x2|mrsV=dB-%@(#Qb3{O_d5GR#9eTEOGrq7!h?bLCXAx0lHv`>D0|3XjOx zP1&bUDD!U!FFoD4_kM6!c+5F-RCfs-~(% zRF5JXsshUaE`P$4)z#+H(R=yWw&7{oG1;?Mc=tmXK#_CR7t-nkBd#AMQZf0^? zB;-im_c8}Vg;hR2qi6aDNp`ClCYU$h!Ri2eTLQHm^aX}lT$FIN$MkHo=7LPn(@Bq= zSX;RnVDOhSydU(T&8T*cSD5bY=xrtSJdL+8&wR{J2atTbOoz)$A4iV{ z>CVe{kUkg1gm)%NdrOo?OhaPmyRj-nUNhBJXGNz~iYXPu1u5i#DYufD+LsaZ4{QOZRwW)8`Tgb^=f|4~KfbycTt_7}1 zeNN)?Z8Cd(i8_8&Nm?%PnQIhxslN_yPxQ0LX0#Vn=`+ZvP1)AS{#yHOV1x^!QU)g` z{ddwy)xc$=sRJroj_SimTX4Eq0z>?>p zYOdzPK-yD?G7RINAPD$6qH{}r<^Us{aoxs0KTMcKgbJ9**ca35=eKBhKU@w9pB%TX zPnD=;ksLYcpak=z%rRA)nBzdS)@A;#j*j8{?yCfzHyu#y(c%?V|5BPmyl2zoKZWvuuxQqJ zMnr_8NeBHcTo&YqaU$+DUQ#Dh0BU0F$Sa;1@#eaL>(D5#)cfPjUU!iN@W#-1&7g(z z;!mrhC6lhfMu4Be`|3nXj5x=&PG)T+^~<6)2$B?$W|3zpg4QS-^y5v8BQfrj?}XzU z7mzpdYs4AEY7eNCC_2xql0)h3TWsN-suLku(bIk#9R~vy9PxIj!Rp*W1w2! z1^Uf)?-c#*5OKAQh~Q34i(*y;L#zWq%YwF zlQ%xuRgjvU4S4K4rFq1z#A8Nrh(sJ%m7bIa30r(?Q4^d2frW7cGnR)i=%41I7pImY z4yl;Fu-zcvXB{(EIr1o%zhnUHMauQGsO}e(Gr~RyjOWDHzs|Yse;<_<)_vt?(i)uC z^X2MbgY(t>Fyk|;QWuLp}Rdb0__S&kFpTI@+6MPl)oCdJM+%`4Tx zOU-Y_*WXHilOxLk(@X<@cen9W+E=Dt^djj@vhAIvK-BbztnAlHUNCH^H{@e!>)Da4 zGyxB7_`XIa3J~#BYjPDV3_ioA&pC$GjfXBzmVJ4Th0xIOT~d9}xMiZ^hTfJ+*7-9<0F9UrGLEz$OxjJm~rjRuKj+CjHg;_y)(UE z7F@=hI&U$O{P3P==EKs2v|cuU&0~R|knr4(S-SgsHX+wDmI_hQ1@*b(%yG~y`dm}o z78*JqBoFs9-s$V>D-NyHY;ab)H?Ha4DR)6wu)NGZJznJgYqH~gq6`c(?8BW1kW+nTpm zxKy1W@*%cIy(*!g`A_@akX75?e;xKDenZXobuWr^bn6zJ%cLv0^FcQ|iN)WzG#b7r z=6sAaSic`OeJV3J@GaRMth;7xnO;`5G87%xa#yf=>8ZGYY~m~Xt-~z^-9O(GHRsS} zRC}5}nrvgmvvCbzPcgxvo$`{Ms?_IE%k2a5M!sV%>GH9t8&Bwl}6 z6j|M~mS<>_4-w%P4=jnJYA%V+J5_91eHLS4`Ly!0)5EvI22yKU0>fffVPqTYrSuv0 z$5MkAg*9rdZxKW7!b8I2!pznx1_N#-_MfrqVe1CYlZHC)H$SQ0xK;k1oGfnd*h{7w zc|XunSvs$c5-fP)Ak$#>u~Pq6ms2Z&t5hhmaHrwFX8%96cK>g!6r5axCQG*OmIw07 z?kQw=^ndl(Zt%!RGX5UuJ;#Xly^~pCE^PQY}1-PzSZ9OH2PEi zI;n`*yqY^triMI|Ro{M`3q`&ustz`I+MhZ7qD<)1KmR|TPITZq#}Lhow|BQ+rumqw z-B;PM&~A$J0%|qk6UD6QP8c1?4KrHjXct&O7a%;pP9Um5S)L2@y%}f3^NM zm_P`F3E=8POI&q7(v4$D%t4oHA~=fsiE+Z8H%)O&&2I+{>zkdpG}tmOc6jzZ9wM{; zB;&v?JlG->z5}GZV1YaSHq4yV(`Jj3KH)3&42rhLu>e0<5 z0ld82pqR!aa0P1=T*8Fa9{Y03dF2@WevW;HekeT0J?EZ|j{Bsueu(jWjKNpYDLLT@ zns#eJpEZ80|7cRa&>lUjY+}XTKM)97lq?z1L7N6&whl@Qp_U-Loy7E*utk|w;m5-; zskJvqmqRwpQCnxs)Xqsby2^dn@gb!x4J{zfmWTsoM2jLLS&6|57$JzSxh^{-s zi8M5cuIa?@0um2fDcPiLujn6)T?`N4AH~-wvXj!d5{DX(;1LKz$)*;_vJ6;Z_O^r2!%F9qt7T2PQ@S zMo%2P$fMb5jR<10swg_;__XIP5%?O*5xtk{&V`CyQdC4;PXPc@BL0=T42oNXJ9l^< z;38pqLC8C7S*uC`1ApQK8<@8VwcfC%^wQLih(>)an=;?je;kH3#K?HSE&%b~ofh>U zWtnf${g1?$(=%;v8{o5K4|))>RZMT~EteTDG$x zq0AISHnJp-2unBFs$}&|dckx{V~o1W3C7RC=kCnVyZ(Oqq)%(=*b;39{OZ3|Z!i2# zRpZ9;21yY$%FJ4|4CA};XU;vT4g8235T}(F3Et@P*JLv5#s5>x_|Mv@8xmi;e-0C7 z=Iye1sErz@AtNGCvLgxmkZ8KMC-^e-Ffa&;l%^qy5G7u{9TA|7f76b&0m6LreHlvZPc9LP5!{J zMNnDarj!e%bP4k|a)wo)Ka?7CCkM+mekCVPBDf`NZiQtdZaPzEc12kf#ZOkxp;n`v zITvYme&pT39!v=uKesm6;A3$rXsdXeOVX$Z;UxoPNVrKJHbHo(c7gBI#1xXo_3s9A zc$)R$B0x%r#m>N~1r6UTaNg+wbW<{uS1aC}zlLusmYkj%C=L?>zwT0kmLbHel ze}=wz%)7s8|(`2!IZ!&LBwS7EQ8Dly6AU#=mkVIDL7vg;=i? zwvi6;yI$($Pfj=`aC?I}%L0KnZH5zk)uG$CCil3VSK!2=GGM^llhsR=Z!Q4qSKrNC zS2?XMcy18kmUcF|_{yJB;_%78VGZX#2;IXj=!Yo_Qj^>923IzGl{M@j--Kil& z1w*H$*IkS{%E4DgBf11Zn-Fm}s?9tyx-=yq(g&4zFG5|ISth9UBIg)SbkY{#)!Z5+ z)*wMY0n*AY$(;-*zAiL~!O0KSizxo?hbbgYT1cnAkG%wjnyM0z%DA}OmsR_E>|P)6 zjk3U|_}HCf`?ey3WbH9*+mSz|6&UTL(4=N0nK<6CT`n@YqC~XI3)S2!Uja_UR~szI zCBGO=fxjhNm%m_a$yceIY8~AkY4%I*F_@hXn9h^uk=JoT7|E28S@ezu-uYeXw#oF2c-dD zsTrRyx<4x$i#a^nz|wQAjr3#v6bUDO>OZMYTQ>O3N78gx)9mJ5{gk{!y>BgUMpjW9 z=NH;>U(#)HV3}!c!P+PUX|44|7)uWy$eQYvBA=%Jmd8_xXI*K;3Pu!hPCc38qSxr>hT zPF$XNp{n40OZ(c{)vK0{HjXI(gfmPqx3(bOJhm`nrujSJm|OZ8O8op+X7EBi{oR`U8dpWMtKC+c@kUDL8sxW%^GV!tXx>IKL01tsK!z}#^FGipMNcVQ?HF! zIV5oI8&2+iFe)SVvpEEeE}8$UF)#*CMK#xm&+DXXzkfVUnsxt_r=qpOSwna$2f23E z|2O2h4~rM-iwTk7QKOn)D8_>Td*9!knY|&$5|@ymi6jdUYdj#2by$9H8AIRORLOP; z>p@yzegIO_4(#Z2I33$FQ*v=%>!!hZk9)%W4nZ7^zL00s*KsYrK1TYn%TX=;FeM%% zK!<3EPHesw%m~PtQa36j+kF2T!CQGvFlTZK&i)a|M0tbcTy*%gI?msEzpA>%e#*HW ztJYBiNwm};Lw_z*J?#>V-m@e&eNW%s8t3~dqTOxuWV9c^OBa_eV&hK?W=tFUiwP9-&{^E@1q<;Q-}C z^IuE+Oc4#^WicDA-j!ePXL$HEKL28GcV2+K-$n1E$iP$n;gk4kj4`l-!2|&jux73*4Y{W1riB(Gq7X@H&1t6<|mFh zSIj5x$gh+$`hQC|e`GkejLbYp1};;sKh~~(Vo_X3Q2=`_oI*10=&zLEK^c(5Rn;%O ze}9>H(O5Hotov%kc_rp%5A0fdO04aZ9^RBAcEl2UXWwa_+LE!woN{b3c(u}cbDID3 zyH>tv2_eI4xG)|c;*AxhYD?^eXNmpa%{&eDpE1`NVf%;bXHRW%j#pnT(qf{{e4q3S%0Lg; zwFFI#@9eI+{VZ-volHJMDt!$!qZu{5n}HzFiD5&g&7Nw5G?t@ML|HwPnyy85BcHbR zb7~0oFq{E;bFT4}ZZu2Ox7+W6_Fw2Zy%GFh!m5&d%|2vprlf8Fy>J?*<2EJ7Y} z;|i})w(9%f^y|~{jl=3cnt6d?snWVYZ@U&xu?ytQmFF?S%RMZ7`NG~aNaNf^Zo>GF z?-5%}N=m>vO;EL;CH9h`7U?qC6R=a0RmoK+rA0uD<{Qw9Z*e$9>p~;tYgV>z*zt%vmt4%FsJ{ChBf*ufyz z{nV7}r23<**4|wqF_gu;#zMm2A12K*E4O zHbK9Qawtv{LlV_n`|fZ*U#-!Q(=c2QRFU}+zMhD4?{|2Hvo+L3eD>~)N0Q=hA=!yi zSMu9^B*FH1uK2t%7HifMvsxAuQ&VPSyr3s8) zyAT2^90L97jFmof+O%LkBOD|btvT-98)rG){IT#8z`~sRsLS{rcG@<81;L_)ey1>M zGB8w&t@BxUPa8gKo|P9qycJmNZ1X#wwe{a3hmzN0)tF7|1rO7;I5gAejEE6(QQors zpqs3SCmv7I-JGv%jnV>Lt1H*5bFIa1LL8+y}4$WnJkFRo%rq+4BiC@jtk$zbFVwy+I zogSjV^Gh6ttn#G*KEtmx15?&60g2h2S4$~A*P?cv>p2aJk}Aj`rtFj#lV(|gJHH}qIasm=h>^$>t|mG36*Sql$A3jauUofhKl#9y$3_6;PU~?HsB4Ki%rpl;7V$^& z8^}j%yh@=`yM;G+=k|d}rn(}%-9LGio3=Kz8#9orgNvD^-mv-wnfTSl)yQja5MkCJ z6j;uGGC8LArF)7Hf?n~P)<)=%U%aZ%$LMZwk#BEn;#UZ6eo66T$Ka|Td<}jUlqLQf{kbxH~$NQF1=)&LSAp4^(-{M(lP>bgSQxutGYS+_cy%>q`16Z z4EpHZ=<^ckcxW4D7P3#diZ4GaY>C+B`K)YDv94fD#&Q38-^<)1U%TBSqEnN!M{*D$ zOY@s8#^7^O?)~&Isi{)lJk#qsrr#1f<+aAKSVsI0mU@GwkPpf$&R|#*tA1|oZ;#fm zJ@X5Eat^W0U(I=F^9ab?+XfXi2ep!l#^u-c1*b7jkH5n3Vrj@!#Id2nF)Lm(>^a zD!INssl3?w*U9uO|7TPb@YkP`@%)~|vos99xD5*Gzur!7VAgDcGbPx^KiQro-`1yX z{1(xqz|UY=YW!}q`-g;I`f3@h^Nm{7Nt# ziMCQyonX3I=!WWE7o)N+2ws~$WN+?wYQY)2i&<#UGD{WOer(Jl%^SWEw&5Z?zBL3@ z6mZ{J-`@*bbVZn7*uk$}EyVBd9ZR*sA`MNh9yohbtPNY!PatD=H?qx#57RRh3^>|{ zH8PuWy&eYm?^(%3i~=uYKxv=2U)QjN@(S7BM=#ywv{oKi+7Lu3ZOqb)pf#kh`vr_P z9Aj!Wx<Y#9Gqgk93;c7NzP+QD11EAPE-0#7ltcy z{J8G1B$k~_^tss%WR)19<69ObNR$@l#9`&LLPy8q&{%7dJ~veo?h(fopJz0qfl|lY z!e~Y}P_ENl=2o{9Sw)#vMko5K4ingUdPIsLYmE5B==ae*F6#DBF^pHeebYE+3fWaM)HVj&3_*%xkmcLTNhQ{;Mld6zlJb3RyV9x0Swi1 zVt%uzjWv>ZRbqqL0v3~_YcnW?o6E+>*w};*J2F@GdFSefZoI}mrKhNDx9{`mF8llL z!^*6685aed*ta|m#FOn#WNUm`T9;r)=IrrY$17uIzr{ACdC4+Hwde_JsmiaSRLA=5 z3}*zV-<|FI^}Lj_c$_0CtoV8{@;Z^vThl}+jDy-_2-2~p(AGa{?n=~BP48U`k8A3< zZGt2kiibZKnZhb}l-x#>-A&sYO9{z`IBNB8Gj@=1cG*m+CCyO~azM3p_de}qeb)Tz zMwz~qIc4{YkARwURsEM`hl#uIOYI_8QWlE*JX#vhKAL2bNsyI3Y6wFojO=7omMnqL zI-*i4rh(O~tH)=*&c)0r+~w~5$zwTWx36g(H5FcaTxnb&%Z;X!RlKL8^QQMI@e2|C zX%np+n8IK2dEgY85kHX4hS1iM(C;NIDgTZqKa5GSQvZO-A9bbQaOjM>1(?H-f)`f) zMg>7B1V;(0Wq0(~U!~Qc%VmL_>i$lw8epZHYum5N*7Cc~@X@M^fSaKXZBg*e?N*oQ z_0gIYdZ21l{Z!n3``%j+1)^Syd&^=W9*}$Z1*6#(A6yvPwHTZt4Xr zw&y@o;am^R7St<^lySvbeku@Wxh&%hlV^FRetr&?c9G5;22Y;nx<7s8!jHJ zT=%T$w3_rVFxq^`i)DvsY_swkh<`#&PrU%ZTMZm>x)6tIz9EhpN|XYXU$8W1UlixPl`J7TnWwG z!(&k8d%OAE1}5PlYgG~&&ohz(WKo$jM+w={&~n6y(sEl~*&x_<`&v_S<^GvM=5I}o zbo^gG?dn8&+{@L}y!cIega?WawXCe`w>ggo>kc}$S1;7B6FrTBiBH!$Zp0m>Nyvu|3?K zyN5q-rma3`10KV$B0^(bfiShw9&jZ1G#z6$Mb|Bkb3>mQxTOf>$DZns)vEotJUD=~ zc+Up)t>^R>MdovQh`1EJDTk#PHhkk?_g68dWq>{}eFaH7wYQ+cbwKlY_TnH^1y92- z*FzoBxJ&0i6*BQ3?}q>d>QiW#Z2t@D!M$i)d8DY;WWHHdK&C(CW>V2og*Px){Ho$# zBZAsyOy>O;@GW@{zL^&b(bf>zBa$)r*Rv^~VAq@iZ7F;coM<+xutJ=vCeE;&rUzvj zP6v7Zcf@Iq8!f%kyzFl>x(MHi;lfyOIyvO|Ff=1xC551iNdB^xt(8+OrO~$}uCV#{ zQ7GBz~|fsLqNz`?NNURtd-2!cQ1Q6@?=DV$Y&sO8y_*TcF@RA@IV*mKIKV9V!cU z|LDLe8QHnf?^2FI?jtddMT+qgxGaQ|OUFUw!Vf;3s4$8OHXiZ8dHm^jwsABG=#A== z9J+|D+upU0s(0Q4UA#0=ZxnJH7w*)w3+#$}tJh?ewaB#g%?>NvMhqs^!EMZ|o(9(T9bw0SNUMV&AzYnk61ks(< zi<5<9&HlyyJ~soh(@hr+*;pg>J2At4%-|`x*U1Evb z*2Wp8hFb=K3+ZQxk1+)eE4>!E>T%+z+%7&Yf$F$mBXR1}6I0VkDczHU{G-ufSuq;o z9FN*%tK%J*b9>k})7)J$0es)8BwZ+x&raYi**&vSjWNCktauGjR~K?$QZ7T=PC?wi z@Tl~Y0fbg88#(^r*Cg#7R#yS08cHaGOPhm=c77o@XL0b)H>#=Hc>)uS8FjUQ<0D@e zR16}RT|N+MKhaB4W+IL*elX8oBL;#;4KK4kTk^}@+FgA3{Cl9ZPcNj5Awdm1bz6*> z(}URDfOquzQu%0n07_vg#qps>P<2y>ImaL%U^t!Hs*7?yJo1z4nl=y0uZufGJs#(Z zcpI_rRt-ysj5v<;bM(cMJo+Riz94*!+)k0%65!l4rg;GS8gru^lm&@l-ol2Xcb&}O zJqt75G#jV06T}p)AmDk>1Nls^x2J|?+Om1FAn*GZX*5_Ld>Dz3PuLpSbyiaOp3JRl zDU%W0`<|P4KMSXL1GGzf)`N|V8MDtG*glgfhAWVG*O9pjb|i*IcxV2Il4oK)CQ)Sx z!xcMwR6xo3BZhUablV@S;g=FUTE5_jOET@^L2I{#OtUQXZv6lnwbk3K=r+vY_ok0x zK&%8fa8Bmn)TuC};Bjox{u8fDoMMMA&ejFoC)Zlke6fz5(@L#s7X8IzR#iR)v8My{jtL^aBEz(o~zF`J1vac*KMmidIYApWrI(@sk zQzmoveD8Dhwi?u;6NdQ>OHl{VjrQ2oqF8QP zc9EsBO2gGhMC^3*cJ9#jO(-!->^DiC7E?bNlN)RQ_Z8Q9Ab_ zWy*Cdm7c&y4gE7~K0>7nK_2)Lwor$5pnHebYZH_p@0&aIi0D~o6f`6x7AvE=GPWc1 zP{5ANi6J^2_$ohFBmm%t-gAQkX%7kr9OaVH6j8+Grv$`5>NSNuW$Hr9s zuzpB=yIf2610h9`4;$fwf4A~aYi7JUlHIFgQkpdMyV0A#pAkw`IP_Xsxhh=#((sFG zaXZT)QKS1yV_%P@rK%cHihCG+k&aV%$?cfkfbCpzKXnDA8p@SOb&=k0tnEwVZAx)@ z6ggf{0c`E(bj-8Qr$Lajd!3+?8^ppKlc~9_B7LSe`TOhRUn*aPY()Z0ExF*PZ@ryK z64-9#rYfq6r(AtKXy6A8e#PzK{K3uI9XHAbW8@6G!An908`l@@YespCo&qrpdoPX? ztz0%aM*XSPEk)f16=%0THF3w{qWMcv(v>vfL_~n+r$qnQPeqcnQ*p?~pZ8fxcST(h)A&g8;4 zN~WcK-T95ol=GYyWR9mHuA{bLqu>m-xW*(*M~%*I6QafN@D~@i^?u!w8?y zM!8e2He;MRIC6-N6lLoEo`*$MFCggsM=i}yU}K``>ix*H@X_ast$T{2gyX-IH%;ag zZIvCC6_#O2W#coD`D<>$D;Xh4)tcP8!D_T$Q$+U!!indjqY$n=NqmymN4sw_qu?A( z8$P%%@B!fvjW8Y=#BTAt^61W4M#j!mzIlD#q4(3q&=fNp0RzNMUMFH&)KlO1Kbg&c z(=`f+P5!rv;9h)A%`93RJs)&wLC#^)%f8i=94!5YmaMzFE0fka9BEa{E_w^$1P*qK zKg%4_ku4sJK}z)$=<~2w>VtRC{J*LXmK86a=g0?JWgYZ6jM?ZHuZ(AK*(P7Qg%Sap1sbF)9?aN#c4bIaqH3l1U@j{SPn;K1pM{xhyL5QKkPl3 zGs)@1DgA1}n}I{>(DhA7ZmD+W+3yJ++jCqms+LTMy2!yBjALcKeXKt5VA6fe^>sjh zyhici*wn6btu%yNdsd7Tu<4={3&bjeTchV>~nk^-1`?!Z* z3BOAJE^Ih03nZeV`@CTwx9PLMukPmm@>v3*EvH%_fh3Lj+lXY7_MuiUCtjFNml)aJ zV3W@k!q#+ujtxm^90v%<+n=(k6SkA%E<$0O-LlNP!`QxU}CPHgRH6 zx5KxRa-u0OmN$_UzNRCPV)2_kjtLRe&&W{OdRGA^fWY&;dkrrb3~9aSL&RPKP3<5pK!_G7igzbw@^vh(x`TcDbVsvnOWz16j1`5k#Wt> z;W}49P30Gtp9Cg)umi4HCr${~Bw;pi{X5`^d@_Q?+pc8)WHZs7mWmLQ_ECW}0ye2g z!%8iA918a_tz}^`#(0QDxVv<>`hs7ctk}#Y%o^9r@Mp0-LwBJ>KMOMwN4s_bfjj?< z-*WfWS{aXy@iU^?d?u9+Hz=kHBi$3Fp_r5!`Z+#>SIk4uiE=94tT_z^olC@(1{tR{ z5(Pk&k`4EStoDhxpP?6Hb_KhUilFyxU7SDUFjJs>8AA(9fz5(x&Vs+|hX;H_ngfKj zMr#j}4RWdz{Mxzy$^K^D|2RxqI9UYgIBJ%79;4r|WU z>@e*Y8L>;JEd0~SgJ;%>i-#6Wh_=95)PZ8=$s(EiyoT%yD+|Wh6Wu3Ue=eP~tcNhh zcJg4#^&p4Ez-!SIof-qbcy0H6)FB|wQ=NfIi4N}7AtItT}OT8gW=f*Exva8PW$DC%bHV3dW6 z7Qz@39O$s@2;pnAJGXJ}mUMMRIq68`&^)%QCvtCdJ%qC@uC&wR(sts8qEUCKLI&I&?Zn_nyLmT}R{N$J!;Am9T}a8cLwLvw#n>8nKvR!a1@ z&(NDgw$2oh5!2t0#C-D)yg2K35KR{|tbj5~oHwC`4Ajgj>dDnE8O-_>V0JXi>`)X> zWLqFT*oFt=C5nAR03mHq3j>7U3R7f3D@a8kY`D6YQR|PWW%6caC++%op4}|N_X=YQBwY19wM6a8E`f4ftBWnI0f*m$ZP3=uu$02$ z%>L&+GjB4!R-~0eI%ILd1`C~T|Ft>nzb0;Z1OzeS;g7TxD}^IS=ll`6aDqy+w?MB( zSkBa$WR%25CE(O?R8h61B4M9h?ZGS%#FaNG3TzZyd4G)A(yyF8THhN4VN}t8@g53Vy-(CaK$`!iV?9K1aM7CU6(E)WpU*)Y1mW? zhyu3T^+2AsFhv!VfE<7J$B8OBL`c(V( zY3GeZnGF*dkF1SOlFKMW_i{Vm(jxy7oTNV&>acS=5vxYp5_=r;o+ zhr(jOxR8>nRpsfH>kMZm8IjN(NJnb zf@F%KpDe=kKe3x+C0+Q{z_o}#WIWR zLsG)K0_~?&2nWzWTH&vPdCpa9yDk~0xZpn7YP%RhSpgPDbLnc-%1<0*d3=-<^#uLO zmP{_HO>$(^+ByAH^ax5e>6bfFw!t`PfEpq$$J9>YvoTuEaIYoP*BF(@npl0?_Q-BTkN^BwGxT?4)!g=Gv`g$TG}-hOl9Zj-6pGQq^O@`k1JA880MLb|RMmp= zlt5oTuDy~S`D3<%c88B`ZrUyl1>@0#PC&wE`ryHKh#Tr}*v6#5AT>)auAx;;WT1j) zy*rgS>0>cwq2HW=G4jcvf?Wr-p`0fYiCx32y%&F^F4oBt{S;d2)>sprs!D`AW*DC?^<_9>fRcb zDuNt+Uc*+_r%qOTVZ?y@Qu6U}`VmQc@RaFFmeXoQCK5_H`jWL_ob~CROnrIUAfsjlP{v+(eifnwf>FNFuM~SH#Dlz zDci(URpMlk-y9n}*aaYr$te47e&Jqsmp-u6T9&nf6aKE;{3tpEDhqr*#J?h{)YeJ$ zlwS6K{E&dJVA5xJbOl$L*ZNex;wrC8#{W0&vxTU(V3m(i#)4|(_B$G@-b)Tzq+=te z%Vg;%Ulz{~XC&XZO`@h(qYUOLsGcLLk|H*j0!2@KtdYSez6itr#42I>{MRP_PdYAz zbWaB^W?CmYMw#imR5M&`JemQ@9_t=lLdul>-qvN-3<55)Pf?wgKV(0uAJP=J5fsv- zkVdlOTB4zz%2`!sXVE9n3z!vwdIQ8IJ32m%|3^Z%(&U_hd4;ga| zbSegRa!FZ^@3e^kbUA{ixwVw){#MLM`7AJANxi@g^E6GWXBBUsHwJqe@XZ)p(5=7y z&={tANWeFn2D3Jt_vR_Po%xz2i~wB<&m-Bjv+uRE?4PfV=-oa)yk8cCcUZ=hI)kru zJum;d!u@(UP;U9IJqKUL?BSKCpfq=C%ezzkQ(td`IZ1{E$$I+O4`*ElL`VQUFw`a! zGXWo4g-4>hcPNW6F7gFo8|y&qFLjG`^Ip5cz>jW(&%7)sbY89q-MD_P&Wk~%E}_ey z4$=t?GvR*^vZ|No>$WTwX2v>(gzGOHt7q@2Ji@H1<1>i1W-qy(69&9_?@~-k#H-HV z^Cge>2S!dUQHU}%A>W@#00sY_J0AQmRp_BI5>pa@baUP>ks-{SxM}284p&1_YrBGh zsgGpOgcAS!?3aP!6UUfPfHr4$9$J8u>+M$mCm=Omkg7|lu2X`R~GsFUSfLy z?FBAAdM7*D&S`%cZkdC9$9gZ_rS$i>`!C zU#saS7a78nXCm_0=y9P}48!t|Vb5tRU&iOrbtp*ObV6=JewK)yrbgb+&p z^~m+k5144H0CY5YYK-6xjJFrrU`bM|f6S0MHf_cb(p6NXOSbLCGn99GFm>ss&RXm- z7tPv)=DtBd>Qzjjy3rAJ2R zqX*kOL$7PX>=a@nt+Z=Hs6J12>>W51m7+v;pQEC;}bP&Yo+R!d*+D9R{rd`00%V`J9lWFuQJ?ZsO=pzD@N6qY`B_m|s~7 za+NEqpQEK2{mpY87?1chhE(OFVx~5eewG$}!YM-bae(bJ;vB?hI5^T9i5~1UeFx~| z4`7Ogo+3l!=DfTt3$A(7!a6}6SVC=iB|z^p|J8pH@>3=U-+~?LC~@dm>F_?EN}Uoz z3V4C|K&M6X$xcZubXGq`_~?_#NMun^_L~)K{vGs+1*LNabEu5zG|Y5K=@VgYm|_*gUBRx;(3?6y-*9OqVNM1nl?7aRCy~@ZLsIAus35(~x*0Y)8WASix_F2> z9b1i;a`;_&OTst_IP&G>=M3|2=Prpg%FCm)DsvD1AUN~6mT~`9H*N~!)Lk+;i1YS> zn<*ny`26Tdr?R#|zg;GR3tX=>x75=5F609q1yR%ysodH`KuZwDcb!?4BZ}By!Qp{} zh;A9khRaF+ODL4$BmKeN_10EDbNDs_D7W?4atO}kx%v;>fj*zq`zvZ5{xw(mFy$W4i{dS|fOwC727~CZ| zhKT8HrWiCb1!R%}@xCI7lf#}blm(Ui95(hr#?t=@VFG~1IT-{-2M;^?753@In{Kd;Z!9~BDL&RFM4v!;3-QozFvA#eWmPX!662DYB5Y zF$AHAvAYUFTUHzMA+x#Q8s)@fA1W_ozOvTz1TfGA!0;|2QqD!ZiQ7dBDDH~8#kxzJVJtQFw{ zYMtLnYeK|NdDo^Cq7lGV4n}FGSAV;&e?3e`^gG$JkuR#NGJ=G!{k=+C9kJh+BrM~- zQ0taPe!~KAWMsh|CZw*laB6X(TNe!LoNi?2hGCkn_;d7feEK+Qi17~)9u=; zR&wMr$^+h)D#lp(=Jxe?xtq>gYrO^-|5AIR!y-blx4PGHHK9$H0|FH6n#3+~)RS7d zE!Sm>ZY9^ey`4+F2K`>nuq&||oGlzgoRTkexrqlQ2X^awP{)o``KDI5&yvm8gA6OK z;AKx~K%=gHb3@`HbM@pY)_{sYy%T4L@m!b7RmhzG0g`1AYEN%%aZw_=pLL_ArmE($ zvll5oUhf{3@90=)DzQ5No4J>dzaQlQ<(e>Tb&1k_Vp%AW>djqe4^{29Q;Zci6J}B;^8HNnMUr9Y{DsW z>JzZPhr8)FSgf&)l{<|x)WjBAlv|$|A-CHV?_2sLW$_9DKs*Cn1{kgTE6AC&X;!Gl z-D9&Z)UpwNW(F10DY;QNfP{n|ur-T8MrN6hQyjI$I+9}~ecuL_FVxpS8{7g-6J?z9 z)+|X*AVXQ@&AK`MOrxuFL9}Xlg(Ak>{%^5#*z^m)%Fm-0Fqg77%aT##<`Qvu99ePz zSGxAK;MPdypNK%`;&FNcH+PM5KYMc^3FOZRfU&#-TxByU|wbq5KPb z+3pRa&~Kv=c`NwzyV2%t?qj=H(g)+x{V7@npxi-?xf>f!WaXDjY#h&_ggsa}W`>RPW#O&EHXK;<`P7RvPvem{LspjZ3& zqe)wz?RpiPG?}3^^Xp^h(MS?AxHktnUF;hbURnilFR8W4bAz46G0Zi)j^1crj z`DR3Z!@iaF-G`4f-F?KWH+Na@&c#hVk4^(O|D)XdqB{S-XqaXH+xbuSqFL2>p}OnV+2EcwkWRb7_HL6Vn|g2w~% zy~JlU%xne~wRf~_Eh~K+mrrsTgMf(z$v=f~! z#LUJ;{VRHly~3uWKegpNYk=0C87r?DfH^e`E#7`EnMj!qTlpn6iM9)Y<|Pi=dguDC z66Hul{+9dEnuo`XeCphvrhQ9!9yUxYNtaimQ?KqErZZ)v1RD6UQRaj*HhXHR!`7)S zm_#on(Z7;dIuVqO<&E!X6#Hf9UN*a@jyS2x`daJnhQmLWzo&hRJBoo8W=44y9ObTB zceH&~nWwmLU?-RRFYFPR{`?X?Gdx4z#So;T{Ym4s*3gPl(%kg%`p2Cc~ZOT+wb z{Fb>Cfw|-)KOa}Rp0tGre#l*F$}2KP!Xd6__*EZd?zSWnEYjL~ZF0s+w!%>(#wJRv zkb;KU@qZ(R3u?sR@C-|y<*w(&`TGu0cir|FefLLlkQ_TcYAT--GtULDyYaxCJVirY zvogZvGtqT%SN^5&`;5NBo!tbn2G_u8O0^n+&V!;aUoZb%d1kQ9R9vkCK1OdSe0lE0 z;6*+H$dua+6#FYs=ozk9ns3e6JXrHYRn*U>7`!=^8N&sE%OzLpRp>7wEN0unF6=)R zvThq<)ENxUk$VMCUZ7OK6zxhbi zMGm;q&Im)l0}CRukZd{27@v4>ut=Enc-|suJSC{L?(&!s4q~WavPYs=C;`&u_VMiP zinAuq{l%%qZG+iK(0mEL&oeynM{d&mr>mUjv0o{BOPipgi}lJkQAzOZ_^gxhUQU$Q z5L+S?tKQs*#<>{oHFN^hCG(mocvJ1Bvjca~{e6sNPdsh4il9jScn2GBmRbI}l2e(?db(I5J2y(kJpXI0WJ7V1=|~-fbG4= zsdvdoU}p`{4azZqT0Z1H)^Kylk<82l;o?^TuwoHP4>$v`oACt`aReqF0YPnV&w9kQ z3mrtAL7ov(^2AuMgiUA?%gb)ZpX>Qss`AXv2(UweT$)Tg_^$6LZSf1cYeT&fi-g&( z;0y7=tae1}mLEUqn1Lwr#E zQ2hYGwVYDS^6$(~pp8*Ol8xG_b^X?5F5Pz8#j|;fvIAivX;}THTGaD}0Cr*cq0}6w z;IX8Xn_AGFb4Wc}3)NhAVVhoj@eNwG_7f9z(x6qk1Yx5GMk_H7KPGl7_prCha2Zs=1CTggIH_ihZhRDD@1%CxN1NTQ( zEX-yfOP{yY+3QlpmjWS(iBe#V^{mDSYDCl}fMS)hBF~7G;4FC*@KP}!pF3I~B`(r1RQ4?wcu1tY@`krvP<;B@G|r>c{Dm)QEog*nK3T!a zIPRZp$9C!8cqjs{E9}hpGB#pLnwo{XxCDd@6#(>pZDdQur-}>G|Hq`}T~kcD%X-y; z?3B6i{+Nb&9>J1S>{k#N?+esRaXBGFGx}pQ9MI|%F20x)^3)RbGcE1A{MrK(M7yH+ ziA0dyKTDL6h7IUm%YsZyHrN{-VL5HUU5a%d0qkb{wx>!OcANm0YBOQHF$-Wcyyi7d zWROD-rfY;d>U#ZaVN63?C%#>$Wxv{EjsA&=_C*Q+{$m%W5I}+ya?7D2n!~cS7b^yg zAmQ3bL4xGlgwL;l4$BhXFXf1@NIUOR#E(a5wDq)%K6hky7AOFM0^llqb@sM(ZQTho zrNb&*M{1msuKO1)tPHTDu(xf^%wrTRi^od5X<*tE?1A5wO@9&$Ew;APO&w1lP7RJP zjYR+C4S}Z_-!3u#oU&yLY(s8gH-9a8>Bn zOqCSWGYJ{CZf5Ei6^+u|DI!8G|cpN$ZRZJkN(umYp@b!)3Esc#T*_8GjwpN z2B|!@@NB6tu+dSyq(0nS9ic)B#=I>dfd0;($&XwrBO!Sb`W{Q1976GALGv*2Q2JIS zVGfcqiiInp4OvI%TP7YDp+;;T^x;1eH{X!zf=cVR+A59HuI)TA-Do%z#EiEd#FlHd z*;}0-0|}7p;z0IrB<2hvD*61ghb#s2?o8M20(WxUWk%Aku?*8I@W1Kf;F`JT%;6G|avK11TRANF{7lev3cjsSFthjzr7E zzNkLg_=ir=m!oR%&43U8N9cUd!#;aEoZavhlv={=BK-HC9uU)A%_fav=w z6CjRi@$U*u#>j>OZ*J(@Roo0oXc!Rbdk0iouc*GrH_VacKgSE=ymvhiZA^qseedtZ zd%WnY+%|Zbgo1eTzTn+cjZgI*Nj@eVYLb8{d*uhf<&n}KjYpkRg#stV60%djmF_*y0p1no=N$7tY&n4;6G`)zoDG`O?fx*4lo1r`f_Zjf^zr zqOqZYp1lP(E=F&4lei?9n3Q!84+$w{ekvJh$4|~C?S88Xz(Dt$=-LmY_R|My-<ptG$_p9@rX~MOde#RE^tmu_c@qXr;NymRhA@Q`GEisjywOnT%F% zyUA5PJpdZ|v=9Zuy+8X%eL6VCZJ{M=v9IqvqiX&gy&an8Z7#goIm9g0S$%_8zy%%Y zg1olhZk8-HWUAdIK_VqOaVV^Y(evQ&%qn6vLss&F-H|Swyt6Q-8m8FVeYwR0lx=G# zZA>zns|PMfM}!NN*?M^Y=~zWpW>c-06$7IqDCB#jvC}*7WA6z=D+E_pl(RP3C@fbu znOOXuF{{Qg@h!WG%tLdq_6OcE{Ld3+$kjx<5~zmI8dc=g{*f-3UFdXYr=Zd)nrAjC z==>pe=-N(FfiHT`THdG_E|S+QsU-Xk#zyCv$dTQVxx;>w zzjZORVy9!z_ndj%|L?m0uyCLqoehp8`&nC?rwjW~#(r6CFGkIQ@liA7tN0>a#e0_Y z)$N6WZ8rX5GB+f-r`n}MX2Sd-GAm_-DR0?aQdM6mnL?kH&UB7nW1QKM0uy+1B9F zfql}R_awIsy`qy4CTGFrt*<%urO>^6c?Y=9Nb4|N<4*QPe$c79wrzE$XNcTX@Po1= zs-tRkv#ux+i%9=@qif`}hHPfHMP~Zw^Jh}++x1lbQ?Ar46B9bHWp}V3A3)51PWMFe ztzuSVZRNsWD|kn#;q%3ee#=nWdkCO6*@U~sR1biTVV6dk=F^@o)xJo%-L zO2JuTP>Oc6_16IhDlPjmOMpbq7%zMqUL z=Sf#G+DO-TS(+1Fb|56*3$WU3%kc-O*6g%&0#(!lWbhw&nZgfriWT)tl8{c}hRol2 zaQ$-3xxeAxZM%~n@y9-Ov4K%tWt8&e;*kvPJClr^T+jG!11f6a8P357EUv93R#J-T z`4c&XpDZ7P8jT6qvemrRaUcwjEgJX^6Lm4dJ`#rI2|9JI9SgFYP%}DvVes0S*Xa~p z&R26lM$r4948k*S$UF4(^>y@@Zc!V9;wDgV7(~z13-SCxOr<@D!kT>Kk9NDpmkT8E zSw|^0F4Fxn>i-MA6hZ5U{FYqqIbW~Y9@@RIVxKe2Yd+bwvs-#O@9%cOTpoXC?=Oqg zw-%=5pH0v&kYq<)+N_M~HEPAP@>cyT=6;BZmcTwLzxQ@$n451R(+#JA-?0%%JS?N@ z()<2s`k~3el)l{s8mXOWShA`v19-3BrOJT)?VQydgemJnD>?aClk--OvvPiu7pfop zef?F9S!eyS={kOkBERKJ@pjf%y0nM=|7zE}HsNSwMwjv;?^`Fb;b~PBnQ3OC=n(7E z%cgDKZ7#5v%i%Qk$w$54)fQ+PV286-jFYFAZ3B}oioZWrF_YY^xY~8vrcO0#H(6IZ z8Pkw<22~j(gr;_y7P1_z3_8^#BAC00000A^-q{d{sO%mg&~8 zYJltLXweIZwSIw6&yjk<##$sIec@6oi{hbb)Nu6t*2lJ@^P6O;Q|9N1pj4NUp4J3N zUZ=1JURqXt%3J z+^c;7#P*hhL>@pZZDdeP3?w47Y6Z0N!J>A%7ILqNW#0Oa*(*9JsvG`XTuz^fB}E06 zphX3r^dA5IRllmSO!StP*B8x%c^@>NB-On`EwG`^2) zNhT4o$cHV;GH(7ikZ{sDvdtIF)>?+8c11Lf-A{{BT2ybgVt z3!9LPfAC-&OV!Xql9zf9wqsU8h-68-@A+K(*sCPp?Vq0N!$k04907n1F7miP9_J;> zOkO40U6ETw|Gq9yn@Wu-SMl)pyF@%rac{8lp%adlp(_U3TyA>WBGEpji*&Eut+ZmG)eVR z2v}WKMOrag`xE{}GN1CH;E-MQMbLTc{VWz=0*l}6jZdjzPGU{diQvEv31jgj7y$D1 zK@KZdK@a8r{gst^2viAS!tg_)OcDq7+#iWbuhr&fEO~VG~F zVF4QjSacYJ2}7s{@4e}{`l4izOS63vQwqR4{}MtFm+Ng-5EY3&wutt6`ywOaQmJJ8 zs+D@*|NPvkadzr+!2wsOm32h%cki;s)T@mpdUzm!byX>T^<0ra<~+TrOYF+0^LK4J ztV-yVrIhJPKsa*rnTE67+nzU}}JjzG;;Vu_Xm#Qv` z{y(0^Gwlq7x~f%hy8hKF{VubA99|IVb^ofZQE^S)`rb{Fm`BF|U3=p{^+T3{;wc?GjTFQr#)xLJ!Ry^N`+`Pk|61!DJ)!B=8I=MQ2 z&^c4qh=G7^FA{x15S8i5tG6QE$4l7|3hbAFcjlcEnr7rsKU^wg{i9q>SqDkdCpudL*j&k|oQqHhyY?2#W-(&lU9ra(nzGto@`go{I-%OAVshf>Wl{*`R z#rySYH!aBpf>a-HyDjG2UrYY=vzgj$Khw}j62yf~+OEl~kDDh30cukDBpxc{kg$q? zbiiO71c_So0Q;@<*f0vfhFv3xKQRHCnCi-4jN&h^&#^f+?v)W;#3ygFzh?jaNi<}a z-fWq{`XS)7uxhkRuOr&~>u2ZVa_>^Yh+q`npk#Nqa@(v)anFre5sZlUU*E=gzw=VB z!*)o=ax+@HboHI$?ql}LzPJBFM?)m}mTigO3-$yl<{*+JSiaxr)7yqYL!VSWj8P^Q zD*hlapE6s*a6joN`sKDk{%pxJH&>|mLnMc~VVzQ??8{r3ormsjr<5flPXFGr?&EZZ z^NC=!X*rXUENvv`W61a4-n*&3{mp7K9hky>N`8o-Q0W^10+jJrw0RkVRO)*+?w`6O zk`AOVR=+R#x4->JYL8?EE7O1fl;7{V>j$=ILYfaS5C|IU%O^p&UDfzi_KGFus+y`Jf%-%nCw-2q`i;9;ag^t*{@OtuJVsd zz61~2p{8Z*aFivFfd8|%vh4Ws%qCC92%w2^wkb6Clu0)7V)Wv&>*+x1k^&d5C~8}q zBM|t_UzI)|>B1_jimr3z;asNn{YfJ@4VEpg*~Q2rTsm5A?#qp8oQIp*)peSzWh}p6 z{{R3700{sGA^-$C00000A^-q{fK@y*8p4#`!HvA1+G;7%@$~MGNxVO30@GVB8^D%T zDm(oNbFv(}GWCZO;_HT{xiP_NK)2?x@sRPUG3hqI%m~|(-PMB;v(oH(OxXj^{dLM*QOiYE? z9=S|+8rO7{AIeRwX5~NJAWvg*^7K^2*uBI@c5xRMMC<8)Fl{D;iT+ielUUoWAj8wr zV!z4MBCMHgZMgNnlHOZM*Zn6Ya}lrRLuvPKQa(SHwx?R^-!z+5wEw?X4Ocw$^EgcEO&Qi=2^U#yeiYxyRq>4-_9q1dtUFuAFcrXq?fLsWE z5@u~4RO#g-`oC353-1+Me2ohRM4x}-bV3YrW z?+?`Py?eg&9V#SW>Xw?(g89x|tIXzh5U1G!B@HNM{kPdaZ!KEeIMp{Ljb*lMn8!_# zyK1HUgZorZXXmvSukeI^KZ>+}**$~RRgArN=QpFJe!o}XGx^5*H!s=R;`q|Pu}9wK zG5UCoZ_B3pRmf z06jFBF0!g*=~~g@41I5c5SK4vp)FMMMSkuQNw(C#{F@fWd#4(1v&Qa3x{+f^;J`Qn z0*imX^{OcU!`8h$;!*wd1Jp~OpoRzk_^Nl=`@XWKXn}~Pb83&?$a>b4My26m>Fd+; zzRTO~wU6-kl%r9*BY3)4Y)lY7*tQAmvfToMrRS_U{g!rR-A;by z*=ag|{-H{osw%SWO03WZmzU5y0m|)`s;cH|3EYh}>Q}EU;w9S}r_BG0|7@B2ZPmY# zsALq4<>-or%yR6kgJ}mmrzq$65JYL7J(_N5oV zP%7nBB#ON~C71HeS$#4G?6kVZF!>+!(@3gIb7FdEemw(%T6(EelTl9R48G7c zV-;V&FE=ITlc)M@$=k_{wO5k6?T`I?Jr!s9c3oLxwU#`(ewexGm%ZH47cN|Sd|wx+ zFv^^ssIWn?Bz8_+G2M?({R30emuK%yNv+h!FqhH5-W$x?w9MlPL8=Mxe*DK-ww9fn%$~X4pfHcOJHA0_#iMslC)>$0P}Rmg zo2NjB?)OSFVeI$*t{hLi{#=O2pmMl-rA4@MQGWMYi{poo#sq)p&|MnN<5qwc| z$@=ST%M(alXpivmV)@+&SBK9*kID7C7`I#g>lR%>JtZ>mk_tTkTiMPD{t4R_= zeehf532wL9Zx~Za`FS$Q6Y3jP*QeQX#Dw@|El;U4P*YjoPx+Rvm`}1+S=feD@6Aw` zjX`^6^8B(s76&PocA&31pH*?>5}R56owAoa0R6$m?En8H?*Rav2(-Pt;CiYDsq2 zj#t_*PdE2YRwbHq?rL}ybpx9VXm*h_mZLekPt(B?JMRPsW#j)J1H%``E0OXI zHg)GwVbX#bu##+C%Dh{+lL}pw4yPV|qBN7_y}!DU0=Xv?{RfI2FZ~<)k>iEjqQ>;K zg$JD0(K7tjhLGYfczI}_59up7-tP9&fsk2<w-(OHQ_VK*Aogji_ zN~X0FpJ&(0<5x{{aTFL3m(NefA@VgMwEMfMvLQPyBw1Q3&82a}sP$HuGDg4qx58hw zvh3k3qBrW-@AV*Qavca)8Asx?sp?x^%&o0*Cu!Z^rSiBn z)@pXqy>m(6V0|lKdTDJGs<$>SBnfz$sNleC?`GfZa}c)COz*< z)4mMkEomT85ENn=w2#e86gjD;z6pcM9u*M!(t=QHU{uHCOm+ngd!1r>|3brnH& z9A8>3g%JKVHMt9Rij@Bon^&M-fd?hjaR%-zVDo8)WiN%+=9!J`O-!?{K#DoNl9^It Yw)D^pdlItgQttW9tIRk5@Bjb>3xH@d?f?J) diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testJPEG_EXIF.jpg b/solr/contrib/morphlines-core/src/test-files/test-documents/testJPEG_EXIF.jpg deleted file mode 100644 index 1b93e771832ca9bb9bcaa5d1f8ec8767a55ccc3c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16357 zcmeHNc|4Tg_kZk5$`WlTMGdmdV2Z&oV;$Kk+Kurr!ub&fp-(F-w|b`DutQ%;L6H#m4jKbx>K1 zN52Y{C0N{c#h^B%cx5@LEX~8TKp1%z*JBVFNM#8FBG4V`BEaG+14nswBH(bLqg2?5sfkL*PS3mkr6u?2{upiPuHgy00H1{-2i3~;vN?)Hs*Tb`;T8}^t)(^*r=p*#t z`f7lgMR*9F5Ck&Re8B(`#T+(NQwvig`I*DKkWO%?5G#;KvfoStJvKXg5;g}BO#EOL zXlb)>RCsVmFv!5Gg$I)LJd2r`G^g7E<$3dLRnJgHQ&+HySu z1Psza^Ycf!Tifz9Ks9q1&(^T8FugECJt{3gA7Nr*q7OIFH!#qJ5W4gT3IiXmOQEYn z8l=@YI;=rDfkp~pkf;`($e7^LBKY4|W*I>=y60)7lw%;8j zP$54~jE?;;j0q7Yi17<#(iy=Zg@M_`#E*9!>f=ugfg-G8WCoKccnaa?5a7Av zckDli^T&w*@hQe#`$u`{Ob4`yPxV{-9)PED12O5V2mHYbz8DiA z&mU<5*i+-}_kds8_Xws^8AND_9gm#}*-6g0kSm@B?TZW=iC~SV1x@UqKa+ojPpnfO z(1y=IFGq5Nds6Txz#lXSXmvuEJ`w1XnmOyf03jaz*Or_}ehexF(5aN)oIYV$Qn8blE~=s$uO15r;3fW z@e2T*@bnFA})Cy$Mcu^jD4m zp@~(7Z|ZU8IalWt_MgNjy)U5s2`zjm0m`5rLaU5mLne8@735Ps?y^Z2aOL?R<7%LO zf>umgD98>q?xEWQ+{CGHYh)4{uA>HNWFIM^3L7@TF+d`9pu;P)B=T{&6`mIE;7w_T zbNHK}I76Ec+L1&7@wA_?9L~6%t3Vo^eG*6N@z&pQ!UTZi3q3qej{mCh`XnHL0zH|7 zm|!Y%@}X)KX$tipot^y%6G!I|CXLKVa`+MSK{Dq_kbPW6@KON5P3fVP{5A&{PO8hlT6 z0^vEHc+@j8Sqe8ou@A7n5n_)hhh@u7D{(ShA$S@c^o$4r&0+2!oyw#UKu){kKoSjV zjCacBigGBBrf~$5!acxG0gV$UaZXe}5N*ISjZ>NAK|7C41jr%sYvCjFW1*7`lp*73 zatAq8h|eQ%x&}qYKNuS_P4LKYPR}$dl=NgLJh{U|rlE)==(t5A5TR*B_V|AiroT1o zAGl>=kr3>i2{xa1CTJ*o5}iSXat>ztztY%)H%WB_1Mq|hoU5l5FO7vjz?Wl}StG2B z43{s%U=3}pk;YcYWhU0k4Urh6HI$P4gYZ8J1P00fH6io%i@bXSn(fe&UU z2RISS-lcxo-XMedcCU%m=|3WKMJM(;!R?xH$n1nN4N6e5cQHu!FEqB-IYdrU9ZD7l zFo@6tmI*XRe<4rBdO?R9BHGZ{cw8+<1D861bY*WT+;bj|!T@QIEevjEb-aP!BPNwV z`wE^0W%m3@0pp2TC^OEn&s6|#LXbNM?BQ@Fg@a_YAy4RM`acQ}=ywajQ(^o!5$+NJ z!BAFEsv3%a_Ky~5%ul}vat@K~WyMVYXDGWrY?>3~J3l-F&)x^POFsgx3pemIFhC)Z zD1(W0{}=4I0(Z#u(CHj{mSID9wSU@qC#OGmKRB4TNJ4EoIiYYAI)nYhgtJC7nIu1y z)iUelR@T-SLnIbqZG%8y4Xlie0C`Ek8t@Ylk_Uw4g+%0q#u@=t z=q)i}5y&#=l`HNgCN3!@EFvK)1VC(8ImmbsVNqdGNlDS!YykAWoT!*QATA-Puy7_q zN)h9xv`F0mzbUnJ))H*YvDUUbD`{uyH4KfctnJ->{Ta;IZLGa%**Rt9mm2Ou8lkr( zg~a&VgWh3;bV`Uxib7lqd8h*UM@mW(vWnZDJRl;vP)q?Kj#=d9j!$Kkwzf&kR8*RU z_1Hu+FhW}S#+1EPKV}$vsduakkb$IxoY;<2Pe_7>R&#M*B-=AQ*;P02pM4T<7jwXeQU&ntx%Se9Nd-!x?LRE6ylh`vC z_m=o2PbYuVl0CcXo{7cCM}074ct}4d+*g8Js?nXRD*XOWnd`BeSr=i`o#So4v2!bvq*?;}TtT?P(IXl+q%nD#)&>0&+G0wtH6+7vgD zUXM2!BmO^YkF6N6Is9 zjsd8fZVv_u?x?0WR0qDAKZg}stB8JrlHI8@d*8|7ElA>7mfa6(S=w(N3w$~+>wR?Z;EZ-_#_6@x+1EOWq$kBC z1qJDZ_&M@9B-KPAMVY62qk&H^_ZKR(c$U=#e%mG`TI5^rs**W_ep$G>{0Xx!J@a8! ze8uhPz!9-4cV2%iyS@r^0A9bB`(TT;She{e_Vec|_v6o;Bi5feYrGw!xKuMuw#&yM zsbm4>+dAcU)9qyLqa0CbOXV|`IJ-vWWLWy@bsY&Soz;2`CQ;pVzoSM>{BED_;I}2u zc3#YzcJono;||PRg;|z)uzppo`!TzHj+r}3HBo1Dncz*dZ#QESE#OAa1O6h;zZ~jL z+xJ3qdu2}5!b~hPk~Pr%PUBP3^d-aPi-&?XY7QzU_vK|L?(kF*_T1yFTxD#mZvMe- zQQpBk?*m#Goav?L`ybjB%VnyYwOqR~vF`>Pt_*nZ9rSJ*Wkw(@gVmot(9702b60`T zb)!h)!I3V@MCMwz`Kv3=Td7~N-1J;!{ur?T)gw8fN8(;@c3wQuw$9^rnP1Ai97p5X z;TzAj_PiWUJa84=Y&g=ds}L$zO|N>m==O;=i>(Eu4Kwq-qL(l2V|*yH&syvESJJA2 z)5~5~k2-B^e6~FP{mdgL(+u^&phNcVekQS{lz0i zLmREu%e;H=a!~-w-%CbmwYZ1^9KJ$0s{FzFDBE?_ol&nml+v{wV$3XKrGA`!sA2}a zv-gBGTh}CQ7Hj6Ug_bdS!^D)D{CKbVCUk0}`K8MTO_Q>3e*DsKAlTE0+JEK@ za+;FqHe$i!Z8b85N$35;Z2RZn8o{gi*u|=V@Yk-V=7Sq|4$d*O%XUl9-h#bnw7SR1 zI3|eL9qRk6H*VMW<&T-mrx$<9t&P7+?@>5-BuK1ld(a}Ays&q^M@r`zqtsakL<@Sa zn!CVUU6Jn`8nySrHFbJi*Je6xTzh8AFp)G{Ctnk`X~E0E;29@nRHENL|52ldTxVJM z5b4^mbx*5@vh^3Z$5ESifyMqd1@SW5{Zl=!&LX1Vk5v4bozFJR!;&)GFT!a6NW*?za9 z$#HM^1$WC+GD+UvR#-53Hc9JIh2q--8yhH~UPs@ZK;iRW8xSE*pJQZ?iVe;i1B|~0 zkS~h7e2&`_JMFaAtPqu5-iK9}KaRUB{Y`o2zRVxVn`_RBshqT4 zuGekt-O|yKi{_t4wy5=6QWIxA6dFXHwwtfFTr=B$w6c%6%wkv1g5!>s@<#hm^+}&b zFnd+!&80uT0Y~4aWLsHX=u#X4?v^q|U^ke7+2W~0nD@C2jxj2^A)QK1Pn;?%Yrd)? z_xf5SI)6>?BKS90Oi%p)`769uvIn|3l&`?a2i zB6JAhICO19njPUyLJgON7*37@_M29+qaw`XXnovNHOXaq-fczU$foz+)l?oltF ztC9CEzUV6Bzi<6~RpFN@S>VBAy@hMf=-73`~(+-6%{TzN~ zMRm|i;}{X#<_gNSYislFg%>R>TKUCDY1x;YjlzCtWtb&SasN}92=`LldsrFZw)V$K zX;qxBq~3s4LX}0O4dV-`d4Klq9W%s_`G^Ti>ISX8w9t_$E1L7b^mzPh`hzgleBy)D zb$W9Oj#j=YG9_lLk=}W2WunGRvtgv+1Q>xBK>mPq(eElmTj_z&pvFK2hOpdcj2*3=&jRT z{@zO|I&ZYgZ5R7&t{Yz3V?1{O<@f=Oic+(p0O6J;)w&_^-kPJD=85Rd)`>ga@u6Q< z$$MY6>D}$K58s_a%_!2asgpC>mHD)B?=_d{xH;c(6`#fcJ14)+lqWMUo(QaeW?^GUNYsXMvBtftc11rRc2XLr<5h<5UCBg0>o%2s zSO0Cs(2)i&^ELPGdbjtL-*od(ysdM}**E?6&eaPH&Yx^)jZOV%J-Vf9`o5fo(wSOH zv&wKP3gkEB`o@Zwy{m?m2cRK2GQ>8F-yKsLE^p}BKNy34>#XhHu&g}eZ}A- z?Ym97Z8rle@>-8o+WH&~+@FWIGI}yWQFc+o3dIzeVg)0IYfF;SUDSF>*5xm6H6CjG zapI`3?ZAzv(YG#FE$OR#y}cPtg8akEBA3bReKA%T#Xh^IGec4qrvVt8<*GJ3Nl$) zMXzbtL7lDotfHW>NI%s6l(6%!wCF$Z0!=JL+8N8 zmVW<$VF&kFXI@AhjLW&SpXF4aA)TI1VU-=*pZLWxTX|)9;kuk{T11C7mcsjvxvhjt zGbC}3SS#f5w>@^4QgSYBE-Aa4P*}`5++V0kNZoc{v#8XGn!h?H!2O7ViI0^ZOTw#o zvGYt7vAlLsx}NFxx}zy1^D8CNZD?{#Gj-^Vl-JE>56|ak?$jOj@qz2uBEA?K zfB%BYz~?M;zkfuD8n)x>p_MpSjPKf`aMkxAvSiug_mjI$7{Yh0#{S4^l@8n4u{C@VuV;IN|jTEo5$#igpC3EW@b;$Rz)F#Vvkf9y*gfP zWu2q}$T0SN@xsoKhO0}ciwc{3PM(WPyuJ`86q#1H=iaJMEF`Iyz)ji?-xql{iiz$Mx}K?!xzEhy)VH{Art#-jW{d&hkCcZ=U9V1L^jAn# zQdw8qXNO9rwPD-Auh~gYY;sxl^~ZBq>7M$1`UK3>z?mdHrY3e zo^)=xqQ0w7GpUhv{76#+{4?Q1X{wVM%hloVj_o(iG=ysswDr>zzTNJMWOct)dbUw9 z%Vk^IwN}4HSLzH?Z_NQ0)qm?Um2O|B87xIO8W>)X246!Ybr|0@WG)kqc}{aVKZ5!Y zAgZwD#Ta0E{1%`9!0g6=%^&IlzdlHl*EKU#+qt-L(}!rqNK@Hb`_cqZ%uIiMe~CgV zt~f2VlU1$$;IMUZ_zpm-2HW0wetX86YWquHa%5{r+K7hQ@Ao1lZI#)&s`)Q$q^MVIUS78L zUz>E3%q?xcOJu)E&FVjR@$1t2xl3EpfRPlR=r=_U=02?BgURjI>nhh_ixP7sU{!Ah zTNinrd?I$$D#UTY+4h!!k7-$m7d zeX5I0Ud+|4Tz@z!Ww&WyV|uKP@?FC#i!FMP_`;4D0+jh+pc$ly@$nbq!c~D8l zCTYfVHG?ak=4nP%4sWz1IKJO{hM4VD+^>{2vkqp5%I%4`v%Bh%sL{~;KuVa?A-&9b zk&731NEV*o{HTvjSJXy2E;Ff1;YRH1dSIH`I&w633 zv}AL;<`VgJ2%A-fZQEj}na*xiJ|uC2>8r3pLF)SX<4?|@Pbh9rM_9`Z^zKR@R=6K# zy!Z^_Za>xSytf_592R$s`xyG>WCNmSPPpZwuDasw-FjA6%|A<2WUqP$k3~i=H;uje z$iU%5;~q)Xdsnk8Zhi_Bi`?zKxN6h1k)XJAgp|;&O{oQYFX~*gIp#aG=7oA8&9Jl2 zbqOrZrk{qUH#bd7sl|7V0j#DI=W2(B|H=*Be-Hk+Gb;6~N9UqybK|z0sOMb`a$DsJ zPUn9QEVnUTmVI#X7Q54yp?8lhJ+4Cq^Y^}OBp=w&Z;t+s^wiPgvTh^E{t#>U8FWQs2ZF72*O^9=i* zx8RhsXWWdm8)A@X=cjy9)1*I&m;`NXL?=L18_`Zx4B|J~^YKD{^Ig zk5a~UhqMoG^VAay>r<#5E(c(>mDWaOPu=e1Ip##CbqeK=MD)SUz0d9J-|oD>ELZz! z)|&)58Q}DpzfPSpe^7sI?{N9y4NG8lJ7v;$r3#hIwvRos2QdIMKQkonA6>b0r@=>I zaa&;Iu+0;nHM1%E6#aecWCQ4XkTY`}CE&G;syuJosM~EwVc(Y2bNczI{?nTir1vi{ zk0Fh&m|w0v2GlySk2@ z!?`n@BAga<^_10Yjf2u(rTI!98-`bQn7?hB-jH18)~?#R=jm`dQdD2B#wg&n@@Col zcfQuwJ}EvC+52OC!gocc9V1|Y=An}fkMuS!izR%^lo9!IZb!*U)J-q%k$PvWMS=9Ik^A;vo6W@(y@0qu9@bv1n1`Fj56O&IL79XJS7@9bK=I_>#}4Hc0K zBE8itZr<71-v*V3}!03I>y?&9z$Zqej<-uD)GMxifkz>(1+@=WJsx@Qm?^ z(0w+yy#2cKRg>XcU3o)efOzVu5{rW+r9<2M2P_^RFETwkeZle9=|zFllvG`$D}H!p zB&O>rdAylsw58?? zlH>6mCjFX)9jM|%HF`~b-QFH6q%wRoMFTHCT@$;={$AsDz)hY7qdJ#-E`s{G)$-V9 zb~guVTmm~k*>>l2t@Z6uQmyEj9fz@Vk+xGcQu}V7>MG`GQ(qubR;K|L9bT0b8@nrK$6?<@%f(T{^8*{=ghqC@Au9-Gw;ltnb2~zg@G>B1whj#?8=>%i@(Aj=@6K&GxA%PEx!vdX zQ_q!w%)axb=Y9f$y+E`FYPmyxnpB|5fgp*bKjZ_7_k#jRWXKDoW`esTNOu75IpFR_ z@`Qj(&{A-Bx1=k;-H${ur-7dk6X3(Zdjq&xARP?eC&1l_jAO9-+afX)0^T>lt(xUw zmOX>;ON3Pre^Qf%%BV36G1g#ajP)&0=&}*2z5PV8XsQ&JtF%}iVlf!;Tqc9dU{P7o z;K_}Nhp?7NFkhAzxFCoMEiJ-pQ2<0ZUSw^)bJEa`dXhqqbAV6m8#d3f!L z0p7!s{tl!`^KlFbt8tm1nY#t+!ZO_cZVYdc9|CDli~Iygdzay5d4k*5lJ1E0i3M{G zU;@kV7Q?RjWn_o~-k^$586gd=viQklFi5L#J1s512TPg}06`Q>n(7BZ9V&49SAqQi zOZqlQcdW4YEUA)WE0;`2Q!4E}>qU_J5YoPcv>zc2S$K&c+(Iq&HuoQc!(}q5SR$Sq z!*(j;$sp*0Hw4A61Z7d}wic#<2mv8`#FAgt=mANdh#$NA1N~2NFrPH;B zC>V?CD7j8UFM#!QW)y=?g$5@S=wW#tVx(pxxhgG>*0Ap&jjB@cXd~EChEy*?l&aLp zC?cDjo+Y20Cy!Uq68L_D3%CUuy#_JD)B=rKYv2~}XlCnNkjBJx8r2Lj=J99}4V;T` zx!4?#rPHaYaZxNL4cI^xIowQfvc&?(@n{xb^Ying@}r}4XfB-@A0JO=u;?sSBtS$O zCTWduL8R6Y4lMXl%{Ih{L5`~QMim&)SRT&SnT$Ld&5vp^4VwyVrX89A0WO)_0U+qW zAeA~8q`1i{RFe;*2%i}R{sz+>WHyi*HYi0pjULe&V61FDgKovPQCUf8K*CQ-j+Ehj ztkF<&=rNJ(s935*hI7pbB3z=)(ec@AI+F?r;K`sfu-lF{1FQ@gf~sJ3x=E9bp!~RK z2Aj=du{k){K?mB7-m3BbeuK(UgYyQJ!x{vIBT+a%(tsF^Ds8U8+`f3*0vcDKMiE#s zNubxORfvM0g_@ATbSDZcUZ^%9sV0Qaz){vLQ7wawCe+deb`(Ks1EZ>3rBRC*3^t%$ zVg-rJRmebfIVv^6PXdjlP#_8%?x=Ih5UX+#gOSgUiDfec(J@RRTgVoP1WBBj7?zkJ zjENS+vxK4~C-y8%n^h&M(*XbF0?=J}MTAzjo$A~WGu;p~?L;+>ff*dRi6dSWfD zA_7Ea86JcYK$O~$z*RNoZYCnvftIrMPFsbIVnPIp@g{>&gJ_L{2`1P^X`^sT!3K|0 zido^5Vr~kj99-j+Bn7Jin9QBhtU9N}p(soU3uh{)v++p)EGjEaqLrfvCWqBo2v{vpESPK_y3jf!gYQyjEKF>LrQibjS$;I2$8Jq*1#n}*NvQ-~TVk3E zQkpA*RHZQLvs%7uy3ht9BE89VfXDn85 zPaH~dEH@oYhx{~^7J<=LShL28AC8~~%x`v-CFHCE8$gwf68qnlw*zvu;2RtfXmlp~ z2xujJuf4NEZreKRO{m&@$f1za5w-cN4cjg=E!$;vL8cerCEkx9|#^G)W)JBI+!JOfOfY0U83L+Iq4s22zY5c!&q~xR%Ew;^G*vrMd zFqpE(BXXnN4I!NC7#qYBFXz!iJOXo$z|X{-IA$r$?3mq^_vkz9yy(B2B;j3$Ww~P5 zh-9fW*v4LER80mGHFn4t$>2mXnOW9D#;BP10gM9qtnxsc~ zv`oaHGof;XhSwVhs!+hQ>^}(-0>Y%lcj*owtI&><>J$i{Wzm~6b{$xGk|rT$A<+s! zOvHjY#XRK@v*y0BSMB}!MvXe}8S|Wq>OjigJ!+JzRDlf@Dp!JGrI!7VgsDrn-lJJ; zlZ5luv8f5IW0P@HRR*IDox~qZZ!2>|rXjhoe3B$1OJq65Vlo+V!dNj=6cZg6D-cE} zi`g6zJ2qY%7tI#1#T=r^HstrJu~C)dVvu4QLOro8SEw=?G7wY-Yb+mPE|xcr^G^L+ z;#O;&^Kx-_uv23cF_;%Ejs;4T*?{Il^)y7AYgB>-HlAg{62*?#5n%VC9jejTK~32M2(0MJNJ;SjT>};J5xoFwHux45qh+TIo{IG78uTV^cal z^)nff3|1D4#bvX(EZe-_25U8t31f$Wd^QfT>i7iEf*v83(SBzzS8 ze_tO8`B2D*LOvAop^y)Sd?@5YAs-5%`q}>v2+=Bdw0v3uzvTubg2*K7!k%R8@}OW( zPY(}@hnJ_9m-+Vg^#QlHmzPgFA74N00xAD?e*RbnGlEmeo)n6wpSPE{-~ah)IRpjz zKw?NiAq7I@KoTX8)N&N+0YPLk1^bUj>*eX?<4dM^dypWY&gcl7r;t6!9zH%ET_F7b z!U>>Q)1=*zP5 z1r?Rm+YZ-U0v1C_5XqBR9^{4D^!D`e0NR8=kOA%E>+1u&!pjSUC>}jMgP2}|Uc)ls z`DI(rU-0e}9MV}Rn}D)n*rM#h>ZjqWql*sTX}J#h0~R@u5(p(gkH33ks5@T&>FY*m z&D@VZRfi0}*7S~N*zvF0-qo8*2c9?-ywE#6Tz0cl*_1jc==N21Yd99z5G*8VXOW~mwXV5jqStBouN}s@ z5AQ7Ba&*@2xijt-Z9h=HSuwZ0`fWtOj^S706JEWfBl_nJ^vZ&4Z}ryRH`ny=YZUa? zNpiy~8wO~0HuqTszdEz7NfvOQzJ}jbr*7m6xgmE;(Kjgtb#uPm+1T(`x4MSK(#M-W zJ~b}&Y0dKLWAQ!@N6d~5NUiPeGp>9~e!}5~`^{H|K3?M;F($mG{gXws6Dh*|!{)!3 z)3|;_Rda_|&%!EOp#2x8LR;?P0n{&YQc(S_SxfBYiN^YZU0XH4zcAt^Iw<(%KfDlJ(#a(%bhYkB7eJmM8W zP5T2zt}3tZ@T=IkWwfr#sjEuW-Hn^8sus#;b`7jl^(ZC<``;^{4mCaec74#ftm;GK z-+t-qu_60NM(DB*hGXPC+wPj`7B2g3<;c{hvchHh(O5`&swK_9_0?xM|!40|FQIP0yHG zxhOF^>iUZOt)0)GqIvH*e(mx;Pp?aLk@au;-k*J7ZM!qKDvo|F_$a7z;#%6H9ue0H z=lpXuFU|M3uEX_sIKKbclY-&|M$CiU&y``vCT60GfA9CzuFBm#mkCW%${ycr?A=t- zzVGvGeH!w{^{Wq_Ter5N`0K1ta@M@`?z=ghaNdhyz1A*WJ90^cKr-m?^lLAE4c_Lz z=VV02O+iuP;~_sk9$8*L@_4gp5;IW~e(y$9#lY>Cg5=kK+2DO+#r4Eu)97Jgqjv5W zg&$6w@E|m-1^V`nTOCQayhc2kePGjt&tzw-6{S}y(>PrU#_c^{`|x@3k`w%s(XSpw z22Jd^$FTdiUS~I5Ncf^kHMY~b5!2%a)EQq?r>-2W_`GCz)t1Rv7{7aiJcJJHewW4%2uc)W@ny(($Et~B5u;FG99XfTVVr1;f8&jZy>iQP zM)-$}@}dMW82=zo-F9Ql)a1|hT%G!dEM#H-A%%kzi+tbh_$_pBqyf2G)+I82LFckg zr+Ow9u6?d7-M4P$h_HBr?kMl@v897bD$YuFKkE7N-!)4#Suwgt+qbjZg$(*qS#|r% zeg5l9_UGg$Kk6zuikw&{?9&4xzqx*oS3hoceb?xeiea<*PZM5=8C5Ig6y_;!PRzc4 zXT~>w#oac=wcpsZ=HSdrhT5Q|EAl+Af0frOX>ERE_KL0DINb2EB_36GPVk1(GBVhW zLyq<@XY?Cbn=yKsblm9e)1E6;T?Ve}N1O2R!+K4JAN)h7KYj3SUljYZ#Pz?iGitt= zcV5ljX6nGGyT8HnV)r{vy#iW6`EtZdVg#;p=wd)0S4MoyQr3TxsI8{ zZ=X~=t7=i_m6yo7R7V2dQ3OG!(Pzi^H~s6>`Z}6N!L0p%D*HXo8Zl*Isp;2Cmyf5F z7yOi&_@jTx$dMu;GPjE=;?~aKr%T4wXpyMPbr;8z_rIxO>ZPv>16Fy~cWZ$-Z*tWK zC=VY<<`uQu646;7`pwAYJ>qW9IOg}Z`|QQb-p!r7Z--~-4=HHbH&<4#m4zr%E9MRD zP7axWL3so65+3WFx@J_*(#hBS3+gTnNxRrQN*#RT@Wja7SBp3`bA1D+KUr~>mb>ws zq6Iq6JSE8Wj9kLqL%Ja7*|6YgVaY4rbIo%7X`ir>FF*Ns8ISXhOCRDh{nrS_PsW>! zUnO^jjlQ(C`PBj5>sQmLj}mpooqr?Mv$v##MaA{2$Z6hHXNpburuO6S(-H$?7ITl3 zG`$j(_vrSK;lVEq{#k8>Nc7Y7;0EZ@RuhHxi)nm?*L)>y2+T4EYU!`AsJ>=)N!g-(g6b4c1vhGC; zE7N91<4{f20i@nk9>(Oy+*0D>O1EtGFyOMt|U3c>Jx|Q26 zEM#m+&74t^DtWMtpF1T6m6przZaz{MlG0iI7Jaw7?CsPI<&#GacoOoT0DTUE@f25T zs3okZf)R0wb&={RT_+#uN@DJ9Jz1{ zoOmnvc(hBoYv?$E14(+eZpzAArq_6?A*xmRG!-`aq+bGLJ{u=MNfemm40%$qN%AMM z08v~0tqL7UVhAqmLx;FOw?%aMX~~$PjEN2gppttR+J7|_#3W~~Ah$Is{^EETQRBclvbpAAQQ-+8%Dygw~BuE(Q@~EhJ zlFsDaXgk1eEVm)9T(o80Sj?4R5|os(jL_l#07CC)1RD@{y`NYKHD$v73fu4Wl6>S| zx^tY5G2CbG8uTtpJ4c($((LdZczd~O=6I?vvqp`aq)E_(4>RTTeHvvLNE*H1MRv}x z+GbcQZY=F8vqmLgdcgB5-#8f`Zljqbwy`{_X9CbBxh-&pSSbD|J%~PfM2Lz~VRrde z5m?1V;f!FHVcd^zmZ~I0zz{Txo8-ik%1;eR82m}L_WuA!O=Q@Q;uF5Lc9_}lzr!Sb z3H~A=52j|H)_J$KcyB-?k7iN0CS{7G5m8l)F)B)*c0kTfi}qcH9=AU6><|yN;qfF{ z_yhSWB8Kyai`&~`-1da^7Rlgh8RL4TmKsT^d{k(IhA2A6Ooa&!Ln&n>hC1wGm~B2Q zK&oYyc4Mm$K?lD=K_SF&aC}~~1;mq9i-6#b91uYd*lxrPVI1?<#+yFn+&c>K!ut;I zp2Jr?TG!NA)#J0xQBxGpEU!*tmCuSNVe|YD%8g<(#>kN zrqQ9dMqr?7n5Hqt{;h`;4#)!^4QScIeu>?~5^QTYUu9`i;MHx5qRy z!xXTrtsONgK{Xr31xv;t`AH2#2%z{LtymY5+PC) zlDS|I9Azv42TcxOLFc%AbdN6?xsu^#!r1$!`DNH#MW@`mDp^ktR^1JI4aaDw6x9@z z_o?gJDQXrviRvY2=N=?7OE0J}lm|1Q!^Kr%aPWe2GXZ5gfCmJHD3v9PGF8DshEb9s zh}Ei+omJ{mLkCeT!zxKp0D_#kRjRHAWj>d+i!co&Xp*%(ES)i7{6fDbbxPRmU(o7iXY)!@7PRLHnuDal zn37GxmI{@XZj!R0n27m6q8nASsa&LlDQHU4k`nUP8PQ|G4hTu!l1mX`&r^vzZlu9_ zuCmmJsjHHm>ZvGWk%ZYOs%r_ImrU`CB& z9$KoVn@>M1#E@?GA3ZXpUF-p7yXgWq7UQIFZfn#ffOmO5hPHs+BG~{J@}( zX+6fS3erAofyr<0?Fm=S$x~G3w9~+yU|=vTMlty~IQ;oKmSLneQrVsrZ+D6MoK;Cs zL1J5=Fdg!64{Q&k*^b(*Gcb!;Lj9l73U`XyW^D9e=RebcBkVeXAVZ`~CN9g8v(?Hc z4BI)*$F>}UpX_ATQ;0XG&Tdk>Ol+8E;j%u2{{RsX3)3eUpPUPP3bQ+Y15z$*_zMU29((+I$;Bbz=B*1Rf%L~L&Vt4~jn!a%rX|2?fDTQH*NZ9v|XwmbZKQZ@c zsRVBj9WXtCEdzJ|053c|;8XJwlwLZxkE&aBmfL_koi|a4hk?M2O|UmIrxql zNr=lKNdbG5z>wD(i;ixFH;%RyjaOlF0ZPf2y$(PldvO~Jjw`sa=iBd;9sMmN8+&L$ zx2mFrNqenro|MkS1w~U7h+_aQPf~zJNbWI!d7Cqi($5A*n_5o%lGD8F!E?u{>D208 z)a>9ujdjva8|CMZIE-Sa5uLiB)k7O2I-e zOsa**2!YrrYz1tmYiR1Q{9Tqtb`W1wny zsFW~TXld3s)H8K90)ggoPrUa!^k)PTTIc@&JEP735&$Ph{{VXQz-s#2lo2gFGa!m3 z#Bub16ni@lzeur%Wd%GhUFxn9u}Q0xk=T;CBio%st9Xe33A;ko*t~_7TDeHU`J@lK zP$b2G0$Jf^?@0(Jtz^jVOFx^)sdjwa%rlz7QesL|0JwQeL2cqqfpX!>v`YYL`E6|hx_m5zJ?W#&W zV3MQmj@3k5w_HN1h@}T7xb5!IB2=y71;|5iOdqw?yQnSDr{2fsYRv6utjP)v@NS;E zpa9_+b|h z(#li--G^IiGxCG8Yp#}Y3UuZmGRF2AD15i`UqE9_@_)2wqF1Zx6_gXeCr3J^-+9xF zxZVUb?;dJycTJown;Ls1CDsV3yn6e#7LJfCfrnXTb0|GN8!7_J8^2jVNo+h}@=-n3 z3lsaFm;V5N+-;tlFv;MM;_h_){{X!Zx50bdd#iZ<6xgr#TAOvYDb`B)qmC%9GR{=7 zk#HZx>5d$O<>4oil#V?|$*vJxF-2CX(u9HwI~UO#oCnNxViHRd3Cp~r;d*>^Z4v@z7`Z&AEhM`d;M226wXcI~ahG2a+?KqA_}tA*SyL&!b&2ezVC z@i2llfHymW$lfwati+HA9oNX8Kh5@P1$&HGCAe`5_iKZdF$n?AagnH#zAVZ`pT*7F$;S(izhT<{0FuY7y1(!9ex-Jp*D&DNe3AN7*dn<-IMSqi z=Gwy$e-Nm!r~GQOMLRa4nk9Wlut0Xyp)0RT@<(Ziqie2wq9Mjl8q!0OKXdyhr?iHs~!weRyOA!h_!gkJqb0fe}FrtHEk3+`p7m zcl@RwuTE5h4wTFV-V`r<71&#b;=Smq;0xmC_MiF)zccW=8bPEfg7l;&PuwtF9YSRg;m z(~9cRq}6qgd?P_Zb(Ef{?u~EBzbW#3{#&ND+tb5_TU5ES>bw_ky zc`fA`v(?K^$T{=Q+NSUS00xn-u0NWtvG0ZI>_6b6symD?#pGIoB1qT7g#b|zk|%FYuK23!9{7Ji zq4-Jaj_B;WPiolqM%<*@TVmf89k$;WgF#163P&9@ApA&?Pz6*)!l(nXs2@(gVpOFP ZOOqc^2}(jhU@g%f|JgEmN)Y8g004!~w2%M* diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testJPEG_EXIF.jpg.tar.gz b/solr/contrib/morphlines-core/src/test-files/test-documents/testJPEG_EXIF.jpg.tar.gz deleted file mode 100644 index 3f35102eaefb0f9cdc648dace30c437b86ffa9de..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 8722 zcmV+tBJJHDiwFRbsclgJ1MFG}SQFRUo+Rv|2!dKgjG{(V$RrR5DG6i&jF%`9>QbxT zAsI*{o0CLPYb&%>YE=}ss&%8NxYR1{OWn}A)$3aKii&!zRS*~SYK8wyCd&kp-2RXM zeV+S2kHa%F=X~G$e&;*i+2)*JLXD|23LGsoaLPmpI~{j z*s$dd!?6hoaU6)nX2r(Fvg6`mHUzWcH~>b6{{z0Xy39rsg3;-aN^exC(0sQzM5$Hl zJlG*B1KZ#KPp;-G&9|WTgVLmF5QRd4o`657`7u2a_G510jGdkdBN9F*>e zpI9DCmmvzZ4$nhu7*6D|U>?k-vtz-N7nca(kw~^&))%-Shz>0&CTdX#Twi8QPX+0o zC4{$lgD|Y=IhKd7MFz9kEZQ8Rb|nDsWletv(v-ObhJx3)EWpCuj5lE!VSg8nx5^KK zw6|4$9Hf2Ah_bxF?Qc!D$J@k;ISVksWkkgAYi=18qJcN4B2-37Lo2O*vS669lCaa< z41BPr;UEa2S=00Y2x?bO*uMho2U*j%K)QXoqh%=-G<&&JQkqubXjvbU+>ey@C#3^O zX~@b;0o^U!T5n7H!CW4TMaL6~JTP~;lJVsbblw+&5?6q-=nh*8(m@XaUG}gwzp}v# zl7e@^+W<>%{cjq3y*^Z#Oa>m4sWZkPc(0C8=(Ws3#K2_5z)U)nlw4>)6#1x$o`dG8 zb$mws-UAG}TFGY&=g450L5!-@11DjqeA3`-#iV>iqLPs;2uLdA6>1Gy)P&FrwHlp~ zSIB2rtn)w`7c&`j3&fPqXGpb39?Ik4b3nFUuc0TzuvrXX15@VmvLtC%3n0g5SbZ%h zD2OSDjnQLyOjcrIA`@ma+3aY5h&E2tnUKP0oiP$v2%uYRNKm5!QyWZbFre`~lA|}9 z_zXq>-D(;!7urobg#iUFS>gZ?Okj{sPXZ}kni|sE_ z6UJ``+5)gK_G=t6B9265I#KIg7o6X@8U?&}j9kWI7{(3+an9+4W6^ItakZ24k zh&G}olUkQ&w8WQ)Euis)8Vp606NLtYMvW>3*_atkV!BY+@FI;F9cV@cFoCjV$!a-b zGGo>zaG*$1I~Y^vsZ2W5XtVTNk91%w>7N&CJ;@A>c6c;N@WQ)Y9F6>#EcB?8(uLb@qgrK>Iiim7(yVQ9g zW_ciHIfxnrQ=2B*vDU0zt=H(W!Fsh3%|exC1v%cXR1TVM0*aZk_30Qg(WpQ`;L;F{ z5w$T}X>CLanu~!BWN&(V-Udi5H0X_HFeKp^*ov`>Y?w5)Mw5ZJvCWD@V1uM%D5@iM zF?-$#(7Kbg6x-kf5u+H>8_2b^iAWH+b$F0Q09k5B0#`M-yIH704KKVtM_1_MgQ@D@L zKFg4Xo;$TCdh5aYZ%{6b5utYNh(qU==+T8%5StmGM#W}qg2gbgN0g)EEg2%GQ48P! zbePyuf{c@f&EX6Hn^%sLmdIGG5S}=d;#}@vFdYgq)H)QwT3{_28-5sy8F9ZkG1e|; z6W9Uj9E{xl_Phg-rvu;Ms8FjnJ4QeY={xP6l?waT*5mOS=0ksk_`x-~MCDfRl8U=TT2ReKnpHUdC zL~{|d#>5c(g(D>wjcB!P@xoCq?uF5uGY(al9Bzn^Jm>F_L1f|$^3wHDvlt4-=jV4}tk8KYrtG>esOJ7kQBOY9HF z@$e0-8pNMbS;%qf;Bhn%&vjiYMs);1c7_jLbmu5XD4k zX13URjKyNX38HujOB@%Q5HA$Prb#$lF(*Dzk`T)gawJ@`$yVfdst>MR8d5#+ zEKj6188cB#j%ckPVs4fO9+*1j|Nup|cZ-VBlfNR2QNeL9CV3QvOceqfqN+DA;}zAvyR zUMBNgeXZ6J0hx>^)fr9r7m`#WFq_p%o;Y5TAeKmku^bUglFDL<*kZ0YRg%a`6DM*x zBDTd>XP!%s2~n0L4NuyCO(I6ofBJe~$ooRx7xKQ4_l3MKI4!0FX!bQ4Egx_h4kpa@(&db4eJ@nM#j(G(y@=IXw|v% z7e2za9qttyCzcG%%E>jEi@z+JH^021YU`oui@;(y1)_M9%Y%Gyo4($DUO<~13^E`- z{{DWzE26w$i00M9JA~yU>^U?GnOnBy+ zD4Sdhh1|NriI2?4&FJS9wej@Ir1aU|tQ!-44BU{hTi7e~3&Yr~gy5s$duo3F+x@2- z%|no@TLLF;uhKWon&5K^`E$Ez_JiBYHXoV3YxWm+inr~bw@Ep>t>#Ts(Dq@M6O&)Q zV4{X+^~{RG9AC|rUN=^E3uqAb(@XOrD(d@dcQo~0h`c<#wox8*kGWdVS+8jjh~Ja>hCpO z9`b0lZ`A0>>b8#;GLEN<_6?o;d}hPC;gwD8UOtVeXomJ(m>ead3T72O1HI4Fi_wZnhxtfzFiCi_NufO5>K4ZS@X4iJ7n<3umq1PW(Ug)-< zdiS{BBRZE&I1nnh!wZ@*pwprso=v$^Ky54yF5n&9+4cU-)tfebtU8%ibt+`9SC%xu zpH7t?TP&F5yMF%SFnRB_lFDzMb^PLW;>mRti#Q*zqc1#vtMiYaXLw)wXZmS#TTWQ| zwCTC$wz5lStr)ScOZ?URv;80Pi=oDSK_gbqt84eG#I$*ozSGGoD)pTWn<^_8D5iA| zu26R?p@atBoi`O~eDLkMkh9rU2gkkn(%)-+&f(1PrR|JIsk^t{G1o3w`rC?WJI+iU z_tN|43xB+*IyDR(1pV=}{qr>b+u^b-&gT7{Ez9|T}wm~`<*!)jkDVJdA7B8eg4?Kb)mCs*OZrhogGfio-?@XE-p8c z|9oiAHA~ivSR5sket2l=)#ty4ZVlXhA}aHSu(;vTpr0R&m{&LASd)1oD@7Z5_j*kE zfNd8;6xV)P?|XguwUiR`sG$)fckB~K9!eR1KRlut`u6vm?I|~XhCiOMf8+U2ro-CdQxp2vnx0n; zTro=d*{oreo8uqsZjy~XaxY=p(+sL$LovsLA%oQsx_4{<|%oWMHnL9>L3;%IrV;{w|cilXC|MK|E>-Eu}rhT&e%BR1}!xr=# zRFsrb?EiNAZ{bPNM)Xcur|87_9m_hL?2%Hm=9#K=@7ig@BNC1JBm6^0mwY&@{ET$h z!yYgGRlQi79jAY|Z5yXe*oR-LDsO$cH*npoeYpi`4?9bbpvTvWdUu1Uudm(Z*NvS~ z*Eu%5eCYIkQ$&~JM%Kuc{qdFSGzZtQtTf(g`jt0ExI%Cn& zx3eeh-R>R!LpoOW&E-{VrFexO!xd(n(hX3u`Y9%DB)p zQWJXp(1hq+SBkmSv;BjoK3;x?k+@Q9u7l(X5yJXWC_k zQ+^R6UVPMjDWCh6#~kE0_17r)C({l1SLyAcqb_b~dbywf>g5#r!xVi<$KNP*oXzPG zF$sOkbDMV7n&Xqdspfnza7I@y7XNfDv>v*+#Y|)T zVjfrSGgrkJv1d$1QTS@Zm9S%XWIK26ecg>SFDJQV@aqNF6uH&OZRb8`yecV<`1bzy zy1c&YUJkzSYS7PbL~}m!D+;02X5Wn(T4wN}9IV5JUsF__I6ZyiXWw6;chD>x`Zaxa z%H!LX9Io0w<4ymHA>RD`YgW(S;FIw`?bqd76ABv!@KMq= znUqL(=K%&t!{}~xfFnjrGgLyl#|R~bQKOlTR8mrq8d8GvVU&Uj2*`WRd%gd~%kT4j zUH9`m-=1|KSejP}&_wb`VTmot@h^Fq53?3x#zk7Kx;q(FBWG15Z>Dpkn)EGV)AS+R zST%Qq+!Nz$*Y$?J4Upa)?~5l_ZODsAOm>8w=mChy7GBL?R~^F`kQ|&v5Z(-t7B?~v z8WG!qzXI~gLf-~%HE85kGy@p62J>7f-oHTS(e_Y1S&6I2t+B*zHH-P62p=_;e=|tz zDA>^B6=y;^z8@lVsuF&k!@U|nczmufS7R&7p`4WRoR#HN&s#~R;7ndDD_`YzcYUXp znGS=NC%epYd<|~pj<>6c#$#i`QuvOz3o0D9vmbke=CVCXZ6-k~DR@L5^FswFa`{2L9=Y zT;fZkW#^GEXj=)$Fn>Cw{oA^`y9=RE#YPVL)VuBe)P-T9XJKy)+r9o)j3$9Z;Cx3% z9VJX;+P$iFA zC-U7^IgyurXXRtb7xYK&iDi7EoJdV`IG;WX;kddw967Q1#RJ^qE7r>0;PeKzV)roz z3Phn=QYY800h&kdr0fi#?@H__V!-*ExOr>`idQhV0HPS5j`iOtBS0g zzU43ZaLp9s)=e5+KI^)&LQVEjCLP%)>5H)&dV8oxfnj39tOFT zg3W>KQBzt7%OwQ&>l&bE{CGZN{r3B}A^8j6y8HQ_QvTuLqOWu-nG0{lh54!TWw@EQ z3BZ!kjONy<(jo>WA?gz*DZd`X-)mL-zRy7acI`W#{Ta!*2v7V;Ldw#T%kr-mKktlu zg|lu%ROGTjY9g&XV&jFBhm7JV>9mAleDTWrF8MxcqUsn}Zis6wg9drGrUv(bzklRI zEZIP|O-j_)w7U!%^z*(0w9a==J`WbCZ7qYTzV}_w9ToT6syck0wZm^<$IUI7K8ZD& zxa53Fw&#@Ts-K*S$@;1N=jkGC9=gBzwlL^U3*;`)`@p+z02iI!crU$2;Z-+df3_M- z&EkQEz_}mHBkv>0!m@hN$b`b-N})dPa7AQxNPK!6j;ow{-Kq;Mh38jF5B{l7V4~;g zH)P77sRhazj0&XXm~yRoYGePN7$_XLsw*Yd-T$~5HSwWMV54=9Q?c~rVv}?G;fI&~ z^X%A_^(mjO%E12qL|n+y^EegZ$C?^s4J$CN#$rwsj zOQFbROZz`qyZC>L36v#*fuqPtLsH_i2)1E#XT^uE|YrtdFKRT=byeImEc zUE;s)Y@ji=yH(Y@d-k>;l03!@zA}nnawNc)J*63*s)G$w(`w$)BYsB*l(yp|mR8$u zW_pUCP$@B8>2pQJ%X8&iXS9OFLNUlMH0gJtHN;HA#j#Y1c_w#r5Y_x5_%1gR8d)f+ZHM9 z6;q6p+bwU`(oW+8&;(_k+iKELN%4~Nk`gPOrL#g3$G9Ox2vkqzXSS4d-}t}MaiBC? z=RhKZk-T?w>b!^mcItG|w&tCsBNWG#I3GL0L!%W_76(Dp`xK)l2FP?QnG#(A5~dbW zy84IGQQ-x$>C>NX=@mGbTq0+>_Id+{YIbJ{4`=s(K1RZ6W`%rDvQ5^?g@`_MmT`or z;TH=6o$mny6k2N6#&q7fq}usMMkHAW&=~Y;N*ZC}hAE=wg}lkUtGql2O_wg-Z=20r zy{Mz=0c`g{!)d0l01<;r9;*Q|(mbNE0$@R&ifQ9T>;*W!%R2WO5Vpd8;<>n6mrKA= zw)!E5@ysC~~q(x=saBzposP%(0T(kOb<=y;2fzLdQz0(~Pzm3JEdf0Wx(f zh9^b9Idbn!%7#$d0J6Mzs9);LwXQkKSOsZjP?QumuEIy$nIIyG0$zMtL?D;Hfwmqom6*Oj zjZQu08SIhGnqYzP^j$21sqjuhK9sJd@UUu>l!J%ooZ_r(umW-P_7l^tQ{^A-q(FzA zYXG=>0ssUE>RkgOe)Pb9FQFL4Rb;u6`3C}iL@^=2^xgW+2{1Ah$p?f6U^Ao^l{JrS z=Upn&t_@29PZ*j|U!gBUE(Xe2Y6u0jmsAh{$Ti-vKohE`nk@R`)M6;ipQdvM=S<3fo7HUvNa97Ro6#0Bn7dZt)( zf;}uyNL7tpgm{d-+g=^wt>S!at~y73`=TW88t{DCr?^0Bg@(l4v3zt2+C=*r?8Hmm`NA-kK+}^BZOYf+!ur6ScsRA?l27=f!IPa;UDO z`{?9ZMM(ZMKr_qtyi~S7G8LFGAF5y@mutM1@{-=6P>LIhQMH>x| z^*j^Hy&59RxCc!)OP(d3u&|DDU3FaeC)toyKZ`|L>GwPhy9l}lZ14|v>0L%C{4K8a zH0&{1U+gMr#^+kiXhsd1+FrQmayQB<^se@A?Q!a9BxlsjgC`thxjH?gKK!*9;?rC@ifwp`1m`Anq| zItn#URCOA@A*E{9kxYm)$#1C;*(^CoV4wjsy?NHysJhfQntg&TdMF^MmrRqBl10?O ztRGXHDZMYK`sRqyEvi*CS>`A4EnUEAk$tGn9SQ7sT@BQ-6|t>MP4=iRLKm{4O40%`!~)VKL`u%0zl)gmsy zGI4(zvWNQTDyA>TZVhnSe6F}kv){#cgC4l9f+F6>eyzfQ%i+h)qXvTUg*flb(jBx< zN96OVivMM(@jqx*JXOuVu+kZa8*?tMwsby^MO_1$t3W^dPps!uzYWp$r?;5nIi@l< zPjci*B_%rL+&{5L&@asV?(1H!twXF`JxI7@3f2p9=M*Ze?_ZI4q!8nJQ9wg--jUQ$ zuRLyRciLydkx&H@CZzv860X1Xl%?exiARhH< z&%~_V9<}}OAr8Tbu;aQvK9fwCLgRxoB6&45gHgrRhiQHz-NMI_f1E<>KXv=g{Fx}3 z*&EvacbIqo8wMIKzQu~gkC_ZBolGoN9$f=&Wi>Y3f7Q@@6i3*<|Fs+gE~Vux-^;#@su>9`mQSJdyoUlGDdGP(;gZ#JQr(RGv z?~f->nsAQ64dwB3N(7;rIpx~IPKlw7Wjjkl%3P=rDg48xLky4p=YcqYIU`cg&!piD z^Ut2o)nA|eyXJnUDSZBy?lOAO(Pb6Hfm>yc)zCAg(&Lchy424yBeT-!t0rlw<}`Wl wVpRb&%;q$sG|*IsicfJ~F92@bxN+mgjT<*^+_-V$#{awj01(2@&HzvV07T6)g#Z8m diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testMP3i18n.mp3 b/solr/contrib/morphlines-core/src/test-files/test-documents/testMP3i18n.mp3 deleted file mode 100644 index 0f253704ebb559780674b1d999a99e586de1a174..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 40832 zcmeF2Wm6o%_VCdJcXxMPT!X{nwz$LM?h*(d7I$}F+#M1$xVt5|li&eD^1!Y8=BfWz zxLwsVJ=JHXPxY_5=k)2*Kp7r*nAe0Fq^&N?{aT=efr0(^R~rTl;{oFdLj@xV3hbcC^h@q%%BeaVc@mQGt=R`XR1@ihauxCFH|rIlWbaIcwL zTUt%^zg0}!0?gV#ZNTdz;cEstgQ+ZS%{^S4sld)ua_;8NN9J}O(rPLyua(NLEy_CE z{QD~n1O>Zig|Np&L@&Bd&^}zpn z;D0^vzaIGal=wCO6ATOvq(s5<1_nNe|EB<_4m=EA5|^Dgl{iEKI)$LfF??okT%g-LLDDT-*V{ z7=YqG=yJ($(pWrK%IKwK|KNpCV5UwA`E7&IqTa*9V1N1rC<;eP4jzCHMnZ#)i7UjZ z{r8JY0|Dkel?7fQ5gQ7;WCH*O0Zoql-=BA(u)PsAAyh=DVbb_Wg$J-ynE(FL{(!{| zQx7I$q9LGohX@mbr7p-q8$2f=sPXUL@9+{{q2fp&%%jvyJU1~KOdpK^jkL|bfB){t zt;Ji?ArK%h7Jv{L76x_$gGll>6JK)ober=9;@RcI;laPBMKP7rwf=M|) z*rc^IJ~~f{k>FMuoX`q##Dj8%ZGQ|R6#x&l3~o-C8M?1*t;LoQ05_UAaTQ&mnL*|E zmU!YjOyUBmOU0pDrP<=C?umgqpcgIN2susvN=mly6lr89ww}_VOO0?Znc^6&O{tO} z!%Sr?Lq1nzKjM+hdhT_6gKMq|uiBQnmd%mN`#_%6_~WtE_NVpkHC%D6MNpI?)~ag$ zZ{bIEYynbqm<_H3BL%81#;e}MiC-}Ppi@~p4@ECt5use~vjGbEg~#!VuLm7>wF&3> zPzjkLp2`MglhvP{-@H_xIlwlPER|Cwi+STFqv zyi2S}ZMy>EnySg=C%I8epW8-wB5 zB`VT}8X%VfI-CI;wC7Kb96e18hTtmJ?8X+E?TN&87roBy&ki2KXRMu`s0Cjf?s5Xw zE}IsPF;V;cOP!c^MKE4!$vbbJqNQTl{2)G5R`bz5wADaHG;AY$)Z;$C*rMfLkR>Q?m}aqQ#Q>?5$~^`vp7M>*XbEQ6YBank27S^X z%GaZiWq+KalJrr04P zFFw0_PA7a+@iAE}7TwcVMImcFv{senfo+}hj|>r3X$k4%l*ckz+;anHf671;ApUn?$-xJgqVOwFJv01zky4ArB*5^PLMc)ZV z#rS%(^GZl*X)p4@Hpb#oxNJ(5k;=xQ8{X~uL6rg$MQ%Y4{C(xvKYv^AE-vbfdc?8M ze6+1zyd76juhjanrBn%8)c5UE*I;d6l+u!FQ*0w>W$kv7#inhtApeV;>KMwN!C*aB zyg#FJc$|aQ=W10oH=(czGjXGh5+FX!UwwWdS@6Yy{lNY4gfmws2&rbo!{BTpz4-z# z|M1suQ`)x`_IlHp2jP#Z@((6hq?-s;I%JLJH3TsX?b5uM5#k7XUJ;V^o2E6sJ}|5t zoermA;$bkJ$qIE8r+``!)^I3Y==(IgN@)^7sT~_EIpO{NtEg4jA5FT!V3pYPp~7yf znY9WP43jpG3W}(5dC9or7_OADr_jIz8Ix^y2<}SV1KDHp2Y)4f+e{_6@VKSHJ&?VD zF|JPYpJBDJ#HB8D4!&y2vm)^eT?1z4^yn-(?x7J@!qbB*i)Rr#6?m6`>A`bD{oWJu zlX#^Ie6eA5D9xR%b~3WHMm49hM!VYy%Mj=47yQkOS{2!oo9Q<~-bYv=D}0aQ=vise zhaW0l2{k?TW~IZETTx;#R1q@G0Rg5D?PrmNg@lj)Fybu zwM4<4-hqNV^9*+KO6qh0b(olHxgV0OOuKeB7Ph;UlKGT!4HvfXYwf|<17||dwzaX9 z?omoXwy9SRGlNJD#P_0gzuND??d~y54~?lM_S3OiExk1c-{x2>4~aaydIOu%E>Ket z49cjuC}Do1Nvdccd`sFWYiCXZQXx&8{xK?C+(L`|;>Y{4RB`Z$DQOW|&#) zBKB_WPV8nEj@&;z0Df=1u-)i0HC$EQ_4>?;*Xap^0>VO+oWdY+RZw?imXM1)wNk=I zh=KJvCU2;T`Yj_Ea-6_8m5?<1hpM4+Sbwr)fvL^yb4sp4W$_!K+z~QZeDp`D+D2H# zyuM>lGmQ`$m03#yzW9`XJ1NWH?5VHA`CJ;Cpt(YT=HVVkWmvRKPSPu+F{`Tb{Ka$~ z620M!-NA-CL^a+nUIXY0K8Ori%7!L<4C;cO4rWc+DCWL^Y)|Qz&FC^6MZJcHYZW?3&TxRV1ntaOHrIrOVEZhfFw~sTPp%!4IOu!d%Ah% zSr6kCM?8a_d{-@GdwwtqqU{n9kzmQ;HgLhDs zhlpolaoUCx8AhEIS?25AxZ#yYP@*#2rG;c-KWm&0*KW~z$?j5yItM3!U#p|}=1a_37Uooj1jM_}GF(`@x&-=+CdwBeGJ1ht2efSpCn zrh*;c%ne%q$Nud~yKVL^3;|gc3qn*G%Bn1hCK#>hIV$yRyEGC_*ZP!H)HNQ{y*;S> zg<78FIRX4hZQ1EiB?LzvtoQh(E8E0L1Nr7hfVO_scB0qB0#O*IA;!Fm8!~mQktC(I zk1#I|SMxZ8ghf!>1qG2v7>Ux!Wz(re>^uAqD@<~B4wY1oP2zPrS%nt+Hg-2=NH&~w z9Rb=FK)aRZo4$;<5f73nnN|s&3SBgY5AV(%IeO^gT0pmO&VOdPc}qC0rPndk3#%B> zSE=7d#jHr6<$SS}49O_d4pxw(OHLZx%MOjXwza!bA4GlY97)u(UJ^wp$|xX2!^-?{7UWAEnBw$dpvkkVh7- zMm_thL9`kjqwJh+^GF>rhS;*Tu-{y#5Lxr+mJ=@8g?O$ZS(0p)TZ6*p4B;e{b~g7E zL}4-hP${Zr{*HFy!0VCW?TNw(8dVC~pR|fhG176Q<{NCAMoC+V`iO-Cu*iCp+ILbd zZLJPgvoE7q3FamGmHQ-%5+@EU{>4Ncy#2zie9dR?)75c*QBjcJL|zw{Pdi`m;$k<4 z0Nv@(dAl2^&wkp4Vz#&?iV?C<=gl1Q<#o%QEk%8>#%D3^AQLYqFGJ)dSr_I=;mW}G zEfrS0D6N^k<)=RZBd*vN%~u2O*bP+VetBxk%=RuSLu2`D8{Y`+h((3sIm1e!m%&Gy ziQBG~P2(9<^=7Qbf#C#*zn~wfnZc& zCsD)dGDUloSSj@bv1Vbj?C5bDya&j|^!0u9`i6(6Aw5Z{qEYgilj(HE$W#f!7C@-m zGMBqp!iWL>WU`Z%%vj0syqR4y?a#&xOw=aU-hLOX7enQ__=*oLCU(2iD>f#8m~6D( z0&F;u8pHpQQ@>At*$0J&jFAZ25;8w^O}> zseRMRSzlHqs29jH)l&m`U(;LTnb)VxXVv2;-qI+5v1ap8y8VG=R2Z)7I!)G-5>6$&Jy?(vPtym?5a`4m@=OluIlueZJ>2&O^XgiulYY zzsBpJtWu+`nLbxzUhS79*J2nP zNPB;12MM8+5c1)uJS}d(xKmxORh#n8pUXb0sw1nJvUuBUv5r7d~O zOtL131!3iSiSPMZPw|aAG(|oMQt=KrHKy1}Y#Y_-hz6pc&gjQqtD^tJN^TdEwkS^) z@O}_AVZ(s6cXf-PZwTs5&y*M0teD5`FLlPVXexT(l14}AApb@Pdww<;H|tSmstk5K zQ=$^oQs{?wpLmdwCM2er9cQiieT}henA*5?{Chz}+xdArf5w&wRYzNr0CM$m_n*RD z&<&1q58EzxfaQ)ypPG_KC*_4@u>EGy z-Bw6xo$l7E{ZnhjwFkb=ryHzv%Xo7C+oPY{@u@We)!OHQD2ZafIQp1A6OM*-U0`$ zr&$o#Q^LVW!F)Je5MqP5^;fm4W6uE`stfU2rKPN~l z3SaXqTWY^JPZ0JMNR>G-=C5Vkj1a7-WvF+Z#e=+zEi$U;6};aFg|poT6IZ<#t?dnw zCh2E9Xm3D%5UZrA8gi8!uU7oy38Cfu#KT)Afbp3=#=#rE#;GF16YDHBjeBRRcL0Zh z$_YiAYr<6v%Vn}%t6Z0oQu`nC`&N2-g<{phU;(4ElU+Y#N4EJhb*SRfGdO?f(tLhn z!wf=tlAy%oQ(O60J4`va-G`#yzukWJW?M)=V#w^H>E>aXs*xgO!S$~IYlb4d%fdx9 zAy;KD(>n9ArfqUm##evc=q+PaddL~}gPq*6WqxFeOtQuzk{T-ech4l=o6Ogh`aN4W zKRtYO_XmU)`K5K~yVW|aptQ)gKZlOrzuQ#(T^A7opOV=70U}nUgW%zPQaG&s4E!zp zaaqB;oB~|`cEX1mmuIro{R2YLG`0Jgp%olRY_@-Oq#v=E`c0ENGI6-I&0u)s>F49# zNi>=QH4*~sKuBaIa#4u@4;Ks0cX4Se#BiyS<(aV;ltp2Hue?E2_gR7QVs}zAxYCI{ z&mwJ((_HG}ndl5S`vZ(~NDNwBUdbpi))KO9Yq&#>7VM4McAgY_BZO!93r4>QQ4GB| zMDgMSV}CQiFYw!P%}1=_a~zOwIe)FX zwqhxv{?0)Iw$2Cu+Epn?5ZIO6<$-!Nlxp2xZ*sj_A|6ItpGQvi96{`wePl z64>v9`eU^8;^(lZC_c#Jvec=nDKoJqS(pLBZWl`DE$OIAvhBjhAmx5hu;HoATX6B1 z7#u*hC$vV_up|{wFTxn^9M)bn`O~uA)BSN!)3;xh%YM~=EEgXp?R7b18Vv`(b)4Sv zjx7F8D@Z!&ohvuS6pxz2*K-=hF}45>Q=2u3fJo1mhS8 z_((zBDC(AK1>}`FC)8=}z&inz{fCm{f=Nv10q(l!P;GZwHPwF|i3@xopf>WX5e*() zp>dxw)MyH$oalSFrl)??vml z!%&6$D?>tQ)-9f%Ko&g$K>NwJ6gSiF;tNje+J$=4(} zP!?`3JyMW?M^)X`X)e|0_mRlq<#xE#*PpZ9koLVdyP;N&$vF=EF6K9y{QA6Tn{EzH ztg7n#`$+R(c=X05T1QZ$Kdwhx6_Z&G9z=?fCKgCTOOROiJ;MYaaXpMHwh-#16sN0D zHSx{1r;G6Sg5tTR->>>{deIQ(H$pEiN5Le~u)-Mh zDdW3enZRvy<}|pc{S#~u*nLGk%|`#}MQ4h!U;wT0G}a$sao%F|aGVy5!lC;C=Cx~ucxH!}mNmhDpRm$) zI`?gd7DEv>iaysgP?$+UaOlv+FrBUUJYq(Uf8De>@Egdop6dWF>ss^K*7mRcg=Euz z{rJkVrMklI`M2{+&C8t^_+wfL-II&{L)x^cFW$$mYlJ*%Y%5<*j(rw#nYjsAk2Tzo zS;Xn~MDg8>R;>k?o7G@HsMS9BNB6)}rwyiV6qPSId}1vdDvf50;(8-APj(o#;g2qi zAr=Nt^X57Iz=YkxXEmC2Weuc@9~Wr+9r8l=_##jOm5|{`@*NL2r3yT z%RX7Qzgu2-`BaDMdfBdzq3jylKTC+#PmE8gi`$nqroF%mO#u`?J3DlbvkqpQ@3MjA z1B(<_lD&!59%n42<>dt#(pZ+V^+ay8+CxC?u`w!MbPW2Lj$Ov}I>Ep$Fzh!Jt=Xh8 zETR=)&NqfC`{>#Xhz2gc31S?x()jtdUEM7wOD?43(CWmg_fYd`VEw%Ic&HUk$;(rF zIyxh{>O(Zfrxo6NyM<4d4_=iVbw|U$%h$Y+YTBt?1H7yMnWTFut}|LbK;aBqZ+|Vm zMAFlJm|r5dXuaq0*$Oy{;qFPE3B>u>$cX-|vcU3`VnZqueHY-}*6jDX#Gi(V4EdmX z4)5Gcp?QXd5MK*CXF4pm<@Anjt=93XVOMlq?TsW>Rk^!18W^<>d?SSaZ7r109`?;I zH$l+PxYW}4JDHTLjcPb9x$s*oSBsKSvaLLdcy`tvL)>MGUHq)iWHX1=a_^{Dr;V5} z6V67DQ|@M@znaooaa9x)NpqAC9$`4?J@Z`@Fnxnbt|v2E&>4A-LZ*6I`_Fw7V7iB2 zx6yqzZ*%>Z%4O(jD?UxMt-3bjiNicSU+VrQna}%Ck3HhB;Y3!9%2{dQ4;=&~kk1p7 z4osgV5>s6nSTWmJ-4BO7>4#{@xq_kInuPrKJlHIGRdr_Q{rQ?^N4y=Zx_6J|;lc5bI=tlPmH;<|BcEVd^w)xr%A zo2lqvSZ}mTE>$#3vlfr5YolyKO(XOyXQy0u$h&$2OH>lq879Eo-GJ=Wplc57^j=PJ zUvQL4v>6tlBJZ-nZ7w?mf6pCxLu z1SVuYR7j4{xeZB!=LE?UG(c@p>7~jJ>9W?qA(SKKmK>UNt`;|qLGcYOJ0q(#8O_qT zqLNw*TZ5UA!ru4jR>o7foN*WrEkzL#VO{zmL<8@7giS@!86ugbo#piQh~K|gPUefP zY;(}u4&XXmoFK)uxTf2cmez)w4425k^>rK%@5Nf)RV|mVN;{Uo!l;Z(nFS~&D`>T# z(^?&F5Osky*lCb*sWq->1Q2XVONTuSzv)RXAK`R-Ig!Qa020VqK2{~aQ8ubs6Ypd1j?3o~D?YK5(T-MJbLtjtnnM{P> z8q3g~sn+CYn$)tHSDcMmoJ=KNbvS9EZ?H7FDi;%@%*x!JA}R5|5z0{8gI|1tmq2es zRnnATl>Ao8y)H%gQMO7Pg-4JyyRtUC(VFFeqF{f@&&1w@WqOn*S98dZO7q91jZ$f&E?F(OsDDRq{|CDb*>zHoRmhD4u`fYk^oy@H%~)3K~Td&;iN5n zXs&dPy(v6)2q$G3cm`;q20mBvH=*X@gx(2$X#Z} zhDB83nmMWY9yI8ww9N+!J-uoUR)XlJW^*!)SK1MkbLYXg zMrZfljjhsdiRTTSr8TRvvTJ(WV802}xp4ZXyKl$Gpiv$Idyk=LjXzusQtw0qCf40V z;!GKr5z<++(0!~f>@gZ>dA~2(N~z}<<)0QhG=GXvEw|g}r#i7hNH(8!+eDC&C z|2kN5|n%q+wwy{bUbirQG=lL=A{7?t9r= zEXvWz;jk3(AQ7#4+1W^iwAb;HyjiClAZ|qxu0b?^+Y@UbI{SZYxmDIPcp|$;ah1}g z3DOToAtQ8tGCe~rqa<8%(#U^gU_RS_0>$4}lI!_3Vb zKgTAbch|LLP_pS)ncKYD`!XTk^lzp}b?lwsnTX@|@AQIPFj-^@XEWlka!uSHeOBin z-*DYBw!h>uIU8|~OXZGRf^F^ViLHxQt)^c#9Kctm-if($Pdv7I-LZDNWi5jdpBd{u zX*_g+WKDzOdwwI9C&B#aKSd*5{kSRoctBT?la(-Llr_Pz5V6|5JGEEo173|)0Y9=0Kn5dh?>hG|$8w;9k&a_;JgN}f$bY6s z&88-s&YI2YeX)niwp0-JZUO&9satNhedVVDNsUvZM>8i^CLe|X<>7aw8y|5q(^V zRB}3=^C|qr^18ewwiL7+2$i-G&F9oiXDZf<(l}o8KmKB4S8%`Ue3n zm*)NFSUEmr5YV65vHC~K!}pJqR#f3Vj^lvJlg^4Gfyu%7gZ`#sFRkGmppcM3+)9dw zAv4RO#gdZFsE-xp4MU5iXtn}VL^yIV92N{25`#UJ6t3}6ohe7FKuY)sk^wmpPFps@ z5QD<{`;dHZ9I)1CW-c&~J2LtmnG-nI_p~)IYpUeL&0q*qFOaRnEphi=$FV7>jo%_f zM&WP8ab9~XZmT_+mzB%y4Ue2jLT!SgkGF6$QRcIJ=clHcx-?RkVx!%1C;PIrE%^3` z(_S;9K_ydeKMVc)7*_9xUfe5C@Z=qd5DQ^%=h%b((&i!Wp<_{9-W z&l7tE^6L8v9!|1$c85H9lT|35|M072{Xk5Hwd#rq#~^f^E)m|O$nQ9UW1C)dRlA2N zz#`l?>a2vMH@0(YrFAoD+KHbe;^?G>?=(tOl*+BInd%eIy0-BjpTdO2X)qDuopf9; zHN3DSW6@Bf+qbkY6F;ZL(>e09^}8q>0cL4YGYOW>iwc7|daQKcCRJKX zcN&^aevff&e{5~MQg*~F(b_h+7}qO253GvBlBCY2b7kB&4Izd);YE>uf;mSBUxJYQ zD)Do>RX(#9;H@5f#K?DhhHC0(j%((nsvit)h*Vk|@suWYWX?;gcB5e0fN%V*M}exjABVrQPDM zbdCxEMPpPBIGvjL*Tzerq%uJ4&n(fak8h2x19;CY!;TY@Op0(sF3&+z7hm<(2 zXnW+K1UKdA-p~98Ma>e&TAmoF!@q`0SHhR?m@Bk&cF>xt59pgzDAm9`zcxJYI&-|_ z={Zg=fFRgO+`li;LFm(M*YeIs=pUxT{}7@Z_?_&U%Hp=>(P!Uc1woyycE%ca`oopd z>dmJd8q3P`OHfzjDRLcV9v4F4$~GpyT@n3hB_x;hjW^al9l#z7eG6!e^*RnAR(W~# zoCYg?`BI74%phcFK10bz$zcaTadawoil-#=m>Xy#FkYNEZp@IXt~WoMTi-)XSSzy& z6fb92lR40LVY4%FCSBjc|9lVWS%f`~|*Y?r7 zc^|}0=B9HS^KoPKXED|$#}7D^de}RXGC_T4&U^g4%9m-8fE3G=)i6m(jze>sKnjBLWO>Ha zGU_(@Fbp14!(o2?dQ2Y1r5d!TPZ2*+IiC*(@u}||w2Ygx`1Ddgw~9;M*kYN~`!8f> z)N{}=#H2SgjHS@Uu~6p^DQ}|ffQOS?lN$Lo`X5N5Cwsl0T)mwH-X^(f&R;OZjgPYI z+;@W?B`YL{`hFXy#mO?WwOVC>MN3tDjSODlWlF7-Lp9+PAC<~bIzUYC6awdf_gH#S$LYTQ}f^|u%4epp8t?At*w&1>L&Ar>4wP!AKt zZTLS5`CwBH6$EVSM#3dWFE5K}MXK8_axqB@*b}Z(QjS#>M$oa#6)CBDjvO(Y58pS{ zd;c8Jdv1h3RAF+<{Pg`3+rIl%TtPFh>1j#vX!-ANEJ*KQ*NKT>vXUO@n+Nabk>Fg( z{OobB_tDA0vv@JMLdIb03dE&ys>vPw!yI9-u+pwKL{?9}44Z;ZGxLp*uhe(=ubZds!U=la^VSk>BK&U#6gDk7acZW_ZW$`i8jsTwnko<+g(j6Tszie+M4 z=TtMnQs3SqMu6T{>zO-Uhb#<`ZC#yhs}(F!s5nY{9a0P)AFEf9#1hS?jq$Q=h@!Da zaTc2v>(+kQiK@5yBF$Pa@9Z;*gPn&`qs&ad>{ZJ)-c)tc6NI1KWrmY zXN0`kiW~H-Vo=He`{gUeT|6eoUq3s(kEAny94zZ@O3^gU*^aSn(;J|41@dW->b2MH!MUyY#S8|K#R)U89iW%s-qHs|gHHA%&8 zTm?7r)_|KogG$cnYTbW(!ZU0)`McAvb2w-x7Wt$@+b6S5e=4P)$p(IN`IF5^5x3p? zye4JtB~mwi+ikbzjM_@8Am{g65Gx}$oy0ot zfPiT1@-U!(9fn}jY@pP^A)zZV*tXbxW1?F$IZ;$(ms+#hD)x!}%;q2V233HXuhtl> z17>F7T%+Ees7~8zSA0)fN7?GGaT}DO8_*NJM&NXb;|gIBm(t(~mLa5-^ehC{>FP3= zH{d|@u#q$6Q5ixC?XrrK)NnndiqU{lHKk9hvTa$qy0#Co?4VBa-*1Ej@ejiA4_+kD zvr-1bzcR{>h(G>xApo~9=YgVY6}%Z4I1o|J(nEI8x*VT_U}^3AJPm@YD50$ zeAn~m`qkoPuZSkkoFGq2>vU87sCwe{EJUd0QpgljqBnv( zXQVVy>H3GoUXB`!oPe=C6D4gKkMvBH9A%PK-bA5(q-iOwQnykh5fWH~Xuy~viDoNT z5Mzg@U_nZ6v7WYf_ikf$ngf|RBh*wT3g>RMn;Xj4f^XM$dM>*06JM;$5!iW9L#Fdh ziGah%aH1nE`y+jag?EBfhHFrSHX6~A0Mz|8RtT&it5W>9R8QZxH$%5xWpbuc&{W1M z5J59YFh?8c2eyqKpX|#d2&!MFlVgz^b|5p(&h2vVXz}uHG#j%}IJ{&0`k<3E_auoT zTqSH^`0YAB^eb+FRn??Bby&1) zDR_N~(Y;z_ZL>xz%na|$i`V&w+3c1;Bv8J8lRS&oLGs%=$ecp-FCoH6ls$kYk)`w^nD4+u*ZDoVVh-n zS)vF{Ur|8%x^O^iNAe_<2bP1bLy`!wv*W&_n>Af0{AfD3ZzHLD%U8vYT+!B9m5MHW zd#WGUSgpx$pzjL_!EYWMqe7*Uc|?|Qk+gO#j-D@-qtDbr;J`_#05_NHl{3Yjd&)qF z`%lu=OntSPS}&CRr|qlgo=Xl-#^bR}v2x{U+W(#GGD9ToL z)AQ|`SFo3Fn~p4xva$_{tk1UF)4uz6_{<%U@{G$}oqcnWu7z4eQdBXt4M$LjNaucD5+jXg=sw(YJ!Wm2ub|17gVR|I+<&@y)(+J20l&?Muu^XFL< zzpP~$z2AeN$CH?*Wdp2H#3OikSadnG(Qan8GOo$2&g6Q^wo*rFe(e_rZNAw5y7U8$ z{`!c@kcvT}7d4Xc`#rK-_^##xA6Ih1QABR29oarxw01yxd~rHc6X6@7=*G=$9tg=HorYm(jD4&deJB|{%D)L41QcKJ6j|mM)mvQlfBDD#8O@5Z} zv)TfDD6Wc(S&3$`_>c%^bz;W!_1NKfTuj)jf>2Z!0Ftp)s+Fe1VJZR+HJ=5$zA_vbNpl$! zp9sOlAdN;b(#f=pS<`jNgZV%*MeI^eN#N^}kseFE7R?T)JI z9$gY46lh2-QPF`FlLA@UBu6AAOYnRZBV|+g3)3s4POJIi(myI-m|(ZiNfS}pr1C5M zX9l&CqMJ5+zc));r?7f?Yz|!rtkg$3;S|z1wOzXDg70?yyHm|7e1UI-XmZct*C=2` zUpF6O%)T7G+H#$WAStiL0lJ%%?g4LSc<(ao_)^FOiiJ5fRhF+X?ULrx4Csv?+Y;H}%4}Does5lb zs{L+2urTZIm*w)q|%N7SUzxM{wQ zn(YjtptXjT#WCG3>N(rYdmdIPjnSrPM9cDHWA|N&maLj|UI^W>8*YYBJcwtZcXy-L zU#(c&^hi~)LOlvi(?KjtrkphwGud|HlS&z~E(u5)9r+b0=$Kf~)V(jCsSLFz?Wskm z%?~nbH`0#1#Xg{V#`;Q2o5}OIKGydZiwonfF4P;zSYx^iF1JM27oANDiB1u|2%s$* zy;vMvOiL2Yy8r8S4!swR_`{_y1e)5}m0f~i2o8ktBNg!6B~w~vENBz4osyKW6>Rvm zW2GVW9+NfPuvDdGeu2Rx*;35_zbVxYd+8n<59fZuK#?{OD3=?-7Q6dKXt-|&Mu!3M zb@PF5f=hz&_4rW0!Yc2e)DtThpt71#JaL+V)C4`Mf!Hbm4`XQcQVrJIY-9(2r=T~b zEG=M%j7O)aRVLrYb4uCcP0)D87G{AJ+)9CYGGUyKjhwL@R@5!!*}yPB7NAkpke)F; zO(@l&t}i1M;eu$1FfS=bLz53QDT`JsGbCVPqEgur7J{QfK;?EYt$nbrZ?#Wy)wi;j z!r9VIGo%tX8!q@;?c==V)f<2VtQ`CZ4$&WIX(Jz zGe+;9orC6}Zdr=ycs)Ug0*k>SUOj+-!=cz(OF}uxnj3jLblG^ly6BA%-=CvUBJOAL z+D2Mb#4n7m)^_Jlbpg3TH2_9xyoRv4Hup?-Wox-bPIgHnE#~9^W$}*y3ss1`QZY7;`dEA7Bv|b2?^ed%`;ycn#+7PsNZgWtXcvpwSaz;n=`J%jB}7?% zDwX7_^r)e63Q;jOo4lpGO~l{D6^)ZCl?$;KWX>tktq*rpnhr(O%H@_JIvET1dh zU8?^YJurF={!hTo&u2X-Kg|#H2ifhEv^G5$RJAV-IN-X9`dHqbGitvPY85{V(Z72U zt)qV!-}}nA^m;g@SVS6;6iBMwxiistjbn>8^GxRXt>S#R*sVEMZvo^!swby z(b7xXA^`lk=gh}k7l1t^Z)=mI&@%b;ygXyZ#CKgF183SeLQJhHwhIi3r$~DIZe>TA zQ^R{HI-ObR`=GJu-XdH>pnMiHjisZ>QreN9jW~g~ecKqVhpD-3WMlnz-&X>Qikv&7 zF*5cNVc`s7-@p1qNk*nZ^cs7iOiyJErl(5~hzC37W}XNjo$t{cGt1aZoZ*FP@v_R> z0edd!m&}qsq4G3_Zx8TyCx>2X0gNNcY<7sF$u@P`%#+POvo_qplDO8je^Fm#>z{pO z23zd|7W}$jL+&pQGd0Ck@uu3Ee7k)Uj8xuJOWI1=bGH3vWw8*!=wnbE+F@d~Ej){K zb}Ee3iI=xGp{GNVSvoHAht-xGfKWNAfBwK_$q0gYxQY{eB@KT)UAOv6TaC|&Xi>jx zC;aZ(JQSs!>-YWm#8blov6F$Ps`}%j6@GLeIVDAtZm1 zg*n8+;Kq9^iQDLy*@Hr>r!QMQGIqzAQuXi3nccbo19w2h5v*2(uO?OU+2dR{xx1M( zGJ65`IsNbx#F4RL@3kh;SKXmMC@BLEKzM1YRk}s_%cZIJ%0Z9)iu#aI!oPTtw+#>V zdw+|{v@T4WYrbsipR8<-rEeJZj!+%08U6d(Tz8d6D5op^+34OTF(m;d*ms1Mde96h zYBs8|3^^*i_%Mfxp^k|`zNZP`J?yBVSB73#-?=4=zM)8XAIhmyd&=ZYpt#PkAB|Sg z-LAe`TUW8zT5iwUf2BsxU*6+Y@6+45sc;=Swkc-5yY z6Dy|A_#i2zB^c3=zZ7{~*aJM#75lN$T%C#jPnPbD1I^c^Ost~e@rxfmhQZF8OcLY@ zTRQZtuk@WiJIl9b8?+b$ol=!lxfv-$^@I)O_w-svSjx~6;`xa6BTeew2r)dKhmwRM z3fGB+pmg+B;pG?-e){?!M7s?c{O2l#XA{j#4D;_=Z3p=@gx5}oeRyUVUA%gF?Ow-?)9Wz|maYrHQwMMC_}Jk@SrzGDzu2IVh{ud{QFnU~isaF1+A z$NZ`jZ6>&1J{Xm*h{1SQ&~Bn%)V!^?v{4qsjDcHhwuhl8ofNrNXU?ZXJGx+(zhIrX zbaQ7>*BxkJw@FN^RGIaQX35m<$50IYmUP5T-m(e{jDE-#G12HMA$1-*9h<^Q@Sz|r z+UG%ZgCu$szM^aJ9wH_+J<>s@IY6V^kCBSe~U>^6?(Csc7fp>3}K28zC>_U!la? zZy~3_sIrn3AtU0yGd>zZh03a&LsAd$?c8OuZ>l8YtD_x_oY7r0tArTTE5DgUII9wr zDbc6N^Cw)2Wdwv;q65@Cz3-=x=83zk9wC@}BWJGO@T8!CaPvV9}U3;5+ zh|27D=VcMm@FZF^1Kd8+eo5>rAp+3xf2a_2RamFeyyMC0C@lu5jBc?!L|OzxnEPAI ze!|a5SfMbupJ3xb4m2bON8^#88Hl4?hvhbh6OQ{0T^LAEE2zFrd8)AcJ7W+IZd{%R zmnVVI$E4#4#<6^9gN~D{hSs&4Mt z+F8+x7DYYkiH5AcT|Up~1Hs)4QY@WP`9@(X(@>uW-xdtRIpV4g7?{F-74d;#(8e;7f~x>oXh={ zRj?NUbZO=N;Ka4jy4feS-1_RMe?VVOL~okDuQ4A6ZagV8ss0~FSN+gr+lA@QQ6q(I zjFHkM3XB08J$jThj8Z}rELsKwVH+J114as?!Jwt3)1=c^5qZUWKm2~#AF%5==f2Om z;kEUOTmZzO=7yrD}`@Cam^-g$Q zH}U6cK2|$wzV|=7Eol|?%w$-8SaXvr%5i^q)yF0Bz-e$0v6yUr-`88}$G2a4896G% zmn##8m$vkudp>vS514!X!nIC3&yYT|ya=74fteG?yGv8 ztEq`t*Rt{#q0|Svv`+26uUTH)<^Ed1JDX)Y4Bq4}V&D)Vs6t&Ry4rP!*E0DQn_Uo> z6>4qVYFfQ?xv=ZmgH29*;6$gJbAPv4KH3uhQPn-9^xBSBWy;BRxbaapg_tJ>fBSl4 zbWHNG@Br^wv|6{NwCB!8K^4JiB78hLn*kiA>ao6Lt{V=o$a3-F{CDhgkx+B>d=3wX z)E8OcEJ}|C&ur`U?gb-)RtnA&Smucs&01AJ=ifk97cuw~hM>I4&$sXu7gB7!0sbuM znekwG1*t3e%J*6ac&A`Tw+Yx*v{HrUs4837n{R@Y$reiAudJZobh5B@2F8qrva;_Y z5fhZmAW^v_QcuxGtbJWmV398YI8fQ*P!gz=rkMzYEEty)Dg+iw3net$C7cMnT3RL6 zGMvGzn^{rquLSsvaT5&S<4*Ru$38~{uj0A$po4d=x0*ZC^y4eb`2bAOu)w2_qE9kw z%_T`1)XP(h1BnJ}bVCdl?Nmauw%gBOo?58eB%jeubAY-*jZ9Jm-|$oNeIMXF5>0lg zZS&I2ay|2LyTdhySR!f9J-)qZ=BLx?bI~^^0S~8j z?J(cD=1>0hasHbACb+7~x$RpXciNZhI+Z+KgGg?%waYf087HX>*;q^BmVdtKtonKxgs|MkdF#}&8r^?V*A zAjYUhkl$`7g_4u0^Pnw%8UIW;ysD(F(^d0bU5RV?hfbL)gnVU8i!Sf`WDt$=HB?l$h4a%x4v zg#Fu7MN^@bO?z|y^)5b@&^a6Ac!To5sED8AYstWK6_t)TdqReTrZlwhC&WzoPDw10 z(~{3QV%FGoXraj3Z+o2bCj2BY4G?x)x@>{nwr##Bkxz?#k%5W!lVpB5&&NxL%m*i= zn{#UnSJs>1b1ZagBVM?Qf#d46X;Y_Qv-1# zI>s~)Tq>|`GpUDmH-6bC-x-cJ_IJCFGzlo1_%Ijqk?zD!)9>B67S#?uEyhmk!tNvU zCZbB3Bj985v4&fdlDx85L9w=XVHFXUBE4&p`gp2SucRSD_A(RwoI{2|Im0BZNCyl< zE*m0+%&?GgfpUKBVO>7+Mv$xzF=seR%gV@sosdvv(`~7diUduW=Vn>M!YN0&^Jtkqyse|HN!nGjg%#ZJ z2=*^RZ36qW&SuOPH6)p?aKR6aNM|{TxN5mxzT%_>9J$u{UiCMbOGCz?>&VspB*?+m zOcl+wz#d{dP{7~hF){G0wDy#EwsAx`B!*hz-(51AX{&sXt9|2o;*!ARY;tpJg6Gmt z&J6dtiDOi*UKZ$)dWNx}YHiyK9Zk1%`){mIlWa>u6tN?{!ZG7Kg4K0)-<@AS0&>>D z5IgIVZtQxQ6VIiG8i#Tu@E~-Gc|%wAinKNNl+Wq%gLaLI_r%;NP=#uiKf|7_Q%DL7 z72D!I`Y_k!E$g+minTEl$Dz`|im@NQnaEPhNK{F@Gd10zigsS%U60CwdK#kI{cb6S zsqjGGrRG~}+qqd2vUwb*At+l{+#+xwbfh_{yHzz-vPix2I;H)|(OVr8UbA-HQ0;gA z_CXKXU)4UYO1UdrXF!Ln@c8JXlkQO}ggz&yxU!653qL@7O9tWyV2T;rYqqa~G#M_& z32$tn4vfh>eI2kdyGP&c9V_LiMXMvF7BQ-Vl@=vXxK-mGY&HpTu z8gI@9MY3nYEPU-1C`~4;#WL662n+lHbC1Z{7M(X{&I0JulmV4tR7S*~%NLx}Ck)EI zuj%=ks3jb2XT_RP+~e7o|KQ8x7eiukLp(zIcRqe~tviH4Mk#|?O>1t4vTUA3u|xCH z)<97tm5!UC&=k3%mytUPwE{{+vp7J!+LiqND|kx^|0h{DE3ev{?9@}b)6O98ZkBKP zkI6+xseB$(ye*?d{h?)SixwmC2hpd}J*(|toKlA?7g<7kWE#(?hvw~pxcINCdpeI^;Yz8I3Bix{aEVL5 zdL*jN{FrVlanv!RMm5}0XHrsm<3UehrSj9X_1GaIYPRsKdot{k)6SW-V$I1Iz*1!I zZw>vve3~HY!l|v~C1oVgx0}-;osG%hk>$jfRr0nzXtr;F_6Utv3PYvUGsdTv78Iko z8|vr1s@AkLJVW>L!ki7IJ;exi?sPK3*;`LG&5iS!OBTF??IB-JI+%rIY;SsrBAOc0 zWXFoHn!Czuk$)5xQqaRXZIaAS}GEyNL5E;{PKf;dvBfXy=q0#vFd`I8;ghs4c>L4MV<`(nq? z>De=T)8;35HZ^qs0;>^PrG~Mu&q96C%y2bCymjR0N`X}u%_$_iS+Xv!B+!NQD!f_4 zwyl5-&UO-cy%VnW-av@}^8l6QHk2=_noq>x9{x8^w`(bz!B1I1!3(PqK`612S#Ju` zfjah50;_&Ef1FYy3w?&tp$zg2_oihGL4@gO3VZCiZg#PVGTFSzGXZIam>e>Cf}Q)) z{3?j2~YL9a$$5HDVuc2v&ygI-b5(q++V=q=xVxG^no8yC!mJIZxe-YY5 z>?b;3W7Jc+DrGLzR}AI{@mr_50|^(5m+~JlJRd^v`2C zP?K#NN95b>OpM8vxw!6pn*V2an+RIkhQ7*$$ygv%H(KhG4E>z?f?CtWS6)Yyo4geq z(?P9Z<^l5NQFJR>ILf2Or{r~@%~cxXa2B*t00`GT^qZf>5l*-x>VTA%X~x_x!1{5& z;QWS~VG8UKaF%Ng(kv8$O$`TLigivQtXBFerIJUzfGv=lE5-m-$1xdI0(%7^g^Aw2 z4In^4q=u?C$=&O_%zX=amJ3Je3 zyk+7MVBf$hW&1VtV7R#1iTU*)8dFczEq|C8d*hgCYinuJ9qYpqob1)(Q8%? z?$UAtyBnTOdIs{aNjpOk#?Mc@iz3R;fCI>f^a_+BGtyHQhsj!qq^w3n+h2rI8V+d$ z-_aW>k|!oi4FAfum4|~;%ri~1#(GSsbaaTBra4qF*Hg(KC9RTZUmhoRMLjp0H3NoAK01W}C|qf(3tAOltIu*q*HR#A(NGZeIkg-QUaIu*%Dr8wjBtQKovkjx{J6m4mAPC6`> zTPRJ5T~`O&&Jbjxwz2AUcT95M7Ji!cngY6M3$m`@G6vOZkjJE?7~M2h^8&rD&M~n% zyr&c5iBb?+NW%y#AfT9#uYB2B*;Qs`CJn>Lww+F$yc@92(>l>b8-*$pc4VQ!l6P$} zIM*6IMRpS6wl`O5bwNwsjQU1nK=UN7m&UPtJMQV739ek_SF@T~fgwNrA2pK3ydBB& zvrocek%T$J45>tjGqS^9^198@hJ8=)Vfly0*iY99J}vUy<@<)TZDoHE%JDv;6PEo? zZ(0R3!SPZiyA|3!{u;~e9;9KLVL%XIPs)pszFEbqc~>xOj4w#T-#LMr4spAO5aO<8sJ z{_VC74X7!9RG~C5cfi_|6;FOAh_ZwENvzH=Ffd#?;cXGf^f?$^8rpbncHt#>;AXnR zYBG+eM}nRUEZXpuDV$g*L2Ygpns3<*o!qGETlRvF+#TZA4vk)@{&Qr$0d<8x_Wsuplhb}>$*B@C zP-%2IJy{veACT$UQK{?&icmMJ^vbiX&uzZu@`SKV-D5nj* zg_e}oKNt1_oJAVT;Hw12rI5m0^TzHn6Hf#6D_HM?)%O}|4FNL81}CFlUc5GlEraE% zosY7wKB*2&Hn&}DC2%!$rj{MpTjiR1AL`nu{3J8d1QN$4@cON(i{;S)Qz2#UbAEEv zr7iwIg2zQ}B8`DIZQB6o6*f*rrNY5P(p_1VF5pBvi9qL+8ZU|k09e;3v&9LlV&VBm z1zwIKCz1|5;Q1kCeisGp+5-hdYLTi~T0yN`iD4xHSttUJ`T5mej$a|3+5QxS&MHhUhq>h|{0D$!W_0kNyYYbGHaT@^}t9DLLE zvPX%wjCPxKFtkB%@6DD=YuSjAYKJiC$HXe%u!PdPVT^DUpYGTJ#4XiEmnT8_Yu@N( zb~qGJb_r1>2P=HVAen{-6*D=)b)PHLCYgjPqDY9wl|PO-AXHwCWLO9s@&|SGtsfte zmthhzBqZa-4h)Z|l|!-^_A@h4=pS=4(cK+mzl<|lDJwF5r$lGCL5nXJX5;j$C=$f+ zp=}lxAu_WL$ETkwS*t4*fk9yg#5xW*opTzjw{QVPeysQxA&A#MG|p$VSGiQc>=!bH zixScq8g;TL$+a{TuYHsw&9BdjmN%DF4WH(vt^qG|*}cCuw0y-eFnFqXGoflC zA(*+a;86z!$6i(nE+5HUY~)!Bq4}w!@JNFtbzFcvd}OkfUW&h`2-~!jJiM(DfW%w2 z$$ZQ{CHfUzJ7xPWXW#G+87`u2ZPydU%0mAguA055WY+m!p~~4xDxt<;@)xq+cy`{IZ^DgrIfg9Vi6|LUes_dFLN1O56=0Rh_w5raPkL%Vqv*R*X zVY#`Fy6MI=hZ?WM4PekNu?7sZpXfNBmZF9`mUbPZUbvfqF}5=HTyfXx6^0=!xB0V& zrhXw1g`8}!d5bPvxT_kgg}v*n#fYRGG&~g)OfxZ(<@t+{V(CFV@PP5c2`-f`ZLpi% z0SP}1#AQG28M;?GmXlY3rTmf9G9! zbVgd?ifv6QNfRFG_G;}>l)vH#LQUa~YYXpC{Gnvg=uANqkA>OUsgYdLsErng_Ys_o zrSePXxznZyE(A{JC2=1>vJh)A_z{a9`NM^43r+SJ>^Ho8Bi+aksYR@UqPBsvP5;&i zlThlvY8$+(01zh_Xr9Y&7LrigeRLz)o#&IxQtY&!sI2mf$|3VA7UI=-B;|w=4HPmt zCb<*Z0bohx3`d)?_+0ZtU$Wq{7MPr9N6&qR{1mD|RmU#>zLbxhUjxRA@eSqoQ!a<) z%sGcu)V`!TPTpAT-Z!1{hN*;)tP1%k1mib*awE(EW|a`G{T;wDwPA7!!%1gQ@u`ml0Dlu4I5AX-{n5-#;v4Xbh(RTi05J)>LjZO zvcd+8;3fCQMH95}(n~rz+IkKtcKstxKG?$wm&Mle{J@zBnJ+#Wd>f{(rP7>g>)G<| z7l7@X-J1JHI7=zS!l(>ul+OMuipdrvl)FURua*%>3u_T{1_voH2&yh5#tvq+oQ3C% zW+F2r?=pb62to3J$k$ok$CUC-Yv%mR+JCFPY~-c@CeVneR6>9~>$^+{9E z!4u!Fr*oR_s~MDt5B0(1^Dj|ptK|?j)pN|p`1%Xnz z5FD5RWs%zL0=zDY74&G*>)B=F_nYi<*BUaS`-_l>&|abo9sPw9T*{23Phso;V#%S; z{ac0a!#D0Km%@L#irtOc9X)#fbwbOhy_Q{eLe6Shsdv+UW5m_|E?wQmzP0nKt2*37 zNwqKHPj3-VW>>d$Y)Jd-H=eV;suw$Q2jJ81-Rb(h@_oef`tp6BRFsoD$-y}%L{6!S z;S7{1R?+^;%FQ)HuFJ*kW&y94>NEG)Z@|OuS=v%5-quGiLhH{Jib_%^2bJ(a;)R_k z+NJd692dgX(Jv8X*2|KgAMQGOIvn#X0eAanuRKo4eU`Ob6O}*oO)xgr$WE5%iaklH zo{F796{WrCRqdX7()!>A;qDX1Ni*G=dWv#|D_(|wadmj!$+1K!HmbB-?>9mGX1hlL zqEPYbm24$JY-%F~nBnm>+k8vOe*Cw6ont~U+)Q5!KrSYSHylSZqMwBxY}9e|iEC#w ztV)4=D2J4PwlzRFxG?xqglwuIt@CGUpuzn2yxz!fBagm$#SkXix(*sgp4ZIoEHP*_ zJo~rbtzr{euAQM~ZiZ`F{uXY<9nXF2|K4Rm5cz*WBhSOH32y$(+D+2g<2^3}X)l}) z-maZ}5j`;BlxeYM+)nrCCYjg1zdc=P^AKpgD(*B?^u%poX*vIjZIEm1uVH^g^|Jn9 z8h0+J@$rN3@ZXhI{`x@a|2)dh_NEf&7MRnZDf>q`?ea=L4#puiBQMnI#I-l-423@? z++I|UYHo{;3c9Tky8N+)leiQzK({@u1LU!>jX&2<%LXjS3XmheRoiFVZ?v$UB<0eD z02xwIra49yrGWS{WMAr*tIbp~=)Iul6HoHa17^k?xnzGgIk@pE#6R$A#1#*POWd-N z5I=|*enTAqO_#2{MMB2Fwa53Z;{CZQ6}5G6W;Ogn+zZvq1Nk1dp-KB9?D}kuEdZ!z zJOej4-q*#XX&C;5!;njHWf2#=(~cK*KCG)`Es=Y`(wFI#J@H4WZ^Td$hZG=xxC`d<@+z_A+mB7 zVgqUQnrHg0#5-tw8zSo#kJ85Bt)8r9QsQrNs91eK%2hn<~EBRd9)d!NlYTg+lNlw2rc8={& z%tD=D8@y`*#4X;GzrR8wt>cxyKpPeK9Cqta3+T<@b-5mf$q4_`$z=JWIjWgcYoVIe zLL3opJ0a(p=v&A4^^&E(gW2|$0sfQNGN`Tg!GFOb3C+^QLOy83#=chZ*e(S=%dOJ0 z^^WlF$9bavyN(ixCmKKMm>M-rFc~)x-dhf~KW^u>EL*EgXy`iMW%Jo~N;;1?Uv}3} zsJ#}g+An?5!%E!tPkW7K`O^0d<-QL8VXs{A<)$Fg` z_YNwZKjbc20*L%ucDvF)*Jlw!^Nodtb{z~H4e|H-^Vk`0WudLe1U%-iK>U}f+m|3# zYHQVI41ho?D&!l|l-=7@tXb%P>YQ&s&*S60{J5d+!aYRjTi)5Cb?pEBSB#ZzW>8}| zMAZod!mfH*DKd?n__oPMqw5Ag=Fax?tgXz(rWmD_SBa=@+QJrk8zt`w@{4VE;Sq+* zFg+)!86#Wf?oK5IJC=^CRSj;=1ud|$(Wv?W&7iXV+{B8{*dQ;H4hY z&5vf@qit?pp|&(d^f>yAOg==lZu>5II>|%7-77(4$gq!sQJ&k)gYn|6Si+ZD4jX>IjPFtaLx$mEei?V# z=fnl%Wu;Ny#4P&(8xXEVJPEp zL4qUK6l#oVlYfm@~aepE>o<*pDHeGneCoUsFuXg=XuozcKf7{{szD9S_rj#n#YBU^k1VtLMh5H+l1< zH+hkthUQLL&<2US|ABN(kR6db-w>M44l;~OYdKMwQk4!Z5MZOuR2vt;d$Hfr6xMW*~(++X=cZt8MOW?D1^39-EOZdZrR-#5f3%H`)L?}VdfuL*Ys)z_^N;`z{ z*ji?3w9T<@F~`N4z|15M9QA-_sFg=s3xp^_6MRvxnPm&j*S*)y7aNOJy-uWyOg1LX zcs0}mZY8Uv>J1E0Y$ov~CqN^DyLi3?f2t_N&T>3>Ye5IlAmR5KY4MFi-5qdx%d5SS z%lITLOSSeX-`O*aN$hR8T$UQny^6W!5&^IdKzcAxEHiqLvwF*yM+sQqDQwG|SbgDe z_|6|8n2p~KCN|pPVV~k7a=*zpGI zwXEEWPuKa2kOZ!y-2JJUBefvOM+(e-c|>dW^Dvtq&LZEJh?BTi5|j{&@HW8>A(JX1 zAqN`9LLeU85Gk1#p2T$Xw@V5+(Yl$H4O(}q- zg^3K^Ogqo)hk0mNvT3+6{KKg{MAnugN=XZ7fGND*`eZc3i`Oth3nFJoGpElVELRBv zL;4dqFMbyZi{tIAR=0wUt%k*&{vzbavrW4w^~I30dC>KNVGZpC#X2|7Uzv7?1iX>9 zY3}2Eip1`_)HkXZpRN|z2McwO60A#P@{(+dsax%{OjLqsaczez+0GK0n?xk`qAD}w zQU~QK9L%9HYA9K45(E-?zwpeR*Pc#Y6?EU&$Sx{qld8eUhfY=gW$l=#y8!8#x2y)m zRcmE_%1>_h4(*=%l(N$I#o&V>&$wrfuf1I-n9uqYwPbsqFDxRZ)4EX(_Ad@`QVWcw z_^RU^zG!Ca7T{K2*;@2_`x5-aW<}%^gTS#(W8SitY+fLc83)Sm9lWG!K?0g_g;#2y zvSi~LV^c7Al*b zKC^}S^N|s`jKeg2G#b8KseQ9_*9&ld5R+IHtrHbe#AZM)RG~)sF}gE&?LjWKapHOU z#a9*RTPP5ak9*m|4)m$Z{EJYK;SLS3<+r}##l)&$5Rg0xk(H7OF!DW}dE~Ss?wBU^ zIb|-fdmD0}RS|0g?`MZUCLW80WmGoV5~EUGvVKAzOozcWRFn80pZ%2nRbMN|qHC>V zEdM4sQ!OcMW64OAh&3Wl$7Ed(JU>IbXn(qk8PjNUxIYYy$xvR~`q=Uz>E8$U_h?_I zG5HAgwLMBd)OVm@-^Msi%uT1=7+zh>}2n4+$v zFAL56zKze@lZqtNqrGJ&2k(|^c+_*dr{V)@bj)nXMIgTlL!yZdFS{~7!&Y`>I@rUW z5NsKtMZ6D}^xJ7cRsKJ1yH9tsxbU76h90^s)w$3gY!xy+B_0~az z+}+cAk#Ki3z+h48wfD!ifZ)Yw)UD{(Z6iHf5%I4A4r3i=T)mb=M&-Hw~Ca=Sp zLj$+$WsdVpA9k$I?H498FBVV@l1DEdOg|zlj}zglHXQH*+AWJz8`Tq%;Fzf#Xt=sZ z@2T{=#p;2xWMp~rRaQi)laimIQLVty9yXU!YrGs5QV#SKo;|hIkFF%T%2Nd$vLW+W~rflpJbDhAyoHA^G>R0f^ zflOB>Gc`zN?$9cKMaAz$G426N?xK(1J?xckPI=ty$?M5jC!W#HEBDPGk~qg<&^8_U z`U)W*EsOk+>4F0qsQ#^?|2B5xFl_XCQ_rQ+nJ$DR7gC>VE%>_z6~0?1V--cYmh6$4 zg1+M&vunX9y=D&ML5mcKJR4!$9|*0A*g7lh^}&7Y36JBeuk_io8RQfe=3 zrlAn}fJgv;X}2j*M49@}&tQ$SuN4^CW=oPb=@?)vslI3T+`N5%5LdcGW^&VF(FGdX zDvmsVvbpjF9IrWuKbjNrr+s=USrO%&E9aFRMEDs}*GJu3@S_tH#9+&KN-Pqhdy;3t zsO(dGa3$X*Wz}iN+C@ zshH84aH74zJo2ig;0xxbJZ$Rya#x30wsiaeUNMc|XmE@cYpfNHhX5a=MH?F0pDG*z z)%^dkAO3sXz&11K*|Ays>W-7}RdNjnH}~dyuP~`DkK2cs&FI_CXyFkhZPhl3rH3;0 z+`a%Rwq150?Au|}9QR*@rf2tPSEPPl7`)Ubgf9e*9n!dz@XpJ3rbDAuLIavM(GNRa z4JAs^Qp>Stf5u^T(5wGSJ`{=gH?43GlNWOfF!GXz+v06Fr>_9u)*qKGHjg3cIb5jq z1WHINsH+(rc@kL}csuks~s=){m zm&{A7W*b>hOK)0;nWa)G3xaiEL))S>f&kbR=SjVDp7p{)5@VP!Y6HcoFexXJ=a+8e zB_u~z*(OX4CERaYx0R*rB|91kpzzIesIFSVYY!Gib3=|~&3EOU^8&w4Rue7)NxlM# zSt9;q&_vEr?tS@u$5&FAQL67Jd#2zD(2}MgWp1gW&C+}oFq>fMt&;r3R4LKY=7sM< zjgGYAA_`2@-S7|NApHLvy0ssNVfstp#fx$o{$(Fv;EYD03vSzBosHt2S!WIjEQ$5T z*%K%S+L)wzJ}q0s1gpF$n1%Xkv`|WEtF%7(=I{qMJrv7(pc@rRI^;c59Osij`{#pp z-cprM9LTde3*l+u=&$}e6?amOp&#lX-yn!*sk22OQ?M<5@}n3X0NiL+I02M8_UI-o8NGvK93x6^S%?TgiBO?yR^9umX~RL_1RJAB z9-pk0U59H% zwrGo<%~HFX5WlNg2UzHPFe_JlOF+^WpR95^{bA!n&mjFw?5}kT0hw%i*bI(`KOC4v zxvxG7QUYPx`BC|Wx`P1$NX?v>Zt}}5UKbvVo*-R$^}>6}?MuO-N%Ntp<*@FyWxI`s z8};njA0K3!zq|ZH=R;F`x>E zyJW=mb?L#_kljTwhu-_KtdUs(J=r(jDDd91b68v&_I+}l9512QIov*J3OHGry3&>Z?|pt( zgEei-!?kRZqw&Wmkoj$Q;>Ir>?QG|njRj3OkC#8*gkiIV<>4T;t{>+4;Q8iBdB_q+ zNOk#Q4Y#;pVap%A?#)q01(ylA0wettIs5a}_jUXgIm>Q$eqis%6rpuH#C(5q-T<~g z{iV0`{BI5M01xS1KAa#^*Sz@)slh;O&F6=^%- z(*N9i%2n9QO9svGL_YB?p3N}1%!tXtw7|7A zdULtZOT2wb3C1m4BS?zl_qOj!`;BnYHk4;DWWcN@0=B9X7L%F$O?5r^+9pTTy1E@< za8u!Fr>Ww8qqdyifgEA$Nb0_fuCrP6QY*HMc=Z%vrWEIO1;xHA4wAG%5*X~_4^?s) zo#UOyL<)CH3dPl9pu(XMw@2m+DV6mwlyp1M7@RQ`a?cTEY;cM8H#E+oDCXnuBv;iO zE2~9_=et;3RwFHUD&iOj&xQgdh7-4tynL*590#A{wnUHHYqmmO7)s9|ADBoI)5QQn zhkz;nzX(C(zr?w)vFdUOC#1_>bh!=)7O5m6b?6dq{lv3c$xH1Y<)pM=Th%RX@F-k$ z|KsISRpM0%EmnM4!io14HjVn0{O7)A&wbgXq!a_kDooaXYCY9>^9p2;l&f7RK)lf? zTp{G;W2Zg1Q)F4@xui?*4x(g_b+pPE)v0K3D~0khzrR94R@AR1VeV@5*#WXb!`=&KLFlHu17p?zuJ3k@2Ox zAx|ccdt-XhvH-_yj`5oBP$aaV=^rLD^6jfzBEe6w+w@{W$Zd6Ony|%F8 ziKYHUsKj_b9uvlTaSnmanP2|B%gHiBiWD`sa(n2qY6v% z%lmSz@bh!e4@kKpx)(YdU$#sA;7 ze;w}8I=`UT<7|<>;8<&NMj+)=m{+_tLG`@0Ur~O*2n7$kqL9ZEX}Ohn47k3w_RM1Fj^q-c}mN!fuY=NsoS}hS`%HG3&@&i!HliZ!N5_tK-dh$toZ! zK6hXAMYZoC6(&}(%PZt|@86V^t-6n>-W}KH>RfuOos=3PE0?W*_OQ}M zv8#fcGl2`}`;6n>sSx)YF9zlgMMgiKgjnTrg0Rr1SBN#_vtqnkgZmm!0$!6f4hxpB ze%3BlaVW;jIc&&t+3;1KZv4?)B2BkmVwIRhp6S>NU;xTO4XTUs%FUEuW0<97_e`zw zBd>D}FkEIRJ=a6nw-gxDO?VTfDtu+L~tTmgy(S3Q$w;Qr`u4K4iI ziz~?f@9#|^Z9tzQaPfjyfhIl$>gL6tFPl*h@^^&|XJ4Jtl$0x;*5z|}kj&4L?3ZcA zA>v@Cr)^iHt9gi_qi-^xs`?^Dl+}PlQ4l60o&#ykBfuK296ur8z*8nf$B?Fp8;B6} zxU+98cpZaF{$!Hp2LM7T`Sudfa2QS)TjG{v@wVxt)*=x#1PPPA@nMq%6C9Vd5yB|N z8u7K&RsHj6Y<>76e^1vo>KbYOOTmrhZCKHjkV@M5Gj)R9*h!Yb!KR|ZT_k_y&3;$L zV&_!Ztx?N7nQNlpmtaIJK1q5@?r$9>3E1Bh2?lNK_VkE-1FJ8Qc2(PG58xV_OO=T&2#j48hZ zec%(=5pNBXuOn5K?4GT}3}jP?yGb9Ol_X_j^?Go4ZtatidJQx+qU#F=B@u|;73=!J zklBWbZ?UJ*wopcv&7k}*dwQ41LLU#ej|oPJ;?7dZ_xG?4EZ4zZRvB7Je-V27_@4yV zYDVn~7JU5Gi#>r>Xv_}8O;1Wyl@5@dW+m&4io|^l005{HEa|N11I=Zy|7{j`eMl}3 zWx$;dcv>v0oY7XO-hFNKi=1zce^JP(N4Rsfl5JY%+L8LW%;6#!KF49nLK(F3^NfjL zy4li`_61P~PUXi+ybe-ywcyjFEzjkvaDV3ZE0uruhSy)tX}?AKeBY`!J!O!Ir&)UuE|Maf6s}8 z$-5l~DEAPLwG<+UvYjZ)mJky*vjxQ_WAfRg_!jc*E8t?K?ASuxOfFpOkyK%6+q0|i zt@}l_6H`++*@GmxE?%L>q{P@8mycI|yIBC>OY)aW_uura&xSa z_7h>|T|JK$FreZk4g6J-GRwSj)|SQ4QIDZW+)G=N&Nl1xNDFn(28X|b#?8{>IFWHi zz3B^t`d$VKv`Tj;Ku0SXAnA)lOFR7vB5w!7k#Z_^N@VoeSX-b<8WeA@l)_HA=U6NV zG-wUt=LRbUC*@jX8)dk7j`D-%05^BB7Xjp2jUsylRhaXg;(Me?R}9m`x(Cyh%cRvy z$3msxsSYfFkCXH}BkZg+l2}Wztrr^HeNP8(@Di$B;C<`UW0QS4Nh@ z7t7TqqV2lYnOxhULJsW&v}lNAe9V|$LPC_u!adeh))YQ1$=blCqS8@Zn|a;PS5N}K zZc)DH+z_E3KyKlA5&`qEVv}r-sANcwy`r)(lyofh+GgE&22P_#x9Q31uCnXs5cc3Y z@>f59ILm3BEMNZ_-Y@yUH3lX7?f&D4NXGuy!T{Kk{dbPc;IZ~UGJX{C!F#R0@)r5f z(e_!?4(P7^^LsmCi(5OL&p-~~_Eo;#-Vu@OK21tNm$}HbH`s1_s|D7Lx+M3@4mRBS z`Y*J=BCD3+daKO8C@!V9*9uDh!&s9%9R)mcZ;UspKX!wTO#6nG6_b-|wsM!&p z?0|tD0qrl8aj*?mn8nb4EE(Xb&6g>)a^Oxcb?*5fM_wVWq5De-#TekF@RNU`&FXsd z(;MGhKhF$TS=3#fz^AWx!r+*P*IQ&d|;Z4zP zQc)b}&=34O4oN&Q%uAL#0QPde_B)+EVh;~+r3w*TItr^+M6jKAy`sm9>Vp&}f<64o zp`{tDr+f24odWeHWwWVl`xu;86%4nac1_{gd~VVQste-52}NTsPb+YCXSGewZl2G;Q&*kL1?3=B*d-tW{K*|mAA|JZ6ZxqnkRl+y;jr$$! zVGto!wKfbp3r4>cDKr-9u@lAzamnT2&zZo`hSZ{v?(xdp&u!hYa_OY7M{g{j$EsZziDr=m|p z1F!W1y4m~^T!OBJqezgpg->hh^G-h6ws)xWFGVJ9T)BHFkZ$(TPok~`g>ykuV~-ZD z@S4I7qIcHUgNz&`X=wX1s$C^n>2l=m>-IXxsx~TBpG)I_->oPSnR-67ErSZu;gXCt ztVZb|tBjGygy-66iIZvmlbIW-0^awH0u*w*$=qM#?X@qJ)B^q@)gxP@pCv! z&)lEKJXYhho1=4)kck6zhTh;7pj2Sj;5xq#U-gp7=5F^Z^|d*Kuhm?`Q*)$22V^}E zJzh}7K537m6RL9i8t%@&)qR-gO1{FZ8xWe1?rx}g0}UD$aWwt#u5M!Xt0*hv4VOxN zOA=Ox#8hxhi8};X%^K1r)k-BD$jV3EPsv3;BH~yNSfStSU9}2{0FV4iH!Tf@lyzBk zilLRCFqpHfC)YrbB$7VoQC~T#L6BphyU|l+^6Db=Z_DymhOK)JB5Xu9M*K-iWtyd$ zS*nwPyOFEPe0Tww)nLLF`6AEOq);l>>ej-t7ysfgkCes#AHmD5JEmQOGhgIgfYX8f zG8b{^wN4=i!9abMx-lpL7Q>d|{fL>&ZdN-M9TRuH!*xlfg~gUlD5BYr2TRNVy&1G_ zS<*dzb{6|_4YIDRrzI8Vl^~K;$T3Mp#oA#B07ZG#g?3|+R~dT z^i@=Jt9-SeDd^zsA>wz^Kav^JY@WJkCv!Y58bIdqyL=LO$;;KSc{ryH;N_mX9r7I2 z!||xd%k&dk@&Q99a5Sluye2D|b;s#45Ps8ehm_~R$OZnL-US%&h4gw)bM~t#L*+7z z*(nJuRsta=dLQgPooH+A$KooEsg>3^HQln5{Hm$-Kmq-lX!(6UQ^WYSz%keSG1y06 zYvppautm2Mr`)9MrWc0z)v}F?rynlToW+r8bJ%zlE_H$JIJYY_k)!LJoQdiNqQ15* zQaZtUb1Q*QTxAlpF3VK9<0GIRX~-M}=hU%$7*Up2^yvTTa#?#jG=e$*87o$C4_aNM zDZC=bEbOLCsIg`5zhOl}XY0EnEL~>b%)lm@CCq+zNLk3w$YSNz<*p~*Hg9^{5LYCi zo^5~ALtZO80cug`$s^^?_VsE(7E;eoP9+((iPntZaDWBoacPF_j&GpKdw2~{4&;h=UVT4#~8|m8#kyvoJ1BV zr;eL_aC1Udp50CBV*P{o0RCe$4+XhZfSpB~6EK+%Xb$;CsX-ksT<6gxy85$!)$#5H zstdggR)W{ zY$MXAp+&y8vYlFY6~4ZZXz?QTb$$OGCAiR^1T@xJ(0+dHv31%9-F@DnDB=_8gi@BD z&&>@_qG`&<3rAXL=~JuJQb{`K_pcG}Nc*zHC)kuJ6JA%Gdf0OrAdrMAr;|&sj|tj@)Drfa8C2W!@vYPn5?Vg3HY5RsdQCuM(ne$-UG_!eYUJ!3vloSp zll`Gjk#asvawD`*tsJY`qZEN&4TmjzWJ8BgVJHbx%8I_2a*1VJ?Y;`u*u7l{;@NRR zU9*cP0wkX^2Pde9NCjX{SX~=XB8yGqaQ0LQDSmF$LrTsks(pa3d1-~;S_gY%f{3^i&&%V$e1rq8Q|K{PhhO*z1I!Y##6JW4KIo|a2a}CP;isbZ znpU^K2bOzEvO0m&)C<57P3MaeWCLxRAOy_PuGAu&^J7weAuv=1Te^2y24(X}+=s~@ z&7BR5GiomNkw-;d&VI+o@5eUQ>pF!xp1`tGl$PT%yMYBp2+Oq#wy1JO*$%FZqhMP+ zNaSB20DxjZH+2`Y>~oO1BM08-CwxtjjtjC_?vQ9YF?&(uku&LrA2a7zG+Z&dh^UnM^Y+jjyTd8LO0$s)S*^+bXC>mlgy2)oZglz>7o*DYnp5E8v`)xYNP;} z16%X&GId3`ls{|8hbU{N_w1xCP0PwU#T@T8UW|Fwfjz zxQGATqlTjm?wMT!Vop=-iekEos79-lQ`KAne5@#wg+PhPnQNrz2_D%CDcM5hO|$M0 z(F2oBIe_!s>r&$RL!B6m+dQqsVz(!30pW@P4EgKhZ+Gvli-`-BnWcx>D?OSEZH3|T z53|g@$3U%cXx%+u3H-5I0*8^Kov0W-1C$QS@p*HVBcpRe#!gctCJ{)kgJKQTnczP1 z;4QBZ@m$2F$_Ir>DO1ZxwNW$xsS}W4Qiqjc28;g8P4f`bJg@eKT}DUPFwQ~*EvbCb zV?KKBn&(vnaYJqt2~BVk=U{VSUw6g(z@#984&`7gTeFqS3z#px=Y}P0I6muU2}N8E zDjY4)<_3lkvw^81fa*oztT~BYYxYFWr|#S3Q$RYFz)B2zyM&kpdojT9GKY3+C-sl&f$oSfF&BP8m{JMMr%7GQo7 zHX4<#%v#Reo|{~reP&f+Y%8?n;3pW%JRE`>{_0t_M19>2-^{Y2(BydeaFL>{2#PO zt+e&wC#EXnm8$!qU89IYGG2PnE9LZ~coVRnb`KhoDI#`_H*)@V_^(l_Heg}ggw^## zH^+%3BZ_zFm9sHW?_0Fw1Sy6o``w$pXCD{UGml=_ukNL<`NiV-9Wi~&l2h5Y{k z?>p-#-YitJnh#4BulJ24yO3nAp$}4wy2p7DkbvmbnzeJHZivoS7dgQJJXO9hvBHG^ z-b?O6MZ(=9E>WX(!Ha?zQP=0wkCSfFnFI`wzHIJ^>_YNM)fwecOY(iWfxesh*j(3? z42%Ym=?o#m7i$cOAdo( z2WoD3>~e|CoLRIVly5${S-e%Sz-oli#-tDkuI_m!(~n737pNmETpBB{nN`xaY6|k- zt?;zDHyTvnuHIJ!FYOG3cuzWAQ|+hWdi{qi+c7_Zo!ai()JKrgzY+kvNDi(1kn?$% z?myhdy%*HcP4a!SmvqYNCetB_(jE+5Q@N@?1Nw_Y?BGk^yyM!pj)NI3?7>RfgSj zHNcuqK)?|aZ4R6Q)^v#vHR|0nlm?eXt`epF+8l5h1o5LJF1dkqOpVnuOYU+ve|@~~ z%J`ksf?=6v)invSAmM&#ZpalhZYP$r?LG}BEW-kV=Au^bKk8M|!tTuM9ed{B5S5?LSo(_NV@1@I%`G(Zm1N(6lsKW9&mZ&o18SS3KmrzcRKgCt*;MmN)n?86;nwHu3b%-na1& z8vN__Y}BVUMM_PBQAh1vO#H@X%Y{(=jnW|)Ha(Hu$8(aO!>u36=J}CXgkrMOPyG|W zuL8{RJscfHzRg8^;##6uNIreE2~rtUq144fgr@EOb7RKVbTU(oDbt~JL%!B z_w;g&cWm>gR;l{_<8o=wT! z8i6t*emcha))zP6_yZ34Y=un7n4dPgr+|A1GuFsJCWmv`s*Qjwa?0%H5?k5%i_K#D z-_ECRqZR)I4wn1Wqo_)7nCm#0*(rtA%2hc~>e@a)zoB-h9OF2Og03X`odkf z^A14fKpjYSOL)9E1if(IQUsScFgT%u994QAIb)p6|a-Q*P>MS4$F z7@RCBeSbdmh z`?yp1puEOQ#p|P($1j8D(x(E`u59$pJal6JVq0D-Uc)niZtR}MFO)wlqxP-9IhJ1j z!IjmKS#(-la(~QBE>C6PnID2@zI=tajr>)0tGeb27jJ#3*~Y|3f6X&i^A&X+Ngb)d z>F`;Hxl?TCvm&;n*yFY}bi?m&1z(W^-2B?f-eJ(}yI}?#gc^(b7D_AS!4J90s`95& z88)~5Q#^bWomn`ewhE4QSOtITXy@u#Xr&>pIuMg?9-mOvbKG?Rrz=NR*H1I z?B=cSi-Xd%Y6_x9AL+sz=`KsXNq}w4Ljhh{ybBH z&wzW{5ZZJwlN#vf;Cy}U3OVnA_@q|sN%3-}cWjMS=NAvmqqV3@eU5nwtsm9Nu~UcQ zDQnNM$mX3sO=@}ITA9ek%X5w9;8@7_(U$f8S!Pw)Zz8QSo~VM*ITt#ys{`rI+JmBn z$=a_VZ3wq^(x@t5E>b}j$u!pfUIX}$3L)s0mv!RNwaHxWwuQXM*xkxAg!Ut!CWxdl zvK-@&^@Gv6EQ74yR;FYYU^S>=^ss{bJh*_-`SpugV&|)I(z#S=TT>Zoo7U!t;w`Md zEf$>O7cs9{rZEt6<9yYIvn5&1xo^mU*lnfO(%6FocjGchpc zliFMsZfpsX1#=1OEM+LB2fNRmShbBj-FD9ny$VfBvy4K!9^@L4i6@N%m~Jh8l?Zuo z`g%JoJ+jJ{tWgj+H%OP6>5C05(Eecphy$AM1%?LT_cc;rrN;q6_0lsj)5 zBr$YIQs0M9$?p9S-u4Q34n!O1%h0IvscYdY3dm$aOOS&RnfC8gn5iXf{B03FUs)gV zRblfYH*MOYUyv2koH~c|F7xz&_|A(qudwsS4NO`#UG?~A72)P_hhlue@YZ~tm;Q7F z&!D=m0JXCIaMS&Jc+{qX$3TlZ+&@PwBuhe*#;G*kyQ{P`>&*ED_EAgVVfr~Iy~KKi zy@3@K(Q2hI2{xjM9phZv#?cOBI|{Y<_hr{9wf8GMRtSKJ2_PEqFM?O??=B{nJch*@ zO>^**;niLSN%D&&WFa~x3I6JpeAltPkmoL!GeUJ=B$ELY|%U-p^J%Hmu@?Wp`&u&~GG;Y#f4#Ewg z&V(lUw)!86zag6>^qR(58b6yaaxx%rUG&e;YIuycPdcXwR;}&GzQNfMECa8|uiE-% zhR=Scy?Ns5ICt#}X=N_ABhB|jkJof4sx&E(lP@)5n3H&sp_2Hm&u3=Uv!DYAM zY!_HpFfD#+GzQ*gDlx_p-Z`(j)YpC&|D1RF_-yo_)JBG>D0ia2bf+h!%~s><#WtC2 zm7ag(>8-wcc4PS2=d($Su#L8sW98(sFS@rGk!MMlq2d+BBQ*#IFC1oV&&E%@mOQ-P z9=ki>U^11ZvYuyZ4@DxulVg2{H;Ru=a%n4gdg|lmKA) lcbEA;GXH-HpV6n@)cL1hXS6t&g{q)Yi)fgOd`y{|`NeJTrCjZ8>%F%qTZ&z# z!?>|9=Oz+o5H^w$++cHC;@ny{ba6Og_(8^u%#Up_T>>s7OqA}tx9u#gTll#jJ!yN+ zd7g9LbKcXF3q=SK#X^}SSCd5`5Ug2?CXM0EEcOwK%wWytQZS5cX$ggqwlxDA;C&RK zrd7ZrBoSYj@(B2&B*52>B7+<$oOV{=$eqA!5q63-!ynhg)v!QaKM}!zI=)C>nAT^A zTC$a;7!pi^rJj7R7h1)@#1hKL`BHha)#S5%TB2YXQ!#i^JnK2h%Op`ATuE^>=CR-& zBdwf~@Y-FxpfiGP;00cst{2}3-vaIq%j0`VQ4|L7+(FW5p5n0~v)k>7pu<4NXk$?v z1_ACx;{cIPiIRhoz+h1vF9>1^*!Wl(B!NOoD6hu`2*$mfs0rX+5*g12Fa&9NpFVDH4p_PN3wWZGQ3v+6QNR?!+n)b zbsQ<+RpUARF7Ws2DSauDgyZk5ZVY8djVv#JT`uJ9_>YmnfW~s^vBW^7*^aDrg`j8p zR}?obixU`e8B<74h$=Y5sI#)fKY?=`yE(kQ@j-#vX4pNR#!rz%2WixS4qojne3}?x zpz(eMN952Z4sRDBa655CoVO5|qDas)qE4U~9zQ*&2Jmw6p{aCGWErqA{%xkI)fs@z zap>89y#h;{@Uufcx0s{FB^Qf@K0*@+O&~OZ&;&vg@SlLJHuPu0e;y$_G=cxq1f-2D z`~#>WWZDv?DsN<2q*R?u>EcaGD$t(0!*ZtY($s>qhTQYm5n3ho7 zg!Y80ySuJrUTf;lb^$JpsryygH%WE)vr#W9O1F%H7i@}k>eZg{^I`5Fl%!FJ+ID%I;1Y=?W+X|w<~O}0J>SGtwku} z!)enF*n0YZ4q*M=@VfZLAD_NmGt$?h_2A2#We-P&yZ_pIVZe(l>g5Jc@%MM_QRUsR zOWeb!5A_n|h*EVOZhIJ(zI=aq*%5ty z@Wfi{S0&vINl~tw?!gGBV)J76z^hlAhHO`UTHW>5L|0fu`TZ5qNw)PhbN`$?)T$U- zcCT~$Qum5hXL--J0u$ZdzHNVB^SP(KXz%`{v2I3orL+IR#uH1XwNxRt$*4q%w&;J?5w|@x8ZeWYjM(<`aZ=+DJ{1;nZ`5Mj+b$|_T-8K-HGy|Nn4LK z7=Exl_s-FrJEq!gs#9m}#>LOaW#4grbENS1m-pZLen;hjs_U82ipsiutB5v5-lg!g VE7K;Ya97u^%vV=SyO3RG{~N;JA|U_( diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testPDF.pdf b/solr/contrib/morphlines-core/src/test-files/test-documents/testPDF.pdf deleted file mode 100644 index 1f1bcff6fe90df04656b43f98640d863a65f8660..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 34824 zcmagkQYu>>MEmE=$n?*>f36t*SC`5j_9g^$@&;!AIy#!zIuo${yL&|wCp#BMBNHbA zrhium+1Wb(z3)Wu&yfF)PsYU9!a&f@9ZHM-Zv_hn3jsSbqs~7={7*ms{O-R~lXtW; zQZjKS(E8h%uqc5J0iC*>p{0qDGr>O-7qwv^VEI>6Uf%x|;Ge4h2KdjjVgFYvx_=M* zzai+9TnwH6H4#xeN1MOl{vQ9;f`Nce@b3r-{y!sxqWhnzvHm;U|Laj#`)Ju|izOa= zeMfpa179zA^iF23U;sl))0yf6J}}ZDiD)&ED9R{a-F?4*^;9{HaVBU~RcK(+Log(# zwuu=FXZhiLAKbbttoTo!-L4qJ z?D=w>zVA$A4^Tdh_F~bdR`YH8aD3Lffft_i9{)`2ZKV$G_%bGeJ$CBB?R?#)4tDNn zrZHol=uC^Ktlsoo14d^v$Hw0~)Zc7F`?GsoNk=pFZRssi_%A6zQ3TTid#~5(527Sd zY|j9^Q2BELvGZ79XFDu`(Q{6~Hy!EGbLtO$34I0FAYno!E3v>24W>ufQ}yDT3jZ zzQB}1DY~O79BGkxT~1%ZO>)Z5$1Qof^>Cc)7sWwa_Q~;F{vj+@7 z4a_tt6HzI+8ra3YV6|TY0D;<_LHo(h^Wo#JB&@P6zz1qzZbKAR3EmKkPzI5(UH?;I zG7eWrVvL@KBmi-^+Z;4UFQk94P(}mP2{@!bYV@=NaTYH&IYiPt!jit0m9=);vxGJi za1z!43ov&SDpZZd7iD)62n4&ay#}oDOpuL$LMJ(dz%@Y46I_O4RnUF%!MHroLO*+* z&|Jx{u5*ZZvP`LZL3cG0(^BK73NSKA1b=aJfM&5nAOI8F{HZU9gm_6t7xnPB$~KZi zQjD90a5Lb;kz*CVu1GCUzB2^1r&|K_aAXw^tg|?T$z;NIAx$uHr;Dg*GBfT{{l$nq zS%Q7pevEqX7(>c2n%PFmuNAj*B9;_;G2}dd&rTX=r5OBpAJ-)_j_YTsQRF&KeF+Bw zU9D~p0(!l@T<)IlVkrSEx`BMUS)8$=yp&(y6fUj>b6d2$PasT~;(P)q0+ApTO5^oe zTA<{iff7`~s9APMDE`kETRjUrfW;v%K*BZzkRvlkb%DebL9UU!!Op)(fNgRg>bpch zn*UTTBhubAWS&VBjo7tj-q>VA;awU@1@T4TU@*brYGa|)t`tnp9?3@Hy0eO;F9o@!>N^8+;jM;;THg^8Ku)%BaC%M&8_lKF*Ee^Wpn$UqCkt9=xz$ zeIh?PfVXL2q3fm8@Bw)i5eI6l@P4<^#(IUt$Gy0|yWeyKCt_==cbbeI%$UK4?`R1l z=q=ufrM*AA0N%EEe|f;yKC@L^(DBt`-^z~Q-W;BA1!#`)6~8I4_<4nf!lf8+tlpip zmYw!lOvLqfIoJW%O+1*nfPK5K9K1iwT!7K!F8e!DFME7j$>Z%K#_e0HIpl0Tz($tL zsN*Ja0?|FY1o?|W(1i9m>Dn^XVT0@=m^Cb&LFA)BcM2F^Ch8XrHjEg+1BOkKz|u7F zFNRC8Ac!(8mPMM?feDUCaAEX-?vH{Asz3y{%Ig~Nj3%iM-gRK=zQf#+6L4E|WSxQ< z)B+LoIV{*arIK8Of*A7YP#e35rfq`AI*Rhw>8f5Q<~KXhzJcJ9K>*9TuzGf1!hf*r z0_ex+ccfNBG}1qe6{GUYFAoSshitFF+-V33V<=&yfg1QhL0|yG7k2ADi|_{nkMamXlx7gfCA*6uTR(V&0sz?AE@OlbX*En_G=d0kV8DYG?&VoU(W z*=*x?10u3F3aB+uL%>8j8cVC{mxc}xk1WW{T9L4;^j`0ju9s%uN>zhE#0Wq(4Da9= z-y1qov*S2`S!62c#kJK$?R?`zP=<#u=7{6PaKqZ4pxo%^Y@G7sgY_<$1_XY#0XXv#o+=^Tx&N4 z5_9$iQ%$>i^|+%HqJDB{D@jTiO>=niWJ=`9YG0W@V#Eo`(T;DlQ?}JIZdMO9sCR(n z`{R>)Mc>m$AF4j`8DbBv$=Mmx`~0GyUlXCB<_20VdmsTz*LPUP*>o!>Lpbm~ZT+EM!PweIU9rLpIdV*-=!Eel)Ag$}=l z|IK__oQ;PqSdt_rrYUIfQYYGAGXL?~kaD^vL$n6yrN$-jP=b5O4V~%^`fc{<)SreL7*;w|-UR4IT8@+fuy zN2zx7S{CaYD^NE4#2f|#Aq1Ya?W^bpg5^&*Fss{Y`Q{&O2=D;r)Xsqok8CLj;|<`g zn{;TvV`ag5Bc90@wk-F96Vu$MuglYxSL|%!p58a4t}6}+x9-t4W5+>MIV55sy^G|f zS-f`?OG)p`55VA_%c`~anwXhM&qM3cQ1l+Qju7!dKfBUEAbGHC<;m4(SEf-MvGKZDz7U>`^v1(Cq>_<3F4TQV_@7!XQA+US?Z z^!lFZ@9Vdv#&E#fHQkS@Tw|tCh4dFY?aFLHfO-TrnUL`Qm@w=BupG6~k+4cK{iI7A z@@zqfOi8X2wsA-xL}mhPE(ZZwd)Yx;{j=nr40V+fD}R?gg~bE$^=A>H z1*3eYZNz>$)VkZv=fCX14NhK8LXQXHu${Yeg>fZ%py;K`p?Mx=HTf1S%j+S?3%_IPgSYHp6j zYnD1;?nhf#|C1=VKC;)Hl^uBSU?f+MZ=@+H2pLHs)^dTS7uYoiMYB>sQh(&+fnqf{#dK#}RCPPnWQ&w~O=i?=kcY8hq~SX5HEy#pW!)u#gPu~rkM--M9mcwkm~e%(qGnl*xm?rpU;vreir`u4Aw-=~0|lmonLVm+z{3+e`bAh}T8OMyh3AWn-g)25wC#lw~{uWI|_3UX{# zi0Q;}5k#oX8A|~<3d-5ej2A-f^Ag>reWs^LBhm*(?AD2`Q`z#nR&?n1?7_>$rXzkj z{d|6<%J)U5t}1ZVF{}0$f&Fic1mVWJ#~DTwgZ_nf2QELPwsCK7dTiF83%YK_J!5Yp zo8BGhI6Ci#+@M12)f`cF)HmX^O)3WAozi~v#O6A z%D38^oAs?lkyxfjCj%vMvwpu1SGON`Kb=XqAH**7w7nGQ?~fh7t~tbdZ$;|p;|vuu zP$r$Th`u!;|7%aFuba_MJN)flUTz-Uc2gf5=O=f|o5mjPG38rSgf`I)t;@G>{2jbM znkzJrA7j4BWt_qu??>aqWk~wZURK>dpKs&CyJ%RXv7VJY+dp@2GgPpM(CGSZLNCs4 zZZCRzwIX=U6tBZ4+lhL)xxSsfUfn4}BI(AW0hU^sz&{l97mmRxq;SD8)Ho3nf%6I_ z4@OP}5t#yboHTN@CTM!V!Y&h-1?e!-ID-wwwq+0zRnd+lH9z0hyd)OKxIh=)l#k7* zgw)p5>vm~YRD!{^DM;N?Q}TvA8?CavzhOgcOrXQXoo>%i0O@1K!cpqP*_B+eDTHBB z3cjM{b9g6U9CxfLDqm&&I;sZw1Rbn|;UC!;4BW24p8bVcThsX)a@>h@0*l(A*K5tpTa77BlvsfsVM47qiMiOz%v24lo8; z>uW2kVF<&;VEVdl!-NEIFt_nZ81LVgo|5)BHj%f?@ ztm6z&?&uj8K*{nO|6Ddig50rW=V}KTDx?Ro_mzrDagRSDT~M80+{$L!MdY9MW!DST zcQFImpchQ**wmy%Vy6xqJGfZt=iO$*x>{=_Pe9-*%3Q(JXCiyM@0b&LSlE}+>1weg zv#HJ#A7Gh0c26p51Yb;B+nzUTRM7u6X8XOWIx^7(uZKXGW>zyCi9HIJG2%|8;eecY zckGGlCGA`{j&y10CAwEgU3!OHil3(R#zRwTFhGCKXi8hss844UN3)nR4HTxyrkY3H z4!L&Xu$j*dJ;#n=G^tHpTpr?KRtBKqZvuBF>M}UumO4LuUE$kqAR~);s=@YLXz4h< zMA@sc^+y0*8e<4!rma(2*5s9@`hlEtE}wk2)taF9nbQzah|}U=X_$gm-xU8@Qc`w# zc!1ymV#{jF%)-K_B2yrTGvAV&(`@wzQ+Qebl;XlDe+-6*TG=?gu2zBn`RhAWc+NED zr(;}-jJ%1n^r~rv!%3qNU~axP_4I?-Pz)}-p0sv0u{=)zHj|h&-Y;@B!2q9#m}w*? z0BUi2&bDf$qryXbq5TCL@xvrdYO=pT#O~6S;O$_$KCIQ1l0L z&C=xpBpyxXnzOdXg;VX#uRiWhr%HRrAQ}^s>K`g#=}~Tgq#Ge3c@%^|+tIJp%5?D_ zv`y#J$CfpHfshB4UU5#B0kIbMrG^_1HidnwJYyc8hYNUTVxrP00`~KYBwOMN4WZe! ztAZw1)5Ag5&S_gc;c9t-UDN~DmgxWl$BxSkV4g@g@b_s|bS@BqU>7 zZU#b5#iC_KNk;U>&_qx)j^ZBE#C)-CHydowxNl(gnN4`yQjn|#_V_#cD zjLW_u`6+fh#GS-5cuYdk=MvWaw^?YS?zx^a23yH$`^U`d_eE$Tqr2_uPy4G2u&{&G z1~!5U+%QKZ>kd_Vz0P{lWF_m^F8FcvW)w@2cky$autJUONPE4Oyvc??+{M_a1VGZk z{I4-8cIwlZ8Sv#STf8^V)oN`08Ou!5iO`-6EM_dB`f-&l(8DU=Y;%(j@u59P$z)!t z(fAW=X#UF#F}JRgq(Heo&FF61;qp<}g1@?Qy;zX#zdK3}kl?{t(Q`c3v&tHqN2{v+ zm6pkI-ZH9T!Q)xXz9rmeE`R9D71&pLJuTRqbhZ;8CU+e<&|NqVDjRctAI(~xR(fTY zh_-)`3MmW{k4T2Wnbp4yny5Sc3rwmgWeLwcsRnXeJW(#WoMJZ>u zYjpYOzY{g)3MKCHb-Eyt7BNLGHyTi`9xlokIuT$e3r`wZ+@`BUG~%;cB1A;j4YEv@ zx*`&>ztG?>C_!0XA3=1jFUEjKZ-7l;ID#bw0}&a*frWD{vZZ1kHY<(JJkN60{(6EOdO-<_hj!!$;&#D9V`_3oGhYMZgPV$O z9VqqBELta$e}MIBUWst-(_ zqGxWgSxz0s6vj&p#U7HT`t$pa8rm{Y?wAHoRSLmkEz9Saf7W8#sYCOIA6@x%k?l8* z(G5S*F+gm(PE#esGt?t4&!DI==hDmUFV@g9Q8VAl@ISXQ${R^cmEY+XS|<0Tw&KZv zj)vq%D^?WckQjt1@(%@rDg;`$bJz@i2{a!#U32taC*??Ri8XIG*){TH@X@qyG2ppg zK0lBQ8}ce;nrtf8n#IcJjxApsv-j`0-DKiQ_c6}K`Wk_5I-KKd%-Cxk8UuUxTZq`rC$ zq{`rVRC_OEsOY_!IE&vSW=t-27n&GGQn#srs;@|OWeD|c^9O;-Um8gPIF5qu5#;r`OkiZhDQGXsCc z!mDoTA$^!Rnay+IbS@Sd3!9VQEOGQY`mh-1NG5le(|J9QP@g8-c_J4EgUzRQwW40K zs`cjkSXP9*Nk97FXikxYOKPb>LrJNBx!l`eNNx3)VeG269JH5$~+1Ymb~6D`?Is-T?`m~jL>P)V}dI}wj)(D3IGdseN{-_n@Y=~a7XyG3L@ z4L!_#e}K_0v=!F;jYva8=~Ib&pR9u)qn2u}Ajcrfaa4bG*mRfG^2B*_MiMC1&0E-U zzH7*PRP*`N4Pt<;Q-35X7h%_StD)USK6~tMr1f~7>QaIPIvo#3yKOlIrN_8UE9(ES z&lKz(kn*tIe^RG}me}eq{z*Dr;jp_1YFt~7ic3Kb-EHHHYvdq`j7BctQu8!xaVNGo zK00F3Ix1@kfJRewA+dSuBYBdx+3`n+jA_@JRnGH8_tt+66u&VNpnGtDA_A|Dv zFi`a(8jq_YY($^g0RR^Xg13PeD+UJG;0N9ggw3Fu(Zy;_>*ccqDZIPs7LGt?^&38a4WF zJ^_6r$a(nuENX#g9k~~l>>BHeD^c>u+?B6%;46k6aG}VxT#IElsr|ZA1qTv*$&8xm z^1BIP2Ysb087=mE*8KEoS+3O-3++uAD^v6LcH#L(gp?_KLkQ>@?kRe@ zc(|BR!!>DTkdjU6CV}yHgb+jGf@(J4SUkKeg%&??kOP)kmQ6|r33k7II6gRMciXv~ zgK?<<$kXV^x`TbpTns!>g#$Y_7M#{=^~|ETSD4k1TwzKkt{_f`|-XvWX}xQHv- zEDl_hx(w_UwH5|V(l4Vo=ZJKJ_zO{16Ivo6qhP$uZvrPc7W8%ZI=+{Rsy(F)8GgTW z5@dExlSa0MW~Fs@VGllny>DGthM%9b$Ha>l@w_3=PTHwSwI@>zRtnCJbjN--_}<>$ zmL)2CZK?B|+AEZv@dgWD;kc_&9_8L1zprOUqo^h$!M^i_5WCE!0m{+7hRbP1y`3LV zzaz)!T4|WABVP~C9@QT%daR39&dTijvydZk+WFq!QCmJ4t&$o4(Nw)cV>zs-hf_HqrlO(mZiO%v=lB4kA!X~X7oVdM5<33l4x{))pC z_Vo;Q<#L322DSQFWBf);*<$Pd?Z|OB~fu zP`!VN{6HmNd{$@WnW=MXs|W0p3B3+LZv)`A1AqdU9Kdet*nn~?rMg{6y$oWd>MKOiBE}qM=1qp)G5|3)XYy#I;k*@~U0>G1-8y>R?mn8gv z?i*b|h0+_|Gq+iN{YKGw8f2Zd-oUJBrX4D3l#{>90lL?yf;}Ajbto7L?F2$y;gsD=J#mU)fzl85Byv1$p@uT7L+1ktg z)xdO828+@CtfOUh86NjG1v0d#0K2-1W5#b|?c4j~Q*onGED+j}b^kQQYGX(`5>*by z>d3iFitJQS2S{%HW|?;Tf((!nCs4(1=Nw#JaG}OAn3=57J<)d-ofBYv!l?Jc#Q>ic z8?3|yDh#kZJ9x#2ZIb&8P_^9>1N44{qDP%5-VaNmMn&9ZLGCt=6{)kz5iE+J>qbTY zWiSXDI4gwSn-MScsI%UC>AKIxsj=HyVC9E`ChvI_RwSZ55{H21S8<2!ERfvAmg$42 zMc}#sCX7d*tnm(#;d5GQ`N#;~GWRv!-e1)X8}7pt8q?2oscY+`UyXw@)v%iLsqZr5 z`m43Zeid-G^O0f|bvG<~jlAk1y1fd(i}Zwz_myD>b@xlwM<9JPZ;YSTxjD(e>hDHI zw`WBNXNqZypUmL9|Dg`1PUCX7#Z`+?&qk$}Z(+ul6?C8O4Lc{Z1J}Wg7m3YY>@9LS zE;G&&x3l^FsFs-7AypaNdO2dr+Y$DkQP+`^?hMlQr_hknEfS-$po2lE+YR3UX`b8` z)N8la!&K{6novR-jZ(0n=4=Nt{j9RRetFTDS$Q#bFD$>6X-MlP%3h3Pp`gNv+llxIcTc|}gP1s{g0V8A zz>3OP^h8r-e!SO#hbWdJNP~oyooSK9HxL(p%!O?l88=WR$yWw0b-|4 zW9+|sz{>_O|Ne6|;*@tFVtMu$F7mB8WyKux@8o#(uA$3K)BEB~JAUgs`kDbx#P!@- z7$P?>F#XOLUNjKqn)VHOGcWo&LV3W+L`1+W$tKnr8{x5AU2(lnw!SI(aKaib`6}F3 z!d05E!0GeFNH)CIs;N# zx|(f?ic_%t2i##biz?L~B5I6a)8Qe4?-EcwE$?ggvi_3_!#Br~U@OOiFgHW1`&w-` zP0V+(qEM15u3lq#^%%x$sK1~tlmkA^-Zx%;3p^r9nG$-WYNx50GpRuTC=J>Z2gJTd zZ0_1zTbT}hRBcoq%k9_}9v~W}HwsWW>FQwRKC$svbVvrWB&;MI2Kn}o(c;?vk`a0w z$?_)&^zyxfU9c557!_Pcm`U{)$=jPUqpBug&s71=NAd*;(5;D%l$tnf!}c2stqjaQsU={ACt@mBe4{@h^SBL`x6F z@E_G7%0Np`!0?ag_=lh{{!{qJizxnO8UIUWF#K=Z@gJJO@E^A#`xlJR3H)PMZ2!U) z0y<$6R|_K(MKQs@@%|sA#V1it97O?Vw6~E&unN?KRPk1}Gy0wGxl~2fFR_5Np z(~k~pAQBLgivLK}=n78GvwPp(M~elr9B0+xJf>k%n@mw6!@k;(StD=GvqPvB)E(oU z%rFWHSv(OQIfVw}yiYD6bvN+KEQPM?w)nr`|t!mJM{`%^n{A5_|cYX-ZyC7PnY)hM~C*dpAbh6FuqGM zx#Vw5iFx$Ci$R^3ugzUSr8Fl!NCkP8><>}TuJwzRD6uD!N@pZ5CI>b2I^h?3S1=4u zn66JqZ6A2;R{&xx|3w!Ny!;NE@4hKysxvP5|Jt|md)#GOKtw_!- z@t-5s27l@;h<0YRX<3fzlQSlU_fd6+*`4jp=})cv2_t-k5knyZfRyLWjq?+dJ6Qu) z2QKSVIFWcwi*`tJZB|b3A`h?Jw*;7DLt26$q`ygjQd$p5-l{*X>WeWdEYOCMORD%j zgdDObm)cNmib?5u1U-HgEhUgxeCLr(sFl}pE~#6Wi#R&olHKv>rEaEDL|d@US7nwT zo>RYzdauc5;H}k6)v~TQ+GOjdZ^G0};b*AqiK~6)RSy2JaP{*2fKSx{n*ZOK`>&+< zCr#PtS^t~4|D@nQRevp%gpiP+fs=_b!M_qpkwE9aZT>-6|IP0IFE6)oc<%f{s(X| z{O@}Y5H-1dtO>^CRY|gx->jWbTayEe~5Z9`2@rb7+{qWD!Cbt+Kc)o1r z{_zuq*r{ov00G{dG$qSOuRy+3wT5#e=`~(r*#W9oPSG z?cTuKPKTjKq9(*fn0F_5xQ0d&^7bnbSFa2FHtWK1z2p?E?PXO;* z*Nv+)0#EQr52J1Ex)%hGzi(`89@(75=|+HUz&3tDPZR7ZXV{6L9_(YY;Eg78@Uy^;gR|{EHg@zmzrtc% zjX>N$xY=Tl;HgfyXCC@H>1egU;Hl^%@FTJ_&?y$!xUGPj?}YaX68v4>(a=cxUp4x4IkJ)MnNHlzNe zBD1LDMe;Hq=?+Q|vfb*zU0-%G-8D`oyT?PSEl;F=bx%mtZ<60=7fx{uZROWlvrG~J zjnDn-pojA9O~#!arw)`Mi}q8 zU1J=oeh-NZN>%VP0MjT$=No8YS$Is7NE*8_lQ-IzH5-_jS#Awx9iAZ&s#`EwF$4YY zAaWRr88^eNy2@?~l02qy9O$kXqB^Ss`BCGDHhaqW1E+Vw@GSR{bS&ejL})B$>^e&P z&~0d&;@ItIGU(UyFuQ31x~ag1g- zuK49n>n6BoeL$3VdkCbf=sa{4b{ND-wGlA`dOX;zaiM@Xe%y*%I`$RS`%R8CZZcWtOg z9-C;|Q9%LR2(fH`scGGmB{)mU+r(N4`g=3(ZDWQnu~P zo%Oi}t01p946*IdFL$!-5H>PVIv(DiDK0xdFWrYO4;m-lZJ!q^=;P)osq&{?q)8paMjz96&{bSdXnvEUyo8|yDG(!N2ck+Tgyf40)@L&EHEN*dlk35E z9wk~FRZ7WS_Z*BA*FH(Yo`7!;R+Z8FnH-+GsmaewWXI8X+)vFttxPoP-c7q-ls|tp zT83q)emmxm|77?5ybwy>mx(+_sS|OuA&UJmY>2QVC)Q<4gFj85-Xlkh4wPDzCKm~& zFWmQ?1XtFi^huE%ZaO%Y9Ik1}oVMdD*F=8?;ko>{GwCIv)5R)8U*n_2YP%Fe>Fm$0 zvkGjh8j7J_3+93tb+ZoyOC%n^sH*1F<*`7;pq_?_VfF#S{_Su)j34-f_=i1Q3#`eC zV19?THd%x_tabrGAsM6rm$ZHWSD|2|AGR82H176-IT%cmL<1|ij98L;co63pq!8`_ zE5VWT&)RQuwDVx@CS3vemDAmnAykJWZKjp!C&x?<*p)(-ZhTM`z=LwAjk$Aq6B75Z zeT5B~yVJ2GvLtZ8VE+O+X2L=bgoy^l?-5zG)?dX$0 z_7$0~(IqE*0oKcu6MJ>KdsQyiNL8(x>P^;-4rO&}8dho+8k+Dq+Y5pEbON_LoJ5tgQu~qv#U&TFvD}9FLnm9zV;pg*xwcODBKmHS$v#VL*;3 zXrzLEfdESoVXdY}mnZWWO=53=f7a!GxUcYE)}CCxbH_CIhoN$AcK?wuKPs?oy6W8m%@&XX6# zE7;1(arwi-RK=2xXhqz(R5!b7wO9lmKT<@96_bV*Uq+6>LkrKa zu~~*PW{v>x=0AE0=^@@RLRsS|N6GB@-5{708?U%0LU5C~P?9x0{ zyXp2!-Q%*66mvDFm#UU*tLu79CIs%?<87*a$8Y20gyXEMsQI@0eB8`IYxcQEq-W7v z(P*=)2J%^fvPEGBRNo%ILZPms6dc%uqZV7MNX{lB2iY+WeFEW9Qj*NMlJ#l7XE=Xt z?be*Co$XuPxZ+}>Fxk4sdy3^;OQ)^ZFtv|r5>zfI@A~JGTtB^QGzt3;aiTGR*dxj+ zjw$v%yKvLC-6Q;a#3~rWlMd7g9M!Pq+YtPm*P@PByBHm|N0ARE7|@peL2?a^ z`naXG47ySSv_lHi=mIHgL<-HwYHJHLOY*y6L1!h_u@fF#8rC^G6N=1n;Y69gb zlX)1M;9w2)G^*(oqIK6ziFFOKR`Et*>r_t%zB7|A?cUW&cQ`Gw=y0oI#4F`&RMjPQ z4>caN^r7w^@;&U6&G7Zav%$Tw(UwZQ!ah$-Rn@=BJWtA?zd zAt^9f2?=P9jTTIxkkBL2Sr*e|aLMH&tspT1YV;^YO4;kxfJ=M*9nuaz##PFI!5S{~ zQpHfj+}pdxvop|OXd6u`bs#fEIw4Z@`WC0&p^lD(@^XkV{&*73djfO}&(KTusqMbli+@*)+zu559EfB zG9+_A62G)e+PdG4*KE0T)Ze8H0-Nn{xeR`{1Z$JvdIxg%Ct9S!N7@~ye$E$b1_Fnels6+xI`LtSxLFxk)258_E%4bUjA z&Kl;Xw}7-_tM2O#((9UuGo)7$;y$?J5K>~z(b`ogZDzope^>^_R$yup;(hJ}_atW~ zc?QI*D~SHIUk233%&@UG?JMHEGT&4gNnJJZZ0ialQ znOc@ntN;3$&cb-x+7OPMnbJ9PqU=p7xpD2_VYG(xW$&}& z_FyZ@O?YJV1dasEyz0Pj5ch+DOY&P3(?ylRcQAkNt3T^Ie*}?0rRh*DmE1iTWK~Q)x(#M z17Cnm-leGt<->iclx|9obau$tu71W_M-jk9qIS=F*6I;xB(V`Ov6&o*Cp5>2q2x-t zX)bpQQlEGI>&Ds(kYQ3Jsa5OvT6D%Emiw-ZP)D={^Cre6xonFC439}KM{2mC(tID^RK7zpk?E%D1^!uug-$); zvWHZYnhcp5NRPhNAQOr9m3^#F7N&~V4oniH2)R`5SyCCE737bcmCPmI6$EvauSv4n z)1DqkZcfK)@0C65Yx9pe^Zl(Mv$JwCn*MU7`&vjn%$yh=^~RT@c37G>(Bf2Isg6b_ zJx(rP%A(U_cJcz~&w*IVLu<>%4c65%3A#y11F+~Ppw~Vci<%~NECvhQcieIbY(=P5 z%O!ASczQoP)v459aa$uDBQzFg6GT&dHhB1Y+&%kudhrRzR~{is$ErH-phIcsnd{r` z44CNof&(i~b`{MEQl#hNogK|w7~ma^eU?+2UlKgaoBHlKFhe*)d}y~-p+u9ms& zN$=06>gV<-Q(2)~swEm!buIV7VNiz0A<`r}0qqgX2IXfhBx0br0}La~cRn@bKrHre zl_{x!?&8cF7DGh|S?2h|_}w4g_RgN+CmeJUm%BiDVv%}VD-vLHu;ico#T8=;)f;KBr>`SHCii}~2yXWuzsa9NmT zDBm^{Ez+^g<8YFYkeNuX;!b-N$U8BkpL8%X1utrVu2ClqmcR(m_SYoNBefSb8`lL3 z!EM@%FPUQClBNzO%v~{#mC%q{D$-*vGDH0Vu?upk4461?!`J6Gd9*5lOyB-wk!mL4 zp|PmVSaj&7X*Tkm=tmn*ES++rDEask+5B=`i`)CY`pdY;K}$fvK}yC_MwMG!^w^q) zoKkTzT|_Kn-SDdZnJ?#Kv($gZ?vCR6XQtHp2c5SV36QqzrNpNyVQqsY> zj3GTf?%Y>^(wrZJ(rnM$7A`#Z+h<_-gz^YEBr|&!*1=Lz2hO{(8$F+=m)g&7H;K=r zQvVc&1G!iy?(vhS6$$@Ypns4472`@Ku3N*UQ_pm3rXkj2}4xKkvC2 z&m5j!0`803$mvWh!tojGil4n8wre}C*2PA~pf%JWc?E@TrvOEQpXUl@>V72-yP!w<7IWba59#@WxJy3;t$Us&n(C{b2|dC@}bWYR_;WcHpE-$ zWN62)!{=EV`k)3Jt(-0Bn~^up zmI;J5P!k1O4GylZy44vzR^_MZ`_fG831l)su5xJLI{kA5rE6%f&+AMX)Vzh3Z(qf) zxG;x)x}LhrLno(9Dl~jo<*x6YubIDk%TrUj=9nCvbhj%4t%5$-)TYzEZ9j>?!IEGv7G%m!x>O=+^3_ zlA5qg0ctGCe#Q*72mOh_Ix@sP*^OkcV6m`#(u{E8R03pgHI6k!0`A~Yn$5&X%v0D+ zFck}hisM%+nfXlOsHl<_5lhn79FsH|m%`*1*VE@lg77YY*-8@+>#KqQ>n-ehbZ zh5Oxgv+sJH02F^@OYhf_qEpuQRamQx~kKkdx9m zqQ*pap*iF2Xi_O76q%m$kb<-r(u!QkVBZifhVYYpc#X$-*A5DcA6y8)bcG@zuEQnI z42dWQ!gN6=4ZPG}C>FqNkr_pE?aG)#$8I7exj( zDY6FA2mbqb5hZjg&wa58Zg;#n1zTXqi;uuyk*^m|kqq7n3x4HAz1p*-{IlYF6UcU7 z+;_73Gw@{F><>h;-Oy&)itc%HQc(CW^R~q#9_3(fc7VD~r?79P3Kv7FmP_oz-Wj#! zqAKkCz|?)<%qoU4%;+NaX`vcv@K!{Ijf)W*7Gx_mXDA!Z!0e)Gjto=&?#0Vi#NrSM ziuoab2HeK-2*TkcsLHJ?=)*WMM(OvO0{=7`vZ9Y-~nG~ju5N-#>|t%3*Rhe zJ4d|)0VtE2aUY=Xlr8mVKhm*p`eO%XP)0Izm?>~n{dGjAxPw-9ub}Iu-*Uu#SYx(5 z6i1>tb3m#66()@A%U5jIs; zsoir^7`;%$9T_!3bne%QelknGrziqR^e@hN@V*H7ab?p6vMa|}%XFM|P!_}z>X4;+ z-4;M%$83F1wz2qe_zswJ(wK+C!>G*UB9D<3z86Kq=h|DM&9MS>=2z4dzcZL>RdeuQ z=^vP;s4}4(17y?LJo>O@NVVAzL1~QQpunQJSxtIIqD8|1$YgWo=FS{w0=V%*tS zOAg1NA`ne-$7DTWN}n+VW{n(~eu1MC1nlQoK0Pc_w&B`Rwc(XSFS~+%uNEE7RPP(G z>YLe7zAHx6A83vwKTUQU+o@h}WrQwMbmiD_Kg>UeKM1>+0-5GQ;%eLuCquaFUpudM zL&|_1f28{MzD^y?KgPfLIrcP4GeakUd3?!x7{T*5iW(h9LPJe}hFKs(2F zM^5*5+guv)SL5eVt=1~wcLv8c5;Y;kR<2?$!as!lcA5^3MdtGi=4Nt;j43qR{R}?F z;jDRVWK-yU$B8cdI)7N1^;U*{!3(Kp`J5^Pj$In(fF|bz4Biu6$8X!G$#;~V%9P^xKHdgz%Y<0k)9okVU8cauUZm*FJWnRPX{ zTS7e#b&)%-em4NNJ7xG}r{H2nr2*HN4r&vNhkToLK0;@>XC-(q5~WVMKTH73_rP_5 z&4v}h?Y8pjs6Fec3W{hqhw#~8MWyJCwjw;oOu)QNgyoPY9T)I0hoTH^mIQ=PE`(wR z4pG3Z5Q8m4WC>^%o7G-~VQ$V-WyWvL2Q??O48!Pvpb?dlC)13qkuNF?y&hEhJSrKy z9=zXM9t;X=#p$`ZUENEC>-ACR^mx4qi(oXqy<2Q@RKvjL(#9<+?yk&+p&ojn^)@C< z?ru zQcTDgqEU@nHVX?{3Op_PHKUIj5;Yv8Dt%Q9gumKxe&<)e4ygU=(CBEc#ILH}mYXLe z(VtpvA60=d4&0H~4oo$Xq74L~HP7DLA2>Gsxqvi&N?v^{L)KF^U{D+dn2Z34KVqd|Hfr){(Xd`UFhs+ zSb0;~p!yJOIi+RTY`^9l&92>qv9$%W$GF@8{y5~WqatI7(%nIQY(7X6W@*5uYy|ac z%XIdZWm5`AV!|UCQO#xrlq5&MCTh0b77CUJ)n?8B3Eib(Z&0*|=6>jhqw8XU`JqBM zEtw&LEyDS@0qzR&D0j_5>C8!9uS=Y#TZN7<1U{F6r~7CFjs+LATS7eTJ8TSu2n1H_ z9PaTr(f8|)AkvvU_o=yPq7dR`x*Xga*x7-;jg9uV*cl7W7Am_7eqvuYt7$(;M!`_j z*j`{0_Ov8Lzv?9FT)ZGd@4_#F&h(t$A0(=!9Z%X+qAMrJ0ajznziOfQ&@ zKN@KP4D_%s8A?omX+i+NI52RGKQe9*(K0d+(E{jMiP)K#h*(*eU+^d|&C+^SKv0OZ zvaqPSD7CVYy`i499_%}DU?AGefZy80(g+y&ey8AQWTpI4c(3QGVrB?Tio(DQ00y^J z|1}8~R$$x9Xh1}X5rF7tpjsaGFOlotF?L2~U^_ki3#jFPx-l^`5xrc$wHR6csKNd! z5V0__{L%hr;kWkB(%}>3RslCYm<6M4sV4-IqVgWD`z3lkUCp!z#%Pzi@Sr{3K z0IWs-8JTP5Hb7#d;&s{>>V5h zP4(=5+N+e_zbXs>p!Et{1C#8SSp)G9@AVAiZLIXHe@XsgT7Va9)Guhpzd#Ydjrxg~ z`UQ$$0MG-~{#UqDxw@Mtx>APEnUa#qH?5U}hq6;?m%?fB=D6|qArl(1pGP`a(y$ViUpCg%cL9H5`pho}%aHW80Xw&?fS!XiqI`#*sii%5NcF5$>J zZO)u{u6~j~tN6Zp_TWA5K?#IK;MT#&iD_1q*+9_sMe~o@xoT@wIY*xB3JM|!xN-{A zSxkmeCw^@CQ4LSLi$f7o@!5POr?E7XEh6SpD^L>(>-%@w$4i+X#ljSI2&h6}S>cT0BQ?weOnyfCtDvEt)K?khC9oT@v5F1ckfD^4}$ zcrUVBe4us$rXHvhE42mR$~j~|g2unosV?ym-bszqqlMUkrquT1h#++*4MX3LVS-Fy zR=zUeeWf`4!LPkcb9Y^?J6GWF_4Rbtj}e)Nj)wgh!iyN(7?d4iG=Sjhn?_Y2HiFox zb4!Sh&l%y36sLWULGEe5D#ZgO7sBtC(~;JAuguF%JX9(u@dDydf%8*HN8T^B zpk9Ggy34PwDJw!Q`5j+95xy62_#(%I!W}t~3(5pBn`QJFac9d#mduSnkykZ-4zx zVHc1BMD#$0q#TUT$z?;yg(w{%)FTqJODAc++@QK6Jwbwf zE5h#6%TQ**TZwYcd630@LiM=#c3W5nXklwXcXmLGXoWukF_+Hb&FRnu^M-gm(p&>o ztCq~^TOt>?{EU20>5l#+l><%?V2c~Mj^oOLku_YqWT|%Z>Uti`{EfM2QV-t97X-ec zr+ueu%)qstGKkNp6TQ^uZ@GMR4cMpG{Z7ac$|ICZ5x~u7f??h5aPI!- z;EDcvz=u5WycfA@U&P*~aD0CQld@4DO(v-XJa{gv8$ zwZX?6UOj=PLu5Q)+R6^IEr5iwRdk7KA{|-wQ*wbeZWVtn1<9Axi7l;lJh7gO;=;NL&P5(~ilZ zhUkNe+)EM;17ZrIWY6R}vb|-V;+(%&>>nNYNl4qAZ<{EDNCsw;-!)RM;tRurcbv)jfh7 zi&)=>yt1? zEhzDmGZhj17W;HFk&g-07ur10?MFlQ4y|zu;Xcbljmk*?e9HT`h?Av zTZ!m-BVWBY^Kv|PI^YIvmOk@u3J*^dI}&ZQz=XtH<)O*BYm=`yyR2ZEolk$;HiaK- zx`Zp1Uw*_}>!s3~Y|*sk5ov~Md2;mH6vEWnW(30{Uw(~1);yEAvub_#^K?5U7QGcANDM_0{hmlgm9(zAL?hn|4KmOV z1`aAvS&zYzjUZmZbpQ2!6ev4mb0XjJ;*Mna%DCj>^XaKNHaS(K;K&xdV-K|CfQTwp zGzeIPKw(afvoFQWBy%}+8|98vc(JuF`0bvUsLDNb*1`%xTiCmD^eF^MO9Kl!KV~c} zfwzfxMvxJpf{sY=+Y+EU_k*3j4t%iznr_&+KeUOXl;BC+Ky(xIT8jm(m(b7I^TT}K ztp^x673!9-Rn7OEBs}51c7^@!SIOsxXw4&b4lcqFVG>FwREtKDr@s$_E{rb$!3j#j zS@<|L&#{6TDb1dm#+8!Rohc)|D;ifs%$~A6A)tsWgY6ki&&6YWavo%pd}EV=SK$5K zO7`Y6$GMgv*JPKbf02L_x-MX13a6=PX_$SJ^u=?cv0LdyH#Zs|0!C9vHigZPYI zP$dah2ik-nZdq2m$k9aF?B~M8S~vxDU@i~a@3SdKxrL<;|y0ElUiP10d5}-1QKH=_q|7!mGHMJ^MPpeieQo!+JBX6L)v} zE%GZ}3lAQ6t@?dqrmog*xX1JTK?6MF>AT8ELP?Vh&U0Anp6xq17?pFuVj@N?uKArD zuX3~S&_Jn#w>dcJ6JE!K&+M5`Y`XWVoDr3T!>Km7{j6b-<-u>5OpMu-4FbH#X6U1M z3nxual)r}Ga&-P+SPtmFMZ~tYOpQYju^8KSAX(dVm^3xR2i3!@?J}%R@xu7_xm%VH z_7=lZn4W)k>wsD$x-!+2+IDnsV1!wUHMf=C%%1q(&R@h6qZ#+zP%dVy$G5$2d+(WYRgy>VvqGcn-^LJF)|Sqwmgi7xXIVN2tQ08g ziRj5|78{Hfv>~G0rQw>VZU{fX1%l0EcbztCRc{sEU76^)5I7B3A;zQzqkzTuBlP1M zL<0F-+iQk6s~63!-|15J{L3(%)rzNzaJZHH5^xvPQ$nD{zmAaKZTFRHsr7BZhB_*D z1s_$@yaIn1rYbFquSC>|#PCR5V`0RMOnka)D13g38^wH7K8u$hxU~79+sC>dB(rk0 z-d&4_?Kh5Ii zuO6;{yPp*)oZbk;AM;i3H_9O|T0U|_4OyvHVS0e)dOr?TKbr6?=rL->$RIfwp9M{% zAWuUw?VxP@n737Azyyo?%z?d|Nm!QNChV9%&kyK4Mi9>I2Oywbpo50h6w9qR-xbwSjS?_Ck@?_24VByg z>Sc&fo@IUJ)$Ub`fv<@0VM@%Erva}Cr%9oPR_*dUN2QaMleN>y z?wQl6{mE8l^dpshEHBPvf~|Avm^P7Yvi{6X*85@GuHq|4RaSnJts5bn)WpSL_yr=f zZvAf!uf)+Xv38jBkZgr|K2#f9l?|}G3hX>I@&vSj(Jh^CfGw-=Z*HLsYa(gC~W{53o?N^gdvc~G#Scl$s)EAqR7=qWQT2@~i zHpP+`CV`8mzgCc{-!~RXCR|S!A4d9&%VOv3w2y6LwvaxRtstgUwjipYo@7~nv9?ob#I)aXq z9xjRe;3-@-Ok=-y3u*i^8QdczIxHF;{mfjXc@4vCzp=!uRH7SGi+Mzj(f(}Qn7f=O z2d-Dkr}G&(Cxl-|b&{@H`vP)@^x?u=fUqkX8GtlF(i`3ui2atMzRw%cQr3>>3?V#D z^$K3hk9|B#X*ROYD z8xVVw3GFGW;51x6UPr1>(e_sdhkmblFB7&qcOWEe(S0WHCHvWA`PP6OdmWxS^UdJN zLK{?-la^JD{yLMJQ!U$#D+`PGuB?&+W!y0zsm-yQW1XU}MFB*w-+GDA^)ctri zsdHV2xV+%;D2aYbH%Ms0_Zd;*xZ<$EX*{t?({??{)(J{epYMbD{QASxP-lYCi4CBS z_L23E%#Z+k8r*~WV{jLQ_;guJ6oCv=2cICv1K{=IL~D*}lm^bl`nmU7trDHE&T;qO zE;@jvP_+O{iVvsg za1&-#0s$@-cD_C2V+4aijZoym?N$~hcD$R>-{HF>(6DKBcx1VM?Hy7i z8^MDb-SA^8>CAZbfGWqUJ|F8oY;CdL!qM3wc$@wMDCEhE_qHhLbv)aK^O5#x(S}#g zLxT*w372J1voN=0n_`nJofs(pNs_)q9r;saL ztwRtqSXV2MNk5{@ONnLN(FvUQ29dpaB;2-KoQO3XZz~L(Usu^Rdr^^ei?F^H;CL-W z6+r$j@Y`+`=n5gkZ+k!1u0<>NTt@fQpNp2 zEhm-?b+OYdQ<=whc4hTu4V2GY(pxfF(_7K|lyX+ik8fkzwFT#1GiTS41h4l(9$@zmdSm2MIxb48Mr)-ces4H*lfwE+7gMDPOW0iFwQV!vaHD{DWKZ z+6Ah<_C#{3UKu+lAb4>wSt?q4o9Y%1tU`8AD1v<-`cp&=I_Jl+fD;xG(V&>Vb$_nl z4|CLpEOAd)Y4Fi*gqXDx0c6?WsaWS|Lp?G0NSWZ)7@vNe?w6M&*{IjF(IMaUOz3z% z$W(I0p*!B%)z>;YG1Q;cgJpJjICU6rK`l7lzE9)L?y$e_o~nR}MS>x^992gbm=9A@ zqM|QwRs!#YmO~iCxTjc~Ux)QZ&+?R_n9yfDt|6OQ7@2WhE)zh3Eun~(_3b*y5zZe! z39n`VIj0}bv6Y&87?}7rb6&G6GYpl8A}GD;%Z2AN1Br=zny8~p?eEdTLmo2MSc$pExEN)w%H)h(&7#{Txhw1QaJNRzM!4*4&prxy#1-_a zf2q0Dq+e{b_olZw(WJX#wR1A56_%tIu8J~4rWSW~Ib%MnlbwcbtKTWrmK*;L>z=@Q zv)=P=m4mp0j9J=%$ra9j1@3*uZT;2rjKWgGv(gmeqdB(w3JE zbRAFh!{o;S6ikws$3NBNdKw$IN^+l?ABs6a2w&FokM~5KT3?Yj)@#M7wJIt$X{Icd zUR1OnDPsfNs7shD(%yqMk#ls)Avo8I!b33KLq<|lY(s%}y*(#TPO=J-Jnwc}#@Kd7 z+13xZE2%o(fj-odTA=nEJR%cE>dWKgTGv?Y2_Omf#2^;z&IE%{#%1*;pTmfk(qt2D z>opGzH{|_96E-(193GomQKo#jW$vKn=o21JqU41GrJ$xm-l$DqL{-uCnq`TkZoIVA z&O+6?&QghXZ#X@~Y(0{u7F8+NrMJHPPODq}3iE1f_uXa3kck;FX`p9~sR|CpVEnHksmcRv;M%WYMWXfEj_Nk1wQ#X%)4 z4CPOf+3EIBo1~h%rwBH%uVEB6mZ(C%&lF6l)@Sj!fyZa}^3^lPezm)2y7hT%;TGMw z3aBrj!;G_rh|8Tb1l#YgC0WC@ua}*$ZzVbO&3!d6f~+kcx3eaQ$j1!6vnBAk+hk6U zyJ0s9S*c0FG6o5mr@4_%J8cgz+LTlNK}mMum;q~kQ&h$BVcss}Y>9_<8lV2N;_zrC zY6c8WW;ksszZV0K(1(%Tx&xe@#tyb4bS7nl9sBB&fCL?+N&^Bqe`XG+49K<2rc#K} zkK``oQeF~Ia1vSe*KpNePm_;{z8@0K`{}xSd46I=xnjOe)(sdTzb!W2mVmyiP){NP z?_5oS0gdFFFVRXlxsTn)XzqlhV3QXgGd za$Q+jY-(s^ld@?cHiR~GU#*HcI6RF4hJ0FZFOX4F*8zP&K-ZQ)&tsc&;}{2RjN!D* zveO8Y^;stnjmD+|5^7-H^s=Aw%JC!AZq3Zjjsl0mtjcTwfRTJAthRslj=C>Vij!wY zURe{dbpyl@ECo<0(d3XQi<@z+Uhw&SFI`fJHHx?U-0&-lzWvY!`OR0Pl!_ToU<+kl zM7dKxE?hnccPM%nkbb{JQoT8s(HJkrh67A0A4w^ZPt4di1g2N@gBLcWDmXKPRSXjs z^5m-+)Nhxw5uloB`&KtQdm8y*%qA|-M zSAaEdTN(#c&ea~IB%Paw?9mG!k4-hr#CXS8`G&~=F)&R>#p4P^<@?imUgTvgPa01S zPrA`!e<@e}7cO)V-+>guUNW>sA&sD%5#!Y+eu?7Zg82Xf?d-z zy?mW&4Ecr9DW(~UV?3FP&Oy*{qRx|(7K(BnA4I{#dsw%z_hajYAM+&*4nD-_@Kq~< zqIFyH2stJP#=IE}I$wa{Z&uP;jQjBv2lAW^qN7h24o|dj!VrAelJl(uick$;=e0ZA z6sjwbZjv^DZo*C_(65Cj>|xG`Z1NHCxniAm_jsDIq$8k=v!f!ovY1w z`RaP{qUXz7yia>hyN9D2&!}`U)6o(jZpfZl&`WQnIS#v&ypJPw#c@dlyuE8nL8S8d zSbf{Sc0IHaQl(^`6;g8O+wZ}oF00xOtn{{xjD?h2=JC}J`4LRE+Io*m5iTPIV)aLT zA1t^ujRxP`o*ocYzKPIkct#dlj%yKq%x1<9FgkvvUW095Ts3^Gpq&AqSmN2Aa#Uu7 zK|`iZ{kC^X*R+PLswkH{Zf{G1^V|Z?7*IE%STtH&eA~7HcO>yAmR9^p<$_MjIb1scam2j#|4xQ`Un9N=&=~Pn+fS-o#WBeAv zKyh;}i}&@u-QeY!U?8r@eEU&V+lDC{oeoDp0c6f7sQK|DjSiHgN@p-1Okx@}k=&aY zifFX*a^{Ziv#R*YVoeQGt$un54apS=TEP=*JZ=Ex@+OixS9!gp`TdZlL|4&FLCVGV z>7^^IA2Qd3`#}^`0y1F=ON-8D_|8!-I4z?sQC=}#UrMjd+c-9N73=AMfzH|H9Dz=7 zW$p>1C2N;OL2K=N!yJx05N^3M1+40dDyo8aRIr!sY)e^t639^yg0Y%njYqM5P$yLE zSy?7Kc<{2-v1fN+AL;L}VBQCr@-77M@xag%l9tYyp0a&#NP?~EgIv@fYANZ-5_`(# zpviULtLCpT68qj=xV zn^<1<@mrBTMSuH)Z*TK;id`Mm+0=9z)2j%i!H%Y~!gb7?-v20f%HVNB#A9oEOvxtk zq|9f0KWZ>4az|YruH(dC!iRHkRXWeu_Q_-c>v`b2w)HOdrTCs>4U!XIbHWS}Eg-jz zddam8{(T|V91~mORS}wPik&R&0u-;XPdP=3@U`W~IMz`$+IOMmvFiCeMU+!%Q{%IQ zlVcw#ZF3XWX;AlW2}~w79(PYT8Rt zhEOt3M4x!?-=}Esq+YE(yd4Y1zl7m_t(1*!&R>41!ilzX7GD62%yFdn8Fx38Szrg{ z=4js?Ntd1VB9Oni{U}YtVtxUkDz6y@!M?kLE&iOSSfAea7321(zgQ?|V2co6GV-i z%p>-_IbS-=z7HW*V^{RSUH^=lFc>Krf?Al};9lfV(I^*+W?+z*jZU85C-{vsyUDg_ z-^wNzG(-*tQNW2uVsH#JKBnbiK)z4b1iG0KN-ukHkiK7Vb(F<60@3YVZ-K`|68yBR zLxo{-z&6rHz02#E?>*S0IkOKf3z}y(N969&-O!az88_{yEyE9{^RkmcdbtWxXs<_n*! zLddph%y>wKm9#SIzjH0@Q+6xdHBO0%jcZuA0u5jm->9=q=SfuceBWw z4{6yp%X1cgy7?^cvJyP&@qA6R%2hGv4Z-J}_=rx&X^E9=t$wq_d@JkX2$kVjh!&#FWw~?>Ypg1VV^-RFJ<2xs~ZJkA-=Yfcaff0kI(>V z(3V}m{EbBJOcnpBI+N4O<{_fun#aYZS!&pDW#!}m_JE|Y=QjGbHgQJJQ$(UU`?$x) z@)O6U~8SxcnAQkyyCbmuzlsNkS;$EGj2#CkdDcK`J>R zTqxBB`BP_&3JrJ>KWj?LkIm^K3{4slX^Aauv`%=-m}cS#B5&-m3c#)6S}YAD^5{TZ zt#|BPnVGS}huRd)@RQ~VRPuNHmM{52!5xcb#IztsP`~KW%e`eCXt$snW|o;?mPtx| z_fXM9I^Rk@-wKtI`$MMVx!{LJ2UrsK=R4wq(%Mx%NnWnDgRk>+4f(8NSb<$*NKsx< zn1|^dy4CSF-wI#DUl&fLf>G+7*MxZI-4Pex)$o}kbI(jIGT+85Wa2hUwMsp!o^&R1 zsx&<6R(z!!tG^t7jdLt*R+TWNsiE$j*CcGd`etnQ=U=_>@|y{=$+;Rsk?F(NcjZmH zDNUs+6T&|rPN8vT{~g=*6L|9i#$l%i0@40;?i3Kp^BbY_8&32a8}ui(jg=XQ()kT- zV_{+aE41yU@h7wm_8-u;7hoF@UIa|(Ld3$x_IF^LgtCOpdm(B?GbpWrrTAWrQs;I^NDqnG?kf3ELW@+A!-0C0}1!1eyWT#PReE?_?Z zD-eza1cSZ2-OKfBj9=xyK3{Zywf~1F?0=B`Px}9v%6xYVCOG*+8<}D{i_ND^}Qe(|5tQczABuj z;!XnZUQ2Y7ONNUvU?@=B-XxjpQy>hC8>nOjDOv;y4Dvoa@~0T^SC`n@Jf;jNZ@PTd zC)G=w@}Cam51x+^RAIuBom-xo8XDBgn^$h;oAc|NH*+9PDr1H+Jji7|>sFpl(%JVO z#_uxrJb#>duC@7s=!Sr<^Bt`&B={#)2Qu-#b+3NdJ?S#aW+nvT61v_yu23cz$XK0R_Ru^*`eOzf)Q!4HD4NU}&()rFpYOIpk0y1mKDs7Y-j$x@ z4=Dmup$f62kobb7#c^O!7$3fU6CJK5Lc4(>`|hgrbvs_S2**X@kdG64=IN);=ikRZ z-KOnL8+{U&Joh<5(9ttO`!NA^*4M`7OW+D;-AnKpk@>hE{DPa*DwuqX|85Psb`6^XWX>z?b-pZ_}^m>C3+0*^qX4R}FKf54RxP zj!feUceSp5io+HC=DrF!Bm3zZPCD}zj4*i37UP(08Fs#>lnssf)Xo~WutsHJ3q)Pe zOUr6(^sodcVe$ZD=cAcXK3Sh|&_;1~mU>JkR&wmo#L$hA8@F}ej^l!rvRciDR}&0@ z_A^iKFFtBYk|M(fp<={c#>)(IsN{KKPjy{|O8Chz4?o1G{h(}G5AzH_RCAa7T)epN z3a;*58K5#iLl7bNrNvhy#S;BU?R8Z7v_qSabQ?+ zNU{fi4w>{+8om_WvUhCwEXycIlBh_m3Z)aXaq8A#NJ0ItnnR|NPows)iu)haO>1AymEn~Zt$TW0-__Mj1#lAp=gxt3888{o-9T13bp zl|up~TwU|Zt=KRmr;1MO%$YeZMxT9G?TNCfhj`M2+gXvn)Ud5$LpPx%_1%oTLC%9# zWtqmmeD$P3j&=!f>=opJhT`e37BfomlFzI6d4Q%B#HV1Nn=D4#}aL z{(IwtN-QYTkT5No=yq>&n+;jnNJb2#Nu&H~rR z5fXWXpPjZ|wG0Zu%Omine}QnM)7X!1S4e#~q8z$GKsu}(wMGlApRQ_ShL%c8-HSLP zVpqT8Y!}|w41x*WqKg%D{Ggx=jK}Q;roRbG(}m#IrRTV?7&A~h3`AzIT|_x9AqX-p zY-wi14gI2t%gN1-Fe#lPPwtFJxn*Er@HVM`I0nv+7=AHXHDJ zVR_>~6+qCVARLb7r50S#T+(1oH>cE+F2-R!%bUM;da}~m-Uv6F%j&IUv81Y;(OB1D zo#iyy)l{p2HGUTYrJ-?}!kkMYtKL!2?nLC{q=F^Q&F$Y+zk=s!ZNo(Ur1Yazx2o0M zxu>`dN_uuqxDaD^;;OJCL!;uhW=O`-XM1827Y=!@276#l)KpSn_%nj~t@mR^k*DKa zI=|Rf#4vy~FX+|*%f}z=IQe|IFDT|fFun*0=X72j6EN0&Y6OP)w#KrbPq8tmZU&GR#HbFJlV!U5&>^`A9Bf9zfCjM|Iv$yU)AraJItHVO*;`?E$+dnV)}&G zFaw8I&>&jH=qBO>j)-^#Dq9}t4G3Q6i$ku0PM`>8rF`dz% z16re59yjN5cj^(GeWK>@tDH8D=^MX`w03M7&hW5HV_oCiH2gv4{qB|siN#D3 zR;^ckY*#gP;gz0W{g7K*ji#!e7OpD!Dk*twEc@3*t4rs-n=+b5qIVM+9=*A~d18`z z)@G@sVyHfY8MGsd0k$PX>b`Dl`|d1jwgTiQO^5qC1TKrf^n=;}7I`*kYY2hfa6oEg zhvNan!M1{iPNqil)fo2y`E-aaWnwl|?G%#Tk%E^8+2dIkSa|M%&IHAPM6<7n0nL-n zfuv-k<%%ubqEm9jNUjSO3E{*Sfo@)7KIt>#(V#~XdJ3*rMMOQ$?39JjOi3uyroBuC zNV6!0$szqGwFOWYlAho798L7U$#rk&AAW^=XG(y$8v{%fqtx^&*PdjqR^$%SGLF+O zp%sR8`02Ccs)dKYSI-R>r?JOJt*RYB{|0isvv^5w-jQKE^$NSwC6BUME_YxAUb?FS zoNPwzQW>WEcRlnZErI&-p@!auv3?D+7BdLv{PAH+W!P7+i+D~zf{W;I7pb)@n;C1H zwa}3??DWSWE3rw?;P~nfOqeMBBpSEX(s^05k$U7-eBn)!^nu-*jwbgyDtL)oL_8_0 zzGus^?V_X2b+8lpSAZMF=!37GV?AOFYzGp=lzHGi2_|N(5m=0bdBbtuSLZFnc!%}$ z_pR;hr}5FEeZ*dUrEFS=_<<&m3D0?%lnwj?Luv5#JOBi}t&Isb)-)y00-p^peM9lca3A^yS&0UMGgx2Nyw>pN~rLci<4^aZ-$wKWDa zMW4l#1tQulR&d*vP*jO}HpKLkRG}*3MQ*VKh{tVdmNfxxl)kD^Nb@E=lRahhU`zMu zx4tES3xNyh+F>-Iqk>KNBB_w0C*nUw?v-AWe))b8r}W!np(sxDy1OIExu@x!FrQi z6UMKM`T*E$b_7S7DY9YlrtY^*Y^9{54gXB@Wt)!KVR)_Nh&#~L(-ZzEn84U(9 za`bd<(7cL$74)x(!lW<04fl<*+Ev zVyu9lNWxM|jXX%Kk)C45J?gTPyDbY@K4l!PXY9l*tlP6NZcguSXYg$?81icU-Eg3` z%jbe6aK%BdotVm9>-74E{>--PPOv&O%>0fX&GEZ2H)<{j<)nvwndB`l+HY5T(;6lN zJsW_ISxwGnc=e3+mM+ABI$8kQoo3=#Qnh|ND4rS_uSP{f!<+OD&(vi@t**pN>K(E< zgtCw|A{RB6YV;0C;OQ9#4QLM$PnXATzzV3oNAd8wcT4_x8%CQU-P;^an@VzchR|LB*!&e(2O}nu^%*sTP|mo z=ynoFlZ3Wl^F#NiIA80JL&p!s9$04>nZ`86)!(A=&!pabBf#CLefuWYBdtoSj-kiD z>KH&{JIjD<-kYqK+6jdnmwPF&*ji3s(?<`98yQ}Ng0p{Ptj=f*Fq0Zi4j_IPys!oR z+39iQAbnuU>NDDzape!33kG1~@i+X}V@Hxv74~CIAbz7R1wM9-Nt)tM2ZSSN=$$dA z(n%fDa(F|D+ERoSnQS}^+qZKj?I&(z^3@900_TvHs1{xCTA?k1G}&mMk)`3jW=nnO z0b~n9SI?E<12RD z*cRWPclq9T5ofw@%U0aa3Qo^JAz>8jWVbp)Bgr|5}lK#YNy!|-J| zfF*y zgerGeGZTv!2Kk58Th`3(trxEHG6%);fotiMiawI)+y?Nrj}L|)7{2r>#NVm?aQudm zlk^a9DD$v#h&_2F^gSbkZA^1wqt}7$lAtl=9+7Gd);s*9qg0-tow{UoR;TOA9zf?`aDyxsf4VlFKCwhS|Kw;&7kUN!YyDmpt%%%+0i^sbmvJ}1b_ zOM>TRN<q<>BC{M)zn z+e!7OpA^Uq{r|-Q272>=6kq@mJ2TLI$U^^jPpP7)xTuocFA6ZNfQ_Z$|Ahz4$_jLa z{TCiE;AaZ@zmhe7ai2ASjMe`W5BN6?_8%oCw*QSc`;QL)t@_{Dx2&xH;_b2m7xLe_ z|7`fn7%Ysef9SF?{j2bwda|(nbM=22;y-oxhm3{guLR|vjVvro{~Y7bJ!WBL_`Tt* zz$E1?Z2vYnW_BP=n3?4-W3aRSi>b`Q%@C&*27vc7|nf@;>*#BT%e>l~Dn>Q@r zGTwujI1oci&Yayz!rG98NJ->Uv7?n zmh@gq)-N~1mmAT`P3YzJQ`z#R^i##)rS!9}CL=S$OXcSnKik|XVc!Yb8|l4}_$h#E zF*3Gsr38jKUPNX#)Dd{W85rmRj11KD0CIYIa-iOO8^aeB*#AKV zc-(}clYtR%cpy)jNJfE36=;EUOmx6wBjlX)>>b^SWQ9Zof&OmbF%-bbD4IC}S^X6J zz;h@}jffP@EcA$IiT+Zf{7Y=Z6*ygBj0C_A3tZE`UPM2iMAk&VWDG!)|EG-p#i9Q4 z`imZb31~Y1kTJXv!T%v+1O{^cA!B1@1A65DuE)*_B)b1Y#>UF>Z!#bs{-5&$u+g&t ze}exu9wXxm5&rKo2DX=<%fHL$+1TlU2KBeT^z5v_h`>K(Y|Q_*4tDl`TN?u_GcaZU z-}?e(O#j>$2H-F4Kj*^0#t6(!@b|vJbO6BLt$&lT|8sr-;Bf!^JOFF}mVdqnBY^GS z_Jt9^{&GUX%j?h2mywZy2^gLEhl~}N8RAcwqrDz5KZHH(ODsge%-slRS48jRZEPHY z=_LO6u8Uh6+Yr5M$IBOATnKn@0vD4ZtC2B)-I$(1j|srSs>fiYZ=}bn&&GQz;?<&CmbijC>gCp?E@YA9hnSh@^EE$=wj0o)i2kir$CIA2c diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testPNG.png b/solr/contrib/morphlines-core/src/test-files/test-documents/testPNG.png deleted file mode 100644 index afbcb5f738863e20949365a7ffc801cf89dbd5a6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 17041 zcmX7O2Q*yI`~EJA)t3lMlwE`sqDAz!tBaoKz4scuCpHMH*XSkT6E%9G_XM$sAc)m_ z@6rFh|KB}l?wtFcnR)M=JM+$SpZ9sMrJ+Pl!axE50LWF8<#q0BoBQ$tjCX%04s2Ms zuYj+0lu&>wEaT??Y#z#nuK@tk2mdQTKvoX)-bm!FqOL&nhk%ffnh%^c;d^iT?X6(o zE$8m)YU}0=P;<1lb$e}V1M_x=c{|v`kRF!S_ti^xJ8wTrFI$+RyN{cVrMIKI8w`Pb zsmKMpf3)?2xx3lI+`V9~?)Ocs-QB#s9Ibr3?+q^hHyq5;-pkhZe-2*r!eB3LZU6t+ z8k(0bjxM$^8^_oG*XiE1pQE?KeFMkWFh6%MXPDjnIF>dxj{kFN=>l_fyEnN0 z-z#|8+FN?rIJ(*2JM{?ga*?PTpaP;8i_lf%h0Kfn$^2k^IGu|yKb#4Pb zOzAh9BUAIYxTy{&ErM`%U7}C>OFUn^5@-MJ>$i};%?`&{`H-SzOIXqzb=ClN4F~$NB@q^JHmpt-}lq7Fy~5z z1|HsBFXo%z+Oy*+Wuzo|x~G*!DBY{4SVFu#d~TdVzO{&oO;@Ol<&@QL3fMW^LG&y zE2E+|3M&_qltXjPM&^P0%ZV^_9lIxX=e$KU76JeDL<#y%RDD`0f!T^-FGHkm8T#nM zddhAi9w0BDg~M~B#{6M6!U_Z1Cw%&Re!Fm-d z`)c}4Iz|ixeO_W8G(M({7Y9hxCv(4b>}$z=LdbXA7p;To|GFOt9*xo#v(G(d25_b6 zaK!Y~{@3P*4H~0h&g3VAe%>u9f|KL_5|)C~Tq`Z%tAXL`ZLjzCzr1!3`jEn&seV+( z++3vzRZl5!H%>sU@A?an<3j~nXLrk*&zMPh`APSu-YN3u{~RiUp0)72$6xoZGdFE# z(X1Ueyj|I3TMkYKxcZn*fFcmYy)cD&smnck@H;I5xY{rg_wLyB%H?);^TuR(_QV#4 zR3@$5NYO@yc^CnlR0RMW8;n0hJ9u5436se^`-6y!tHA|vr8UIsU?6%kvH&VHE~3N@ zc+x3X^}Btc#kKVLQkzSb^vHus%xl`~?>#{YFnMsmJ`jKn`1pyzs1i{5F125ol0J_A z2#nm$g|egB!GD6i9NJ9z+~d;CI|At;bbT~d85~`9Epk{YyhW)wXv{sQ6UMY}i0VsGa&GHx$0O(vCQ}e3nn2|p& z*e0@w@@*khv}Ts{2>du=Qy!*OFMdNe;G)E1!H83I=oWvmlyBPf^nWs zoH_?l(~zx~r&Uf@;wzurelZdl&y7_L^pj_5T)UB5UdYLNUquO~gzi48U#}`75Vpu! zJz$rV?C}MAbWykkCZcYcDC|%_Y!~ARi79-fUU$=|d~5@y;ZP&*iRN*epU?0K@b~(o zdk(rRS=vN2%B@28i0*Dfq3Yv3Q6s`FdrPeI`&$>m-B)lGt8!OUVfE*TLcGWo*sF1T zN3LT~nYNnFmRZP!mn8jwsv4bPPU5uW9nm_?{T^G8k_98d#PI&JUMC_*^zp~mcJn=9 zWGZ(3{i|p#kYqQc%6GUYs#J$w;KzmJ#q|8`X^&E=N{;_vS&3ALhfypqhsw0K3e=yH zIqWj{s#HNb*9Z=0o87)D?s?ZR8OSCPk9V}PnZnn8DG2_hq~a76Do!5+&OJRYzE}&L z+Idp_GdS>z@jjt+PF`N?KfAPv)0^Ev@wm4_vJNVLj`VURMT@n(8~*S()AENKBu6PK zGbBbRRDH_IQxnN($^$YiTx=G~3FrRwMB5R)bp7l5QZkzI&aCe)jQ@i2d{P`;AV3I5 zL&)$Hx4f(^94c$EIxCzhqri$C7|$sbN!bH{gyURey@fB=`t(z@75QiO&Pm^Hm&Cta z2yF;H`?qW(gje!FRnIn;ogQ`8j#v#0@%HXd3p+n@HTCK9eVNv>xwz9B4AXV5hebn| z%hccVH@COvYu3&=iR5E&eC|A!o_X%Vw$fRROr`Ad{A4}gM1HKj{W={FdQ(&)$REWY z!x|%?Gb0!S3q!Cn=UCz$# zPG6w4pHyOsN9AM&1i>G4NbZK_Z}=pU0P1j0!wDL}UWfD*sAHvqVOpwI6g^1K(KO=< zfpMUbsI2#K6jn_Z8tzyBMvTyzA*4}q^z}KVMKw>mT$Vf=_1Qf*^vk1dF*R?Rzq+m7 zo4l&p|6YuK?1=hsQ03NSvQbs>-7y`bwCS=!#)?sf&$?NtbOHo%FKJ6v;(Mq*sbs;` z{zNE66-(M-EHG{7Moy}qU++cy^D| zZr~u3^7qgA2wcY-7ma5lC=f}f#*>tf7F>CgY!WR!G~RMr<~YMn%*9H?8HL+qk88g; za!=vsY2033ey~eif?idrmY*eWF(qzG(g{+_^=T7oDKe;RY-$p*Z5Ub~4|#glV*72* z3Vk#X9qH{UG|}kX%#7T}TWQ@u$FO-H9^h&7G-C}*E9z@kZ}Uc&< zP3sg4ey*2N{1dgp1zA@J7&-Uht+h?5y6gUrO`!UHO z^X`+6REYtw7{@H7;|Kv1F2I@$F=&L(E(I0;{k)Mk2qRz#pD_98h@!7i-NPVaLMDds zew5V;PWq$bH7WP?r%Fq|({}!_v@JN6%Na$0!Qr%`j9Ps`>pqNq8T$I=t7{wnzMg^2 zPY1^k`NIid+f0CU@9wrAG{U9*&-`3b*(8%a{5@-W$!DU*|F)M?#{GqyIxfXo`s&1v z^=ush-NEgdZX&h=SI1GMG@oJ>x~=5FdlDM-#i`KckFPA1xr$?#*#O}Uli;v1OFf7P~` z{r%HD16IBs;rtsj&CcZ|_lUcgp9511Vam;=&|Mbb*CXAi!nH+!H4-R``CXhe zlt)iOaw4y4$%|p?$$g7>)<55Ee)PqGn3c$Yh>E01BWEe=Ua_er6N+xJc_FF(A4cmK zrq$)u7UyndNOB`0t9CxE1p%&}$4PuD7VULS4U4TN5^c=t^<1jz6w`WX}d2>Spt00tgYL4qr2|LA+%9(ihhf znG2Q>c00qo0ML9}ut3&Iu_OWZiG8+SGP_%gBL1_18>0Z)PJoTUFH?0v*|$}{Qz_$ktu!Nu8EKC|_@hHU8D_A;ZN zrB@*YOoO!ulEZ;bWn->)D5_&l&Zxnx-6{~EM8zP@m04kX*jH=s(wAOIyOZDckN86l zOX_!C8z=lb``cc#6>fgu8n1>dO^Q|m(^(dlJVusW`)U8SP?mRNTbEB2Xajm$)-LQ> z&Rdc2Ph_~KBHOE57fPDRqFxYZSGWB=!piC@Tt9HAR)CbverI%GyF6RJdL9^)Ec)VC z0arPKBqG{JU6a3Y((zLczRrI)zTbliQ_pbp)~TC7|Hu-vMSi3Zvq$|J2Ie= z<4#Kr8&NP4zZO=@q|;Ezn5Nlo@t{CKr`f6}B-_BQR1osp^sew#?XLego<${X(^OMt z|9xPYp`uyw>3QRushB>y=r-%=Xgv3J$2Df1n}>vm4ZfBl zQxSZNz`33&N|>Ep%}J9%(dkWBi1>P~ke3G5(!UQp8BcRW)a@%>x$_bP3mFS(fZXdu zDL3tdMSQ6U(Yd1kfOPO9Pj^Hy<8>TPJ7=ZN&3R zrgsBB*nK#+_=bClIdYs7;$HnF%065bbFA`nA#@y)#j}6zqa8&hC7Q*sIxe|qyjbYI z2ZPMunNT__8|W>1TQ?r!x=3|fh>bhk9HR&`x57Hd~wP9?p5Vh zS3Ap^$wJI4{Q2>hjmC`3u1yj+?_V?DHJOjwZ98|)=WKGg`M3r?Zq&=!-k)t8j$R&t zucn?K`1*vzwlaTx1IBzWo;V5(2zgNVU-|h>K}%uKzXKb{U-1+U`ku>Mw{M1(xq7Kq zT^FoxqMkcEBHH6j!`z)ku{fZx*_RFYEu^cLpwYuLC9mUP;lgg%n@)ZY)NrYF7Z zs=xATz&(1LCSm;X$r<+BCf?M*EE| zyByfsa0L-}^Wg*vM5IpUYjWXYIH7#ke=dY*G!;?#^(?Gj6t;~(UqJ8vq7fgKoCp69USnwXh0bsEyf~v*hNl7h?#tbNYcj=mHTwGevFo zuemJohj#}=cdAbwJQmpZH=LrNma|iDb0G{n+X#3aM$9#?lNMnGz#D4~3F|PObDl6R zUKSEC4e<#6m~&FglKLJslN#hG8O}fStML(naA=nTAR1GvN3JuVo%Oe}5qQ=gz%Xi= z7^8*vrhe;sak{`&Fbi}1pSxIwpp~}iu4Jb+wZwYbbGwBL89O#K2)R{4v&^F}6b1LAncg^xd!YJ@rmG#uz28-6nf zZg9a<2`m|_GRhmBm5L=}rZ*#A3l8>#jF-I`>HnZpGVhA!1D9ClSz@%ImPDRv#P4}L z6v5Me{Uo3o9X#1^e7rYonb6O*7zascFV`Tzt9*``!7s=#9?vS?vgEr6uy zc{-Kim=oVW%wOL!JQlHh6DZ;^o@xsv5Sj9`j2dlkoY$Ew0lp;)dDn zZjtFMEWBAIB3>{aS+L&ruCKZBzCyK;b9tH2;9rw@Tb0^rTGweHz^o(q;_Rro>}(W@ zjk;I_y$sK(!&#FUVBRgDh&M3ryxOMC+rJr^o3kzDg{TH;h}LQ%1>*+u4=QeiRZ4Sdku;G zTS#T+n^?MsIv?`(+F)l%b^NwMVd;zy0@qvSxa{rc`T_j+|Jvy?<}ezYDFD^SPZm6yGu3(L z&0^9u?H??51~#cL{oC5QT)KXBBA2fDE3-0ISpBUV)OS11M>skqbIB|QQD6z498#C3 zdcRbQpnn{j!bbfiy$JxB7EzonafHd2uX0elRHKsbqBA$8F9u~O#d^2ld~zi-H9tZ; zWlJq#t!*8%B`QJ)dIJB^|H`&&c>pLC=4T9|yE3;s(=FRUV@;+qC#F;BLC!l~-%gn0 zIa8YUaZ7jC`8TA~lvk1Nd+rUjR*C$1UJ;3p&^t7A5csE8V4==l=K-No6%72p7!vqS z&V(Xg#_C1HgQ82(f#wqRU)O7wM_oPNXHS2La}(a|^=+4GYL1Msgq{u$AypSg`N>R; zaX8lsCGR z4;psAW}KQr&zG`?R^2@PO{Us>y#yJ|=PH?GuhbhhBRej#=_Z5Z50{CEbSY=b2oy&W z)^)saKC?INt}|6Ul$sQJLsIK=p~j-SkEH+3|4ra5&M@!D8m9ZFbhjdd)LMj`RubTh z7iO_fz~yt;(@&I2gYdQ7?;wA70NGnGTEGf!qhjm=w8S!4|Mr?QkrY+OBbCBE9EP#8 z?Ky;A4{y&l&V}Ayf8#vlGiyK7StA}t0m|_bAbmjiuT% z>z+9B_A*~0rPs7G1K!jED~{~qq27YuF@!2!Uf%w%oAz1?AFd5&>t9XoI(@+n)zVK6 z?TWb1cLX2CqsOlOB|YAFTJZ4ry`Tp5{|Oz%B&`m&J7#zNnwZluEF~lt*b$|C`z5{K zSpWHRQt<5@%Z>7pTbpB`G`Oo$0b!9TFE^W6=o=2jF|0J4_80TVw+hr&B2YiM_d{lX zcLo7!1lEqqIx|->t^Vh?Q(?<>p3=9m`N3QbS!*d~7tOpVym9uY&5=l&#{#t^LF7`z zisSqg94fV$36+g1%H81*#-}BYx|0l<7dr)irYHa;odcUT-)1n$hgHmaLmcmQ7(SWb zF7-9;h`OU)qS~4KyWY7C26mXgFN_@@8((LMle4H40U%uCgb6Rb^yO8IXdib9s~*+z zR(uDE@}~Ldk!Vkav{WPDuH$G+%-vAf;Yek2>@q&dE-aFr={W=aCj$4+t{eYA-|5E~ z_-Q|i5nA+SerE^i&@;V6e56uEQ361dA6k4)Z`&~l2K&R=rt`UiAYLW_r&5&8#_$?1 zxu-PK5YkfZNLz!|D>&5OD^v~6H4_PXBb+SKy7kcRtCM(gJmOdR1oL*pGB2wVm0Wb_B}AgDjyYk zocpp#U_bi8i9IQv;i0`%i4z>A9^&MjO?6*sXHnGZV~-1a5=NL=e%1|@OH`8MZ3G&N z%yc1KeU0A+)0R?^w?3ollcmV>8+;Ub6Wa70Ywg_Tr^j9R@f;hl4b(E)-L~x5Z7g41 zTJTtqqAB}JhyFTI_SJlMDhQYALlaewYHVP2X|C>bdPzn?b5%ljURUIpPo$q_R-hZoj+G? zPLxf6)KJi*eZ;xG$dgy0eKz#mX;PIN9rCB@n=RoIkDcNZa9d;5KH9VXbJOvmmGFB- z?Z9u%)iYRpaN$w3zjAexgsRR(=p*W>&(zvJCeLOKSB8ABGxXE30MYC5eHGan-C*-s zQ6z;E&R-49a3ndeZ%hQkla6C^S9=C_t~|#g@ud#2RlZM)4_yLlm6fOH0NM$iFzwBr zT3W0BQeE1fF#{Ibj{7CH+adsg3jmLv(R2V3G>;7T1W*OOzL;cK1%^Y|dZlj{t9Z9X z+}jTT0MMH#@KVHh&GWO{t-gtD^Z27_|0G7{_^{Kdy9?jbjG^;=Wl1^p!UJpya}-n# z(q|)S=+Aa($I|;=uW?&qIlQUUh&wuRqlVH`}|1HUT6dhne5~f>rV|jnVLL#ynnMrc~vrrqeRiQ40nd;HrQ;082^{;^?hpB zOaF=1BFr+6#I}AVYJJ4U{KQqEGPbDRU|yo?mr%AwNo#pdE%@b%54R}kThAO1E_L)w zyaeSNBAhX0jbIf^|}bq zscC8?KWsfi=I*l>#TgUDMh>_3{*=iA#-y z3<3l29CV7^huluK-kfJRiT&0nieTrz2h3k76=48CxJ}@Jkc!eeohDwSZ_CZnf|Zbz zbkM*K|>gOIeU z@AlSkES>)Ty!DQGhyRB)wxiUUEygHkq|Jwv!B!}Ql&F}5lvKr_?R;fU-J~HP+>ZnfhM(YqC}HxG6c9Wx1U-rm z0v_Q4_TeW0+56KV8Z^e)ag<$J%4sxVn0~t;az|D8`-vL{*x7KZcyM;IBFn5XTiY6N zr4Vf?{2Wta1j=rK=VMy-aKPGG;`JbgzV9PiP|ekFtc>(ZPb6!IpI?ao2~E+$$&zuf7IsEx{>A7)XEr_a8)=z z*XD%-?uQdW0vFy~rJF+lL@+b}2rjsj9d}PU>cj^~g#-LiJ=05FeSLC%@RVQ=0Yt7j=3i_%gfZYUB*S*T0+=9 zer*YvAE3SJy~ia_Kc;U_JS`?e?Eo>$T;v|JG=E-vYHqHSu;e$!Q1UTLUe!C#THWxn zFy%`J)|%mTuO6*eoU`=rLr|Ug(J;_`gnbhQfulQ2%ftLvJ05Q~`5Lov9v{cXDs^UC zIvD(jtn$LHIPN?&uf{n)XJq7!A6ZYH@8wZmE)Moy`wc3LrruIV!ww?@So#hbEq3df zR!QYQjfDzHGiY*R--Q0BRBXMB;WuXVUOCvI^0jI)+ducC!Lt}%T%1ix1Ur;E4%RY03aJ39z{X#-Z8=R4>?J7Ex`ypfLL!y zzD0xE`=G2e>T60^F!*Nk;$r<|>L|G4qzu=wUUtmv-A3VmBB^nhaC?|g$HNr&-PI7O7JAUbl4m9< z5B0JxcM|!)5=3Nw(G~iVXDupaK#~<${et>&CBG-*BO<*1vN<J2u6FLx74-{8-?M+ z=!ObA&|l;5__&ScO&|RS6u^qHb-GPW`4Drh#l7s-5nKw^l=Migkz~5il#v#f9y`x$uV~4va-MT01l<2Hk`Z&49KWLfU4Lqm>d-|vM-`)N?^gdft~2Y6 zy63m*qGtD$3+->lYbs}J+4(`36#j64A8Ku!P3*_Cw#CZ9+2zF1<=nB-NbTXys)&r3 zpt3>R{5<;Qlbu^TVo8l`t7p-;A6-F|;XYWv$}+p1C98rQWK>VgevwGN^?X4tfDnep z;RAU7CE@|!LI4Q>d5GnH+qo#qNafn3mYPR0CRZP1jiG1%EvohE1+^%kjo*w??*b@} zH$CaB{~YmxtnhLHQ4`-gP_xp8uQ0^j?;+w3Ko07F357?(Ydr1ZDxgCL_W2$mO_Jh(g^YBZb(sl%DE(T>83 z1Ri5{{dDKaRkI6cA3ZZDm-0Y^RD>nOrE*yq&zbbgh?oJCIU->h^ttDda7q6<6Yxr_B}bU4(AoX8QA5$;eZdV5VoS$QH? z{f<5P1)Q)P5sQy5cs&V^_(I7VB}@9HvoLMY0+mV)W``E^1?RCW*mP5jtYbcnFWoQJxZ;X!f&+bp^Vcm?9U?&PxqhIRXz3izH;}-8l~PQhUw-MN zv&zFb0D!^di?Qzk(h}k={wD{(E-+FF@fa*xy5vog(*^Cgy-XnVq1Rf~u^a~V_edIv!%_XbKXOcR{X}LfS`>u0Z6b@#6e4pxm^gciA zD0m^O=I2T88(6_hN*-1UJl4|%pdTT1#Z#UBKxYK{kN9_wwdWK?Ce9jE5tD2=@FJ1& zN7T(-G`UXxL1taxZ=KNHl-ybV&6h7S6WdJY6khkXsxBTZ|1irmuuC81C<#N zLD!1sKLV;JEaGC=oG_I!Sp)+@Qa^vysR&QGMc+$9bZFUCG6Tp`A3!zYQhsx?4OOVUFF~Q+jt8jh`5B>!!wn5>SZlsh$HmGG*fkc1=zJlDLaw_WF!9L>Mfgvtq8m) zO9Wdl(|lybT0y)Gu`_1Bul8mI2QI=1C$oYBh4L^dqF!dm{kj1J_@&s!*ou~S8W9%% zhrv$Uc_$L=2#L3+l)h@14?H-~&Qe!O>ABaRVUGS9zk#r)T4I2JObe6)v~Os(I`r%~ z?5PjIdtVBRWQU1e;TM`;(<4X0NEX}lKE;cM8dKBVt&)_PC&dU`MeYR4(np~8lh4Bo z33jvj+WYMSeUF%aWyNigdvxO`jb_dQ70EoN4q`xH0*{S2f@P5ZQa?zPQVf~YW7Z6m zmp?0Tis2XjvjrP`SK0PQ!LrPUC>$Rmx6*pdYycN!{rblw-$e3vzn1)Sh-D|UWT;Rs z7SIcxIlVEijN)hJ8havOfqLuEDk_<$UWc$yQi@7d=%gn>ESSkx3L2xuxziG?e z(U6Q@x9r=?rhf-y)m)7ai<_y+{iPd9e*6vr)M~4-`mk zz*3)BhG5WUe93@BSFkbD#{l~}*|q^n9BWc8+>25N_c>oUs?!)%B$U6loT^jNV3CSk zD2^tCW0l6)KR~Eia!h&$x*f!_@^bTJTK%utEzmHG00>vZ?~6qYNO`0ZrV{XgyAg3q zJ99%F?XB02Bia1WXNoy;AVL$G$uDmx$0Oe9=z$k2D;cWleX+7T{Jxn6gCL5x;qd!B zk)83bVv7Co1-u@xDov@eR=WOg;FuUsR%Ib6_BdeH+o&vFA5&7ykthAj9R`m zgcwS-hMv-)|C2N$izGsGjni9G($aZm5dv*feH-erVwzCWAZY|wK(zpy=`=|n;uMP) zNeAZHn>VYfp11WhdYCQB`iG(P;q8t-iy{|?^D~gcqp4wD?jlp|v z5DAS0q6&S=TnB@N1p4_&87lxTn6y`Gl{}2Hs1GgYZ4CAJSP5$(KaHS!b&%pva4QT< z{J3Hg0qgy)Sg$uKHa08s&i@4$rI+#r%R{3?UpUdA*xq z|IY8qpRS}h&+g*KPR9QEo*NGr6&8K z8`1GrGN7#bCtQ?6HxPPJ(BVi+9^Yfx+*DgT`HqXot!S zkjgM5_G=-x04Y#@@~h>BySq6}-)Z~Tt*b%&`u32!%f6BFT1XE7c3@_`B48(Uzv{5x zNwH1lp*LaVUJAuOzQQ%4$9dK5>JH_z1c$7;h3Jx(xJJXZ1hI6$*i`Kq<$iG2Pj2?- zr5?@me~rB#_+868?e6HPoEG3ozKmO2Ah89PCU$`ps9PRJ4?%&OADQqdy0mvj7L zna6PD;+ciG+K0)w@%X~U zE9h$YXshfqH~35H+jsXtX@9%KuxiiTMSqII+bJ=`csL<%cs*8BC3OStHFK(+2$SbW zL6L7&prtLlNA~8|RQvD72m!xDcK;nvaTHDx25mE*n7!zEbDpZb4d??%uzX4MQ{@JW z>zKICMso6#EHf%9W<61MfPRG2)5le6PkLToP_p2wl#IU?+qvKCtd;*pn~>-9fzI>#u6oQXgl6SoFBT zNPxC<-n{^ch^!!bih#x6c(+Nn=IZ2_MZWlOb}Gy)y$QQzgK6+KYDs*YkQXcy(-W~g zp3rgpp=NG(_yww%oheFN-mteDG0fp^emi`2W<=h+PehAWC0}g^uWJ&Y(FE)hgGiL% zAX+G?V|gQgV%xMl&cUeum;^(HqAW?r*^13951CD z4>oyzJ{S$q{I(>3=KWAxi<@q3M) z{AqChoL2KUaGlJ_*DmlM&j2lse)}ofLOgd=Hf^(jew zy%#GfN(o{7*^*VWSIt#JtGX}UxTxer7Cac@)_F)x*2C8QZYACdy^vFS9{Zkv!)kdo zFn5K>h@!>nW@&hYuC$mZ7Nl<}o1=hjfn_dm<})GzaFB}7z_^Xzz_Q23RPyeE)hScI zRY%Rw&x~xRx!8(p-L>(OtZ3{rF)Lc?oc84t6o#Ee(GZ)QU#J6(m%-vvl23*Gisl8@ z6vq+yg1u_8I{Q>CPLvi9j^e|m!@7|rp&C6wboe=(Tz-G@d_T+WfqcsNal*1F_V|$R zb}fNP2ogYfW8;+<*N=ooX{X1PHf5N5X#){ubHAFUtdZf?43I*iPYN#wTUz`kxi&^w zHSuX>LJo{+9v27mn!r^Yld^5rGpai-R*pwHu9w_XUyroEMz_h)25Ze~fcFXB_ozTE zE%0~4wVn&55<;k4U-7lsWv*-rbPc$KqJHh%#O4#A;elGwOs`yjcx})521PMF(18{d zrw$_J(Btg%CJuA_-rUi;lKnVJ*YCMU7(q%`Nhu##;#~W_-h6O_^Po~}O5K?MUQUWm zd)U?M;szgKaE{(RzT_DA7u0bxf4ic2r$xjFXyBW-Pf^K=zvqRJ%7t3G;)7z&fu^i4 z5P6P|e_1RQvVAi*3Eo-ygAA&>F3H#~`KIXPmb5q1UwBt7FEJap^Ocg{O!saEJ6W1- zQ#WlAS?5$?8p%Ej^h#<);@AViC13HrX6Xcyfg7z1nFosLlY8jh+HxDt%1kx{q_5jk zA)9j?v*qXOrI?`$&nn%0fnrBh+f+xsA1^W(?^D)Eo)PC+zkV1-!knb=x33==0AVi+dA5=4;FPjVk~HrD7$ng4qZag#p-ak zNs+FL6-hh}8#7JHquMHaqOi;5FrPUJg&Ap#v{olizeTQBKb0zf{Pu+I)vcyIO{h)xaH`9^sVA z9P?HpyNIq+fh2vK#ED{N0cUaxy^{Jhtgr zx{j8hj!#^rm>(pD<1g_h6Ev(pXeU!Kzym=l&0S>W!xO6$7qM)HO3e1i_wDXBaDjNE z7SGbffjaB*_Y?+S9IJ)lNsVKQoTc=sbzA>zTY@%-+Wc=%cUD%8hRx5hGN)i`i@Lte zof8%Bvm3s!VtVXg^K|_fn=ExJ~Cn$MNGb!iTJr5-PR zwPfz+dOS1|7S9C(by;Hv%gkQ&ea!jkE^R`g{!;Js%y-Dqu|HZb(0~39*1)Ew%y$=m zC;)8dYjRvBpfU85;II9I!Uny{qdyPOi{&;L|DrDrUl!04eBtdal#_dA%v=y%sja5$?!x-D!sf`q*CU41DlwLLv7y{nz8sKQ-`vwqmk-x$*ku&bb>&t7?0)F~h(9U$h`+;@91 zcK8zXgR;59S%lKq3YA)z_4^gmy4-RKHn{NArwgiXzjNC^D8L1cQI?-hWZU_)v<2UcjoI5LNN->I%+50~ z75Ulf2GMZ6SNL#h8Wwm)3}EmR8Jj-OkYVD>b8@ZUqPL_{ke39SXgco@Km4f_MMcfX z->t$e@9ND=&<~|crPJN~xq%vupyU$lG|W5P2)kYkEAy(o4`>N~cOjxzzbPqW>o|8C zvxm=))}KEWIHD&St8WsOpJ*7^yf_NJvj{l*Hy#+uXw{7o9>&r?7we@j6%i?y%pkLN z)@j;oJ7w;}thV?>-QCFC9`B6Vo5{4c{~H?ak%0_IFqTAdb_Ac#{iz>Ytm6xNeYRAt zI%w&V>N+R-b#~WnN-rqLPKzeZz51?IG9*iq)B&bu39x@E_x0s+*U5m@txe1L&+@NI zzIG`*0@76J{oAwunW`xskRe<|xC?B;gym?q$1b4Sd+#lCj%`v@VyL1;Eih5oMH+#} zM0IlAu%bKw=bP88+tbAA$`=CSM?>>r{?}3K$Lpm%BU^iOKGaMbUVQnwDS>UbiR%-w znjQUJkJYm5LPAboth=-A37}kA&L|wTR8ko^J5w=%px!MNilO43!r9Qf9u}@ogJiNj zizoGMZSH=~7+2|soFg5hID@OifwG?^JVTJj3x&Jx-yo@SONv@Zv67#vX8g%w68|=c zWBLOnpfSs17oJn6b6XVlDCNzdIC2~L-@0AVT7ej$#~65t%5!Zq&&Zc*Ils*DA3>(< zY~C8KzgehAwl!5Kyf1lm{)>0()F|&2cG2N0d@lWv4Uk0JdSd_A%eX)@W z*RF$>DhLuUo5E8|uNWva=N1&C#d5DyE*f`g&}hrkRa#%yj4v^)?w4m>S*nm{aDclfaOhw}l zV+sX>Hh753i96Pa++Qi6ZWJ3vh+VxNdg#(xR;Jk^Se=$#M*0yKHKmUg=zEZZ979i zau_PANWfCPoITm<$H zGFrD4c{A&n;oL74-?k6o!R)in+ORB^ZcoHyPVb z(vQ%VYR=!@1hKS~i)4gdZ2S2QGXZ-3fQG=6yaWZ|cy|x~&U!wq;4ttaiC%CNYrx{F zczyo3w!DxxB@>66(EgG@8!-0gS_txqk|5AOz9|C>l`B+U0oeHCX-h#EK|)w2gCd%m zsBl>CGsSUuA`yz8HI+KNSFjUa8TQSPS<*V(^Y{4_T@+^m*N}Tv^$1qxF-%l$*G(de znJSWB@x2FarK0HLK}WK1aHRbRHfJu6IaDlP%564sa<2J8mYz~Ti>6=Q?|iYHr?%(l zgCFu#^+V$<8C~@)S=#BCn+Q2A5F~EG>pPoE%AoMWaF4u5rx$E(Z>^KZ?i@E(wvhHL zQn2jj4|ZC5@_1W}2Mqb~B3@_Zt!TcN;d$W86~o%ad!2r_i@4N}*rT3_+2WM%P)lsriSZ+o zDKVM{UBsd)@s=>zWWWdQt^umx<6%;@XZF{D(o*R3Nscp=Ud-TzpD(3{s>A^0mpwKqhCF7Xp1F7 z@Y&Ys%mn2<`&8eQB*+dLH&V!NjrHnQ28~PPb@gRCca^VNO*hOAZ&}v4Vl}W<^2?)& zC!eC!vWsx>e4zK~HMN{q>mQeIMp>k#u<_ zttZE39g7;DB!Kr!lyRP2E0|cTcMNT zYm4_O_p_N1G6q=4WBMJo(`s%UgojI9n>=p)+h=_z^1fL12$l`wTcA;SbdN`JC(=h# z-+$I!^9Ws0ORTL3c)I%0a6bEYuZ>?ze_-VV1^nZ~1!#)jWZ(z;oITai?R%KmZ&Ra+{MS4Y&Ck(0Yd^&2oh|GAx-L9?~uaZs(g)E;kux))y0aml=mGK&+Rqx zo|n6@L=1D+II=*f>!wdws?FP4+W#2m7}!e*0+^{dpC#k13LcSxS9DEJOf%@}Gv8bO zabxz4)?zV{P61o~Toy&Q?|y*2efA7p`0Q2qIU9ass005%f622!t4M1UR*BcZg93pJ zFCv!(>fkAY328qI>Vf+S47UHX3Fyy#3{|oZ)*d#X!IcHp{-~AYvz2%B340Ibv-{3D zq=Yjr{(TiIqR+zXH|Hz<;c0x{pTftE>`?XBm;VNo#y9_R67iWyDuUVe)3z@c^VSiH zzdi@iyc~B^dPdFT#>;!zzRak(b>sUi{o>?$=c4B(1dr%vsm)C^znuxm+W-`zXw07o zBer@Ct*G%4GiJ(DNu}~t{x0=Zq&PiK-N?7uwI+> zU7!c5DV)UC65`qe7AO?E663oKVk^7{sILHwiU-xRpW)90hdsYZN?&ZACQNcY5#`$$ z_0=_FX3EF6MP{GwP87VFYB2G@uJ{J|H8s~sp9#CQAQPOvnJ|xJ92~8_c$=gmvHi3d z!|(F&Gv`p$@acHz2xMaHd&)8&H9Tjit?kp9#3zxt#<^xcH3LOzU?f*R6}1c^@T)9-T(T|HvDIQxko{5HtvSFQ3A$lGX%~B zp)NWzEAvcNKm!x0ZCe!6ZuJmdh{vBSbkg@qAqX00ApWL_t(XDKgHy_sK=0eJ&~#R5`6Z3V;v+TI9@IkNlL1 z$e2JB-(0^d^L+x!r2+GZB1y`3U6Z^)L~!6Vh1p~t2_%syc*rE9*awXejxO%n-Mgzx z`0|Tqt?y*PwW4~oRZ)~#h-~>bh zrcBh5aR6*=PY_=0zd#6#X+}K+;|n&+irH$tNiim$fFmie(ILeo1uye5M5;$su(}jk zaF#=+2W5Q#LSstN2g(Q#b~s;%ET7lY)8$9n=%E|EPh~MlgBp!W0S$vDCOdojbhBBx zfX7yGk)t&1&G$?&CU%qRBt{PukDXRJBIiqy0Thz6dNK{pIqOE-QNc4IP)I{(2*PDG d%|-5P_X$ch&5EKw;X^>V@5T)zR z1)HOr?SA(@=e^H!&v(B)d48<9W{lZmj5+4~ukNmMlUZe?Q?N5m1cm@RJVJvZ{}he@ z`~q;l1>nM9BES#59v&SXL7_t6MG)W$00{cW{U6W+2cVApV7#C)Lkpa7r*paP%< zpaGx-paY->U;wxXzzDzuzzlE+fCYdRfDM2hfCGRNfD3>dfCu0*051R^06%~Lz!d;N z03iTj01*ID05Je@fU5x40ImZ_07wGd0FVNZ29N=e1&{-f2T%Y|1W*D{22cS|1yBRH z37`&e3qS)v6F>_<8$bu(Hh?aG9)Lc80e~TZ5r8p(34kep8Gt!}1%M@h6@WE>4S+3x z9e_Q61Arrd6M!>-3xF$t8-P222Y?py;k;0NFjpaLk_ft*yJ<+kAIc$@=7 z{7rWr%$U<@4hQn!Mg|zCDR!N^vj{7Iyp%rY!VdI^3uvv$amY#9U;Sr5iam1JEB9zz_BJpE;;}L=bt}2Y;6C=kR9mf;)2(g7k;B{(JS|2j!oZ z{de0Bl{^Vm8aSvIPWulEf`;XE{X_lqSL+|*5vS`P;vvxbho1kq|1)|Z2M6K`U=1IL zf;B`#1$WGaU|<~?AM^7e5Gw{vkYkB(bhHY2!EA>>1$n^)g4QxV)dx6`8O)^%$L};$ z-SAL8cpwE%;@1%L!6JnRRH6487%r0k@f@Ur<={Xo?4%Ar8vL#f6i^w!L9~8V2S_GC zbpY96RL3m|0ks?fl?u90j)APu(aTh ziJTBSeW!w21I7Fne^Lj8f4xJ!2!kR1>;0rX`hYUj7sr{Q1i;yYFcI*CN;`4T=$xj2 zM&`IvK@>D9$7GNFc!2t8dOVo_FQ^dzM1|ldDtJFp!TpH}E+9vXs0|p4k}Luu0pKtY zZ=r@U4jQ)Z0ACHL6r@3DU$e7yHnDZqQ}eJlak|asZeyJgQ)Ao2fhn;Ce}E!xb*^ne zi6CSJ)5qEea{&bx&jzC(BO8y-Pj93R*BP~2I+O0BQ>&{Js_t@GjBHuFRlig~58cxA zP}}Ot=%O-u0WRw{L0S9_x)%#}b|WjCv}j+VKD0d<*w4M3f`$+yLqc1R`2_K?PLi@2 zhbTw(E4x$_8an@uYIEN*^}8|WXKyLp#x`edQ0Vt+M9h>&wA)R$wrkZUa5drxOu zj7_jEn)?yU3>CtnAU0jK$&2`}X>c8ZOjmNKTpz(Z_=Jw_KU~@SXnDo+Ah|7%k}&R~ z)%?A5MpKrz0des7JEIz51+9DHyp=lpHqgZUV>2VH(REqCOr*YWq8b`II%87lArVoFk#uJ1}L?+$FjU9g9L%Gv7r(zJdP z+bgKNzewKVPb5U#@%TdFfZ#yCr4sMuTl<1b@_-yNOZ(0SGXd+5;<(zn< zdsaH0&t1JuarFtyL0nvG5Vk%4sk^ znbNbD25XB{t47uP&j*g!R*zO=q;ng8{LJoApWd+^?9CW2A)Z_qVt=&^aYG{MmLYwS zXL!Y3d^p4Jql#*zPgdsQ@amd+aBqBnaHT(3JGrw_A$+z~rfnHnv84X3`J}rp5lahw zWNbOrD?_4v?B#HlJTmTTjzj}q6a=~REU}ilZRc+~ikp72jFVcTuoBobH|I0UkBDx7 zNje}}`N7*Dvgoq=+;~2XqFyAC#G}|Ekf&8JEzZVNg4B+JLekTbUy+@pqi%c$pX#7^ zdn`*uIGi@{p**gRBHi@xb((w2TG#3jj0B(iTJ|%iCM`HNH6wkx#kuJ5MQoVIjEU;5 zve{)N{7k`W(PX@V31ben6l?2U;Vw=_)j6)Vx6fM_R2HHN>6&zKm2}!0dYm25Xg&EQ z>+265uMAEOsV*1Zwl}u9pqU>+yR@H}DY#M< z|AcRWdm96V7h0B{1l75+3}WqBPV(#9TuFLr^zDIQ5{X41RsZbk~! zq#~NE{bG~-MZJA!mTe7F7?1XFFCtVL*ZGL@BUy|S`@ZK%Ys=e}H#%I2Jp9*EgM()U zoc_6I5?`#NmQsVSUF{IM&TzFOT&STCn3a1r4?Zjp6`d5X~PUXLDj7 zQ~A3&p?&UYnI{AK5{C_hn1k}1ffjTHSPR&J&jReC4S@B)0E9u^Z4SDb?Rc+e2I6r+ zt^WO7P|YDxc3S?iW(X)j&mV1e1SOgN)^ezZr!7AXKTCxGBKRLs`MLk8exm}7!;iZB zS^xjsi(vHt_n-8~|2w@%`&;?{ZZEDQ#DO`<4r@K0a~!bV<9P_pSsYkDO6&3Dt;V?y z-miffU=3!3Ay{Y@;9Vw`ieu<0K1?GK|5!JzeT)x-V4Z{y(+6RuuP$;xPiV4TJ$8WUcfo+2d zSgatc3N+gxo6)~R%O8f!DfT;BHvkQ1K*<8o`}b%;U9a>TE!HFMBPtljQ3XdeM-xX0 z2Z;K&>IKn)!E%pzi2tz#5AJwJns+?vAPN;)$6$zndvcr#^1}m;1%v&6v+u)!VIZ!7 zI^e)yYx7XGV90AR;6I$$iIo+yQbK$IY8g~lknY%miVdF7jDzB!aLD=!iL=w49<&pH z@<7}adV($E@e|5(N(*{Id7vl6&7mj6v7zTFM~4_W^d1C68V7g=KsGJZASej+P9yja zgc&ZVMFS9NFw{54AxChipl)#RATO#IjN&U)M<^GR0t(v#)ux24;Q)yX+&2V>r$OcZ zf~3hG+On%WAiT#($ghwUL8hk;;w;5mH3e*Mr?~MajL(f1=>ghRvO~*pf&))0HUzr-+0T&u|&ZF+(hvgVK`^z zJG9>ZHRq4V0g_I?UlY*UJevlbeIULDz5RP@iFz?g6IfNSCx8+d6Zl_f=v3K=@Qj(D zcK^8lt9!sZrbD5L12=e8sy1lu(p_?Xs^HY`4PEDA7;#GZ)0zv8X2;b#MrU#*u0FOe zAE@E}W}&_mP~k+1RR+Fb(@QqN7;xRnlyn+ zrexIn1zjsHT2CrM{5$0fMw<;L?}HTdt2i=WJz3JWG1SM@`MOL!+E2S#>csK@du~rE zt|yiNr=>~YX>3f~Gvwzni$3$dd20EHHxt^iB6)jmHpc8)zpmX8oXquN?Kd9Hau0gX zK8bP1blZ8mGya;&QCfP|Js&$P*Pdl>3$eU=Y9En?1UH&?({$J+s`bEN|IwJVGK(o& z17aA!dcgqe<);nlKkMZG>gDmT#s^1tK(^E1)xcF?J5?19lj_sGzE-dSH{iS%WPulV zb9pq)2qUm%*x=PLQ$d$`bK2Dl={HL|9C3{+t&m5Vo)3QRfV*y7fkkR2-FW?6fnyzW zv2lzl<7QAajk2{`H}_sng7OxR;rsN9Y3TzY=LFiirK5Ml$=hx}ST#xJ&eY*}lc$W! z)^Yy2@XiNdI~{TDc6Qvo95)ASr(X@`-^NQ{x*nNDtIi>4x}Y6nLQS;vj{TQ&edr)E zx)8)v0YKWLvw@+tiK?@QwTTmCCdGU&jUC8AEH+ga?q+`I!+K4bj^H)bCAUR5<^<{m zG4mX}nQ@by^*HtpQJL)f8q3B!2uG7}crP!H=b|7-dj%PZ4LFQDSXAhjqGNbeP2>qV zD0(1vqGjjZRw$(r*k8>Y%*HZL%9wYR37QQP56RM<^vckm~xdH54k~2d9Tn5bVqH6pT{Z2c{TM2WwD*ZXX=M{J|gWw^JFg z^8qJ!KcAZYc7g}OV1IgQ#sWoxD&WGLpeX`_W#d4#rG|omD>&XXL2pnsp$Ub!23_ts zwSXZ&`U=={K-&d%;Qf~ctU(Il&uw1}Amyp;%LH6bfG!>UndR%mjB#T5T0@`$mM?sq zvz9NYZcvv&mahlEe`@(Usq^FGgSKB=zD(Smsg4`&gr#a?WM^wkWo_bWVvVI{ZsB<9 zm$Y+nbTVWvuK5S^HNBJu&;*XxZ4$K11z)bMxJqc}K@L@DxXMj5Sq-p#QN~PA! zInzUsW*T%Gv{CxSdP<0uS8S%jy|c=2h{AQNPeqv6NM=iaYrFOk7yDgF-^f9yOvUZ!hyp8-E_H#?8>$W{f%sQ~a@=zDZ(u(Cw4SZI z3SZW0(Sm*_;dNa?AxdhDEM@DQ^yisiCQ2iAghi}PI=g~Jb<0(}7izg7aPs3Sp%0Dl zqDQ@bIKp|j!a|#-d-Gt|h>PAP$j>dmk4~B%Qu;LO8;U}<&1~hrUulq%&d&cycfIer zl~k>vTo=CymIaOMOubI-i+!?k4F#R;;%BMvcpcM>?06m)cJ_EQh>;xV?eE12HqIS} zrrLqd_@hq2Ij(2W3u2-{cbz$h%Z+KY1SgeuR)gw-dg>fTu94&@*oElIqST;kOUoIx zYcSYMyn4i;A!#SLNVF^Rb#=3v<4H=ndld43JjUH@lg~5ylS|$jKK^lmJyXrGm}AoV z6Ls|~3Ooi+ZgB0=6{utr6U$0J)QTWy%H&CB9FXr4(0^h?;e6GrG2r_no9Zh^ref9l z1v%U?(Y6SS-irPEq2#q4EMxcII(Cn3eSDQ|*)r&!{X&HUM-B5iOBYAq*oV(z^_#^< z3yJ6OeQq6I-Isg1(Nais*%cBXe{8xiswSN~U}gzm)7+EfxQWkFULo5>dJO~ z9JszWrbRV;CGt8(oLRK)zI+XPa+Q!=baIj3>ame_mG%hrq2Onym$pJw%`xucsVW!U zHhs2y+&@$ zSxs`CnwIe%W4&Ni5Esqg*KOF!hfAl;i0f#hMo&(*?)y2_74B*ZS=m=|ifwYC#GsPo zSz1Q9G&k&LZHE3L0Rq##sF#o{t zBxvyf^X{7Ahj~NL{)An#Rg+i!g@XuAD?p;zP&;E?=r@XEvrhy{u$bPM`{*nRt_ACs;D{`C1!6ZkYuP(Z;}bnJ51?MC?jei{Y<9VUaLt&nREq{F zm+al-)%7Hf!CSWYqjCL`p-eUbHXM(6I`PL-9NJ00v+`~jK1HH*9otYjM%FjfauEN_6^yNu+ht_r*6*Wol zQ5%)_VAkWSavV_sNAl~p(&*$gY8^}YzJ`E|mIblQG3{a|%M*D9cO46RaQJ2zWL6IW=FUXN~+LEt2i*t&W|T^lHv zR&k#Ek$gm$gaft05=yPYN4Yx+x_9WR`s(?fchAk~cFq-6=PbQRxwc@()#QQv~z1Uz8v&p)W{Gn+le)&F&l=6~8{kGKeCqca<5`&oy5T4H z3ki2t@jJ`qf8=HdeE)yY7|-&^9~qf}P3A1eJX`G9vHOv`8(5OgjO3}l2^}?VfRw$E z1_#`U(v*t>lqTf6xM!6n$TEZi_s8QxNM(V+GI5}IW`HalNK5@;M>_E#1NA06JRIl? z+sAe=kOa6BRR>A|ItMsN)$#gC)j|4AU4&E}eGq@D>P(t7{iUh{)#XHKlmurF(CNd; z(deHkj*wLtQgF zqKza}iAjk0tMLMLCxp`8%cZ+ge z65pnq<=^h4L%*%Io%-$m=dW!@W}OJ)$lWodHSL$rthMr_y^_9OhWoi0|S8Cefa%Ca2qsSy? z7Q=m`qPhy}0jbquWQ+|l&n8SWzwoz%g^lU(W@cqrlNgG75fiN*cB zr_Xf!Gx^R?GX&3mu44Ifzlq`N^=$KT-wCNd@MWb&5(ISg5h+Bsg6P z$!~s*s*e8|ArXZrcCa~^WM?CTjknQJLGPV+!Q<%9(K0Ff>=omVk9;%RcT5IWRp&+6 zc7wiAur8OG9!k#i*QpC!q!kn2n$=NyK%4PyCuM?hM>B?6r&5LFCIZ5Sj1Q4c@c7jv z#R`(U%orqCUJ6eOh;PZfkuc}*C}Git%lf{=GO~^CGa^9gyr5$!1UGL!K$RSNU?4(^XZYpnEAR6A z9F2lCmJJ?W1fH7TVXF3FO*i zZH{J3w+-F83eTae*eo0w4qAT3nw#<dhh6o zrPINiqgLIKY>p?s^P-??-AkvlQTk?x9(-)1r)4Eu&a$On30?QiC%%Npu1cYJ3d*?| zIi9&RN9->0_$4eB-0_AEBML`#L_{xz(wp42la|_W7O!(|1hBq0Lnd$_*Xb$>DH;D%3`^uHZ=&YV ze6AF@vAAm)o`+#pJYUgKgom1s$LcH)oj360ORG_8NUX+7hU?LqhSrr4?&qIVet(^O zW2MZ@W8OY@FlupirAIqC)$)4$qSrI4hawRj+K^sSSF zEztch$YKSlAlATps)E2Q{#pg$1?f*z5Y^)=Bhc+E=yC@1^Ay9sP)9C=RHo26SHt3IM+hBr(? zxyv$CU)*Hoh7MF=ERGTD(ZEgq9NcBa4RzGa6~N#|u?op)0*o)1ALc>3kz z4?dEswk2NNGKEG3`YM-VdvZ#@uk3e;7N>sTMNCVSPrGNEVb$w9v0n5z#V3a>EAvw2 zla!|3u`M#%LZv7H{sNODLyqL{R_aVwT9hs8Ee77o?wi*0<}!0#$+})f%EnjDjQGN0 zNr$*R>0U%p64ET^iktSdEwa?54!5eNH@=9s42J4aH%aKSzoP^_##_y#6w6p+YWSv* z#_G_5JYUEH!=e*5{w~{o5C0DQ5p_IQw??eNj2&f9tV7o{)l6oufq8Lemd1W4TB_E>sBn+!`%~p}eoB&&lh`Dt zOrs@*->e_pMzo~GmtVxjxP76{B#KXlDrulyH7xbcQ4udrLTmIp)9+j)k+?50B6{3C z1429s+8$b@>sY2J6LY?CP>_7V9jjn5{+2w?omGmmc&r_g?l zMHS1rk0GpuE~F`My_GXawX5XDb6yMk4fNmZ;T4;uk^J3|_m~kkM=AZMEYWxMuYcHxekjJmYI5$= ztY4cUPbx!SZgqO(#Fpf?y()+76jlbvWPOOkAlhEirX4v=Yz$dQ6{xWuf*V`0`{X&D$nn^yHnGLO# zs_u-6spCvPEPbCsSibItj+gg+#QZk$@PI87Qj>DodfvK5;^rGlk}$moYFr~T`Ob(8 zCYpxE8RmAk3S?!M>f9U|%NxwX+Zl8G`U@`J=Cc^)LvPFu9JM1Db3F)W*@hk9ZjaN5 zR!WSeZ)2cjgkRi!)5oG#W_o95XSe42!^&=U#sep=t9MC-y8Mu+MobUEuipM>FGVdw zfBFIN|36iDejL$PN{u}pM^9vQ)~3E5fKS7UQyR%9id#ZETCp%{|#XVkrfS>s=?X4J->{$G=qt*U*fB9z+M|20-c(xabFxpq^au1DD!>iiL;( z-8U0OfHW;ge_;j*o;1;c|G&SIbk@@L3)|X1r$;}2adFnJ_>(2^|L>aNZDNI${|mW($DXof4;ggppHqs6-@1E2Bn!5S@{>$0_9 zuh-$rj^zw~)px4;SRE<>R1`reLzdeI!#>60vw_y#NDNp<2?l+BEH8(!`6+DBFY;zz z<*!z5%1RiLNohgwc;Aj2ZdHZTX7t3WoQvUIGm7kZ zQazhC-)0xd;$XZo;zT>L3?T~O&*|eBLN=ag3tbiwPWw&-TJ%TGFY6>OHvq)|{K5t= z7cDN^B!(LTez0di(wH{Rs6p59h(H1n=Wb+Xi-qgUa8t8uu4!)%J@a)J=Qx|lK6;mE zD7;do#VW(Hp7cx#peYWr876Trp(yzx_9oc63_}jtm+^JEkmx0jYJ)C2p5k~_yw{vJ z5aeZiid_oH)``usE}pj|LyndnXtWTV)P3!fZ&z+zEL_DxNxHo7Dr?uP?i;c$#iyCZ z2@0K!osE~hsvEJwH5ZN8*dO)7t7S7~L`{XU&nuz#F)R;B_LIcWEGWK-_i$3XeSTNe zZ3i5K{&($m&rUTk3ctC(!~4$;-f3R&T?-)P{=x`Pa?HMQ|K=qQ=>9K1pbuT=n2TZ<3)4Ul`jCj;e7b zwn7erx~qm9Uoz$mb!JPu`LOij5YLveT1E%+ptVi7Ek9R!iOT9si)+}K;&rn)dL|Q_ zIR2a3!cATC5M!U2$4|fSePOB@Wad;t2)bDK9W7pWaPZdL7Xl8Zlt*d{4+MmWTeuChvU_n~PqLI==RmRao#&pY>NK2L@g=Yj2_hQmK;*91(cchVWdDq;J%m$O zbz@;#$FG_CfOS;GiY?yh@Iy@u8!_Ac>I#ET^+lq9?fh|}<_+Y3ZU9}y3u4cl<_AZ2 zL8tklt0CYt-$%K(iJ0D%Jul`OdyWrmiqX%UhHR8eVXs|-kKROuD>mhm%|VOZW?T(b$gmQE_(lG*!kbZ~Aah_7>kpg&C( zOOquX1?#R!2U}S6nm0>6+2awZ3vXDgr&yJ z>Ze|18P|P5XDJ+UZ5key0mfAP6W`I%Fr`miejLCrvo{C9tNV z-6$_XJ zU+RXhaem049a@C2OSmZ|Q%>A2)r&k@`X=-`ji2|?#0;&=!P}hVtZOuMbJC+v^@=>g z=tOYL@v7>)nhq1OaHK=xVcZX5s`zMoYsxkby#~KNx+^7zTc|k8VCxmDvheBM-R}m( z?N5d1(cVYC2rCP`b*>l@o)T_6xu|f!a_JK1qwr9?3kbtCGn^Y}q?U0X)fH-6CNC?i z;B3ob@>on4at!IK45A9e-)88hHnqXoM%*`-qF}<|WFeHRs7u}PxTHOJYt6JO>TZ}2Wt?G0%(*nJov8-0rX(Vykj3x7N3s+w^Z z6g#o=^{@O_^{k5cbZX%w?lHt8Rk)hhKU8L?GU{}_G(`rRcx0=h zz;Hh%LIlPWV<3T_Ysy4oM4A0Of`lBcfMilWP?p=Z^*+Y6RHC?7nNKXeC?64X87i!H zw69LodPjY%^>*Fge)DzaVbnt*0pI53j?ld`vO z>-lZZ-hU`sf_xRZgWhvv?%9N{kk>|ke!bsT1^g@4wa1v0y~$F}p%=0cD1vO?aum-R z({)xG5*ujgmSqIvUDR-Tk~yIo8}y313G>$EmFCJxA2`BXba$uS?sM+A;ZNS_d@*+g zx2Y#7J{LAgF_aR&?DY^p36} z$)0G$;n!UJvX{XBMWl{&EGLV$S4iypdo^>ttCg;I?d9Uz?|Of_Tgz&0BD4J9aZ|hf z#iw@nBm58dB37{5PgV0EWMYGCCJa`32Z5B5PYtr{|p(eSR|X7^g4ftY0Vc4nymLy zp?G;Zuc^B*Z}kqin-3)l)9`LgH4Y$z_fz03uccCScA+AXJX;C=M3u^|)rZ*m&U!^l zlHh|cnW+owhm;rnk>-y+2#ld8HikwNH0m43&tJ#=&aOY_!vEB2z+S5H9^%LsSNzU# z!&>`TwzzZHQlDsxwz8ua-%5KFCTqimWuQ|oAWSf+UvjXrC|_+@VNqayX2oJ~-)r@D zZNiPUG1)72in+A1@7mh%oIh-8b@V?9^-iy<<4m`k;#6o{QsMJU4zZV>y=2gI_`#T0 zQ)J8Z(yjOMfmd`iHEP%ymp=rkNmT}!7+-6WGofz4>u7sdKZ5v7HvyivcjD^ZN-Wen zesbugIZIkx6!?0H8MjG$s-Bf-GR>(Z&gJ5owR(Q0(4!=dHI*{cMG8`i7kbefW_*VUwTQnbo$%*K>v6$dPN+HENz zX4w`CRe|BTXO}e*pl;N$=e=XZN7WEznHK6 zTU~Ww*n{M1%}Ak3{+bq-%uOtgl4#PMo>mfCiO=+#vR!a@P&OiFAt$F(Q@#d5k%t(1}3c~@(jmZVftAulco1M_rB<9kN$KNqbNF^YJiMtuyZ zIb>4pMa1{k!z0$}3TQ)+@@?;78Q=Yv3jFg;9IT$0hgCqeLx7Fy^O^xN~gTMf>q3d$`)$yD#fDaenuO< z5!Xjf^8!-TD+`GZsf{+zm!rf~&MSlu3mTW|2S(ZPtq_{U$}-9`(5W!8aV9%rUzCYY z@4l{r_VStS%C`^o*nOmOxt%vsg!NA63YKY!Xd zpUrqy)%lT;04T(N&>ep2{K#8Q_56y`u@-Wc#@R{mgGLyzu>4$`|6xj;UiCo& zKj6F#S|0??03e5h2z_c%NEF=bqH+aaN3pQ86{g{2BpL3mIH7vUZd=u%`k9Y@Mzc2-BE5JKJAjXLm9J`k{dn zo1MLhEr>IUeJMbA> z0iU567Zk7pQhnC4dFQrxqKRO`80(0 zG=!Ii%G}z*)(W%;deBgr+F47T1f2S4*hNZV6o`|5uxZIhecL|(z573Wl*l`#U7;EJ zD3R3uwFFknUV{K!OF;~ZONcCYa7yv2BU@!1uB09cTbRYbunJj3i@Pe*RvI;{TCOu} zC)r)^An_!u_nX)yRuB%(%JQ(Gxi7K>VpgZv0igaxG)|hJyIb1gcgGyTT>HF}wT@3dD=f$TwD{ z8s#u;p6T}ABITUJ#o;Y&B{;q{E-o{GAm18-Gh5!b!*B48Ym+O}S6|UlJ4ukFo}`;4 zdD@zw`Pmb(u}?8+N6&l)HRW%VJj8$TJ?HAWIo56a&8~zpVlNqVxmi0S>H?Zol;IDK zU#lT={U4jThB``61Xg_&IKrg{DUPr5K_4cPGw`r;asKTqMJF%EjG*GCyd5bgZr!a# zzfy0-^SolM=wVtT-{Wx9q*jXujf>oE+Ih9Nx5!7WXiyWA)a#Z*g}<48sw44otSLgW zOfJ)DWBQ`E$zd2*5Z8X6$yay$_C3z>sMa`C`eB0AqgF~P|Mq2eO^a6chGL1NEzg`n2A%1F3Foi3mM-TnM`Wj^ zN)w4=;x~&QR!xsE@QfqLlbOOXxKlowbtNr8G90@r*eKHGSowx=C=JV?aE3kqrf5%tL>FMLlWX_1!t;uO?9-Bqh0h3JTtg^G2QG#4 zJ^~)hpTiOkU)9cEgU#sn+=|Ez3>3yGQ`jA%Q`4)B&bdTz*+CcQUcywSPQv4jyN?1M z&DtRJGaU(w=P3Jf&r77PFE!3vdbi&ENTwU=(=}u-w2br305vX#N-B6DWYkxEu9N!! z|0C67yHqCAW+F<15Et2Me3p5*!dQ5jKonw(X_Uyeh|lzxS}UId&k;AP3-YC5DCoti zh9!<=p%{{LGb+K1&|>g$Tg2@uV&sxA3|qHojk}ncsJLb`*0Jc_O^*s^`n||~_*fCM z3RSX$)g1MxF|Xn+m(XoAlQW?k!3n4BX4RAjOzFD+Ja)moWLw-H?>sHinV&7HqneU9^A$w6I@r*^UaBiDUbN2cO z?t$O0{7QPCWr1MA|K$FL^ghcKe&lBRpQZQNb@n48Ay|EXx#~_O<>{A!po0x?trhV3 z^ndvLKYaeL`Fy4rd9MYSX#>FC^tbqYA>i|5)W>{YY%IRx#?70TMq?Lw>t>q@GCtCRvSX8dC0p6T^$&zj9~>#LX53`Uu5th-tSZE4(^feV}5QJNhml=SO;LA*eq-1$(GP&IHVk}x`e1mi)tZ`Q>x zyXROO6xNDO=#e-V1cwqk3R}!6GcC?vvK1XQy&rbe%*U=$bWP`NMv^9Ou>9o)PBCBk zC$IAaJT$Uj*YFR%(Evn9oC!nakow z9pB=b_F{L@AF(r_8F@#W)&qw5-^HWPu95#GkFGmDGl>B_S{od0{QfNCtRVV@==nz% z2cP$WPKBspXQl2>yX0SfB=rAKUHp`bTFY0~ zg(cG^;&^QWtOqo_MExHPqN)sCDB*1;`(_&w7LXWz+ zs+uLmMN0N*DMsDA5YN<>3H2??M-Ij;s!Z{Tc;B`qlDGRdO6qm`xoP`QHdSn_5iw0n zttmba3_R2ATGL5*?_Fq_Rho43rSbWPyHD<0Zjuk>EpM>1y{(HQWX9gx;M{x4a_RYY z-E^7?E*dQ+o)66J-&DNKz7*oBO<9dWPRJyV6!x6xd|>jP8g5hO zmg$~qP_t=`-2K5@*2vML`Cv*UJX%&j{TpQ9x| z@;~Ig`$afbj&1LN$sMCTRJC+@I`Woe(}H_gTbMUDY4pXQCxW@+uc;|sJ{TUiNqiL} zJGdRu%K7FSy4>wPY~J{{Y%3Ah(>XuBLnM`u6QNF|(`Yb#()!BqU2Uh)8u^W4vi7ZY zR7WBdHdAhXc+nwd5|eBt%2%wh;dyoOfp8+p#wDx9jSp$HC1_Z!zo(E~a-O{qS{q^- zY&%YZt@7Bpk%u;Kq?G`Jx2dhPu59>4!1ULdOq|qAK8w3be2J7Ze$2DJvKRuLuz{pvBIa(`1@zB_zw#zm473PVw+g!3+&6b)jtP z$&JPt*VS}WFRXn^hZILsq(o!KTlEfo>DttSr0lA4q492Go_ChYhZ1CyI*;qIuFclx z>|mdep216yFRO2c7g4->g5eY0xI{UIC!Sd1HUweJ(qK=V59L>H2*#Xs;n2~ZfY@t zB6JBb_hI-8(F_-nLzF(Ud!x8-yGPVK#VFPXV}x9C1&8P-z8MsJ|-W9@@w zPaAb6uNh|UYY(W+dcNP(cq=XNj^zt|MC7#8MayrkLY0LASaKaiW0Km)Lx@oV-B0Nq zE|pi8tF4nUz|T-Q#AnfY=~xNb%CZ#}^SP;%xUyvorp+z+y5p$oguYj@{`@tPExDOK zuT-es%!In9ANiVdb-D$ujsq(3-lG2SZlhA3#g&AvB-}amX!nl^5}F69+f>e4=k`~2 z!jmK`-0IUj3Y7xqJ0C}wG3FQKk-h18*6rMsYe?X5hfEB&@WrS!VNBcf&HXu%!S|?| z-I4gy1zPh3@5Y{w!%O=;nZI>&Dp)0ne8uLayVZg&>)HAnYt5Uo`H6 z;{3YHRd4ZX82!}@?vxP8<=rc6DEXJCdth6vep6S*R0T7@rO~|<(b+JvlyCcrTdlc$ zLGz@apSw6lJM-W;3*Cs$Ex>1VdO+bZ{8sQ<#PVcD)-~6)&Tc9}ceD9u@>=eS7VM8l zZY|vy1*@a7kA z-KQNGNdA_J@%~nOTpB;YIQZ&9&|M!xgtxA$Hu;w6Esy#ZNChdWg2p(`;n9DH$|!kA z8!*&;5dwZ^swPnho;PVLyWP#Dm}TChsOTC-m z$Taj?u>74@_5Qc$QLbWl&l}O}()jja%#SeG`(EC=k;B}kA^xXI&mLLWk=-9D|31xUla(I7akO;7G#w|&{rFp#6AYI!0S!&Yb5>5@X{Tk7%9BYgh^0H{={iV8r8&N!c$*V7Lf>7t8mGEwy z>*hD9sV`GI&~WEKn8B%H4b-3v+ir zxAA4u8(0*#nQ2)(3X3E&#%RZ^sah>fkjuqUQ#Mjd8_o@U>&`vBzVVImyE1y^US($Cs0;Z^ zspreu7L}^~xwZEfy4CdLhv6rA%@-?g6Mo|zfb+-d(!Xd^hyW852)f+wTHXlfAIWAw z4jWpahXQMYU10j3@0Vi3mOJRJvH0_;m+AER!{-uHxrF;=3Kq=^1|kV@*uErI*_Xt8 zO{G`K4Sps(gD_uIRh&6t{DQR87Rdo=@5W9p$H5&^QO*z#ET30`?NOXt+BL(?F3%#R zU1}1C{q>h?lsB>sOc_wM_$9?9cRHI$U|5|Dz1Wxg9G5#r)N3tXnJ(Pw>v$L(tGBf8 zG3)kr)Yx+-I={r$J~wl*lY-MRcDlmIHqv4=-|GUp4X?Iv$Le<`ZhWlsGCV6qS&rPE zxO({yH$9D;+-4NtYqwFoQm|Xz=(*E{V|M7wIMu=KSu~S|;?%c@LNYY9v@vss9rY8( zD!q5zgOzPle8H;X>XN}hGLdV%{M6)rUUd^+<#UGH*S{Vm<0w@L$`2vj>3_L*z2WXo zcBvkmj?bgH`*n$!B3bsS+QWHxl=7wz5Oy&+)xOYL@BH@(%Cn5>e<>)h9^YNJ2ZGWC z?8^TUl>Z3IzYvs1U%ne$1r{oMU^D$&g7Th9JM?ABTH3ETNvj`NQG&A>65VzE)h&1Y z-#DbA)>pV*Ti>!CLquH6BC-;GK(gvyr{f~|OdVXt=-4sj{Vb=s@b2vwr^8oXFJI2F zTrZ_JH@n&CMYvt?yjEkEWquvW(=F_tkaa87XP0{-_%y9rB&gh<6Z4+E%Lg?i{_n z19*|>Ulm_HOs866=6Qq1N`EPfhBw$P0NS>Vze6GE5MZuVjLC1K;BM7>H_3UBFb%Fh{VX9xd($$XN4!Ucor z1LlMC&tc6!hc$nFSo2(kVrUF7pVwgG{5|HQ+HMQJ*-=Y0FN#}xvD}Ff{k4iezYV*z z^@mOr>nD};!8B-&IWm3|^z`&Lsn^)NS*iN^T{(j1|Y=rY5&9ybY;`G&-6oheu!X6VfO zyOVWFkWcbCxjlNWRrHF0yh6QmXNq>`i?ikBk5>-Hvhu>|GU29ojrw+h(YB>T(6|$c4W)5L+Cd56omZQBG@H!y)LE!G?g0yi# zjd5=;l-n#s_DJma8E(dBAI<5oIx6d#AV(LvMg&&@e6=8A1HFhD|alpUBPWuMIxqJp)ws z6J4K_D=!!M;>{usu!ehu1&hzJP3koykhz9#JDAE;Y*4`OdpD(j513ASF`su;mjUA& z&jt#+D*oJgjwy=$;x3o-QglI(u~RgE>LIUmPTy zy}EVQn*M{pUz}l|ot8f`{zqrn9LMJ)N`Niq{$srOhb{g!TU;YDpmzY1MhyHq?QgNg zWsT9e5%BGRiGl-^#0h+ku~z?{9drX=8B-)Rw%+kz}CFuJx%|Xz#v~$GQ8DeN0;#G>t~6*0(@(#_0EPJdI5r6 z$KN$-^I)#Cth0TlTcmD5=Xu~`}@}IqR(mA>5ptZn)Ku8GGy?;#^+8X(#g+@`<}~kUJJFkwd3K-WTLLr^Fp={ zF3TFxJwET5u_%s|^AJy;V?Le5&p}RTtk_wk?lblEAe)O6bT*+6OT$79tu%};<}_I+ z2;FXwY@;OVCf|ZD)jv{cbu$pY;+Fq;Of#1O|NVrQN!EoVk;n70ZMpEGJw8ji-=8}e z&vW+@p*_7Ctwz`>lIzazjmDX$pR!`d{4r}rcKeYug=t<&+HGVe|E~<+nBau&y;)=@ z8yVqo*A5Me7eZ8>M2JgtC`6#1@Of~JeAYAx4ckQ}WJJzPlhnsRlJi{t9iQpEbEzbE zuZAO!A=Ct@sV7{0+oj{gF&##d(|FCD`}zESKo*v})6m{tsZr-cZi?{hCY&1GdlV!M z?w+N$4cK1%8$MLR@W*~Pp>&AL{NZgFHHDBt-M;h0Gi!6Kn z%P8v<%{15h=lAfCEXyhqHkNO;nFiZp6I7fOY=`llQqEM-u}AlYafd=6R920nk_JILD8DEeprrC zCQ*EMvshB_!o62aCan@$DkW$tJl<%_FYBCpMoVt-6I{T1zAiOYtX$UcWJpdLq0*Di zIG5D^!^@!Q$Qx+tZm*x+q&HGGT+_(on9B5hpno;hr0{ce;8*iCLXxEkEN@2a>lHnm zaao?(oeBj(S2W*|!tVS(?7ekdmCg1(yy-3_1pxtR328(S5T#o|N;;&wySqE3TN-JR z66x;l2I+q9t&ivN;rPVwocHtjo16hR@@06)@veW3S+FJOL$$h5Gq6qBlU4mLsHkxBPBAJ-UQQ%X9uAcM0DS~O(m{yq zxc-s=>uDFr;eBex{xOPSyV6tPR@&S6p~{rKGIqKCY`yehvXwgT#IJ_^_N5BSg%t3K zM*R#Vk_+id_>I3P*Oa;rl(*P^wsgnSLNXth*6f zr>A&+j@bjKb>gUlbJfe9Js*NyAcCG_ljI7YXdH+tjM<~ICg&*y|QpX4!`xug*B&wnl zzG;NS+U{h)>+J+3Uo?(cOs5|_T{Ez^-D%c@r}wTl_bsN#JFPG2z9G8<)vs%!CtoIY zo*Arl1TdE<=EBM~SlHRrdS!~!mY$FgWLRE6TpNY8D#ho#GFygr9b<;&a3bc(V$F+X zuSzD#Lt(uSsU-ZQll8?hT<1bh9!(EB4h=9u0Ku_=r^R?i^x-4%rxJ+ZT z^6H!BL)J&&!7kbGG$`>EWfzktw?!o7Ob&l>&C4RmjTYi6h(_d&R#t9isQJpmD#*?< zXL!3@!D{p15CNyW+OZR^IaAqUn4@OZQW^9({B{az>4>KO;O2DgyrCQ`v-sHLXZ@|6 zgAK8_TX#TB-Pi_YL5CpK8pWal%X~NxeqCF|%;+q{hor5KcA%m!l*`r* zxB#c7h~>kZsfcitfIRYiGFAD=E{ePk?GPHe%gpL^#PP}g``ZvOkF|U;spj(49nkFL zwBUMsN|d&Gwh`w+M>MDTq7{{UQp%zB-V4;p##(JuUtXUp9K5SvW^S}yM{|b?_7ALI zo?i(V!D|r>@YsML6q_iy zM}N!rBjv3DiJ}g;`T0kqzs_hyz>zIx+YH6lBY9hI^0oW-Qw8qePe8F1>_P-A^uxFL z8Vr=5ADk0C(nEsEnSVQfUu<*1+-XI-CQBzzxHyBkBL9#~IB3B~=)rpGU~u^A)%LtD z0prGdy7#%|2~YL)(n{Ax=(EI(UYNFz9F}VIzBTNo==)ejzH^u_|H*G+p*r-A`Tn@< zgbbz^Z zdH6(Nq(iakvs*-&f11jpKEG256vo;El?Tq7%zF+jY(obTD?YG_VkBgSViL6TRZ1M> zA5%ty{93y3OVsxkP}Wp*K+yPUpR8|loy{bcnXeRD2$yIsyg*-)w|nJJ(P4qY(|$Rd z^j^j4^CunY8CrSxI?@*fN-+m}tBVD}dT`agPopv~(|4K}ahub^PhLot(2Bs}YGTHv z^9U~(tOi=dsgh+InN^;>#L-7wgK`SxC~Qf67Rjm5Xs$KYS%CE=;5bA0N%MQVr}$K| z5Ism7*MaxWt#37w8E~I8BL%^mWGIww38$ZDJcEv3fkz;eOl3bz8+?AubfB@+9l6Lb zXt$ENl8Y(PIV6gmp&>WPsTYHGuNo&HeaaSJC1bGV0b9bZ6^qPE8`KUTzLh#Oc&+8j z1JdcYSlii0Mp!szQZ&p^BJDyJBGD^oFCYbY_$E>^Yd)Sz@}^VlQSzIfELtjmDIxYU zO$aDuWqR9wMCIe1vR~kq{l4k|MPVKC5lWr3I8oBO;zYZ!#D|ShxS56ppXXC${WLF) zsCy#SL<%JxX~x|`x}|Fv!XPH(Aj2v#j&dgZ+^baEB@A-Aqm6_!@CD(TkTKzp3E#J$ zxV&!`G;*(PXn8EuyL>($gz6&_@L*fjKmMT9gYpQ8-gX&j7BuuEdm)HO~iev3^+->VT={Z58vKF_2W};PSV_ted`+jV} z%5@)}A7%TCg}%$`OAiZ+Ym`i|Q=tDXiFLEb_%9_^$5)9(3`i`uU(-0hB-YZ zfOUF&eWJGhIc4cVnH3J6UX0|&06E6$5=y{Vx|fOG-8*L$#r0l-@0=6`(HewmhFdh5 z&y)sJkazYksZOm*r63z#73cLei#Mlv2_AY2S21K)$qXj6J$cA+ZYfl3xj4*yIZq3B z>ZQ`=AYR`hETWx~>00i02d_^JNuPvj$z^t!kkK=4+1r|?@_=Kf{*_&(X8+p0ZJ+aM z)0e!r40e>$>BeyPDUb1A_hd=lLasf9TycsquwxWg*RtgxtXgMvi`WYJ!jgp9Ib{`R{wa zoR1agkj(wGP5ktL7KeYrg)HkSMgROknC{{&t_z2>gbQ9FCYe5d&x6r>7i^#MAT(#h z403o?waU{;$ak7@$yi#RT=Af1z^GRD!CCB(C5*E|p}vE+#$*z%MwZdW*P&fU^vrmi zTg>a|=0NV~bx|#SWpS8S7NnCsGnyDY)ZXIEDW_r>lsh0~W@WsBz0=KYC%U4^JzdrS zb= z%r8?-1RvwX;V|~XsQeA}QE6VqgWIjHl&v`sS9Sw}k5k%vQiNKVKa6fxMs%w1m1i<0 zz%+a15R%WULPW}msNCjNjIsF$U$Q9HwMp|J8B+t7>}*DuZppm_&-&7xL@W@Vn+g6B ziAb9W#-owgA-bM0M7K&Y$BHrSK0L40KIme6fu5`{YzsMOQqwY_fc2QD`DOfTqNm5S?^Zps88Wik>cwe;NO)t$(ou^e(Y=d#d+@l*A2JnvC|$f`TD;lTyDPH|D|xT`bzTwcY;@BZ7X|p91;Ouvw}TgLzyUoG8VG9F83^nIjw?k8ltF+HcE7p91b+OGAVG*< z+ks>8q@=q+9AATNcDV68o+d~zTZURijOZ(weYAGkvbdjR9&g`aQhHSN;Dj1F zKKPkRz9!ODuY69Q9R3L3hYZD_{BV_Z5X9~BHx|r<&iLiMy4;Ov2iitDbPDrMOy>DH zh)&J1Uf43zLfPpWOfY;$q%o?M=Ky^}!HU7Y0Xmod&AK@9a z&XY_GdC@m=w41c^3NKKr#&9ll;OGQ#80VrepSKnb$(V*1>yJ2L&Mgv{$&45%nrsfy zaWm2{n&HQM2!5_ZF}8-Q4C&bRS|UUjGODCaaKqt!CS=pBMvVf-i7sB`D0CcZliYW^*d*Oy2* znsoA*cQ(1Xiz%cm1Px&$c_EIhzWmDo`bRf=9()KAL%`edr(qWU=xm!A?m_W{G<1QR zq|9R*0_!XxhDZ2TBg}^*k8FAu79Q8PRVqo;%aOJW!mu(VW2N{Qn%rfBomSa-hn*t* z%xV;l9Ew7?NF_1(L9yA~CzinTNlI6f+}qUHO#gf9zX^DG zvyb|{;Xm=Uee>S^-dYk6;Nbr5htQ?pefxY5cqt8Zi{SJv;3YUG2}cZZGvFn-9Pzg} z3r64{pdTYkWq&0sK-_P0V&EU(j*nPZOHEVDjMxH*5l#mVgNX^W!28+`{P8UZ`8CS- zx0qAl5-|Azr%W_3!`KBd2+<3<3xy5?rXaNdm-!xl0qO?|@)D@`or0)t{uKow2Woz! zAb%h48yqCxe@Q!-X27y zEQld<-I?a$E354(-*RBevE>n=3q=A&2jjoys%OV^aBMR>K#E?L2u@)XuJ)vZZHL`GryMmR-oP{Glg+#>oBCf_-kJ4+Ys-Og^ zHkL9J7sKzF{g5AH`D#(gel|zu;rM-+K5n@s#bT7gdttWX4D+_QM|6&hE9f#ke4ajA zC_;UhN{5%dO(dDR8_EZv+R-9`jdm}E;~$aF1bC17tLMtUDe^#XG|CLj*uwm1DsL_+ zpv@CD)@Vn5639|$zcQomDdxX7B7K)?ZpI^MEplcq|N|Do0UY?R}abpBq^NkM} z($DFhdhM@9JrH*zjt-$kPg}SZOeJH_6yqB@K59i#g{i$N?OqW2i5sB@upEZpP*$-Ymog|qTb1+DP%CrSC}E+BJSmo4HJ&=TR>y#P5>v4tKs zXld4CCn>W)Nzrs6>B&6$S01}t&jTN#6Uzv^XPeHv0G2bY_+iVYY~d!(<+<}^9bLl z;Vn_-GUF|lu5p&o1)Ap>!E4dd7Vq^2QB?472fbt)9;R38Q7n$mpW{Mwe5?s_!9%C{ z!ft!gY+D2o^JzKd1LEX!4N{`(;$jKeksXEOFvHh{Cp-Ld?!WuZ{gKPyz$+{NC6{Rg zubq3kBY5Es4jC2x(v5nsfSL?GAr}*s0LLExdGp36tAU7jZR-$0X?S&gK+LES|4<-! zHw~o@47~c5SX+Tx@bOB`eV>>Rl%ph}##j=A;So8S`~V(P{sr!G5`OlV*&i~QWu>1P zwyPzay0Tlt5fNk(@Y9HWknyTf>Gq#j;oM~BktbMf54)NsN_0E*U+HziHdWl2{;aLE zp6o3|V`=)wJmSm9T+5Z@3RXm&H+_XOiay6gSZF0g@0zb^MA)> zZgzNpB>gS@6E1V}-hFQi0ww-EmjO?Gfzx*`6O1r}{_k7{%roEs>5c|M+1379|3@zK zW6UUQU@I_k1Sf!fe2*FRjko+&=A9>AJ-0678=a%lpG=l=ItnnHkjNDZU|C|3Slt-=Jf1oIm2ue`?f7q9un zYyLxCgOwP2N(MZSfxt8LGhUPa;)h6@`k~|EldM>&m_jZVP`I8LSwsw}l!`1Yn1bWw z`-k>|!g}M7{sNcj7&>0&%blKb$*74K_FE46RS~6?GE~eoij}o=OCwq5r6N=g#D(qh z!x>y}=y{Z{FGpUyK2iA)>)01Do_Ji41YmYf$zSW=>S{EId}jp>3MGT?jD_ z@iBrPhuF*TSm^s&pk{Cdo6Ln2gfVcQMQ~wxw_0YoHPokmWDR`MJZ9j;n3$T{CSBj) zV2yx*p55zBjCrtcO(Y4siiJ$Kyh5}f0UaCqbYS{)_tiwWl&_5x)!l6V#+;PPkKP1t zkTHVCrYl<$gn5PgcXLL3c2agPpxAwTYX$J1S&%-kZuNZ}EB$h{ee1&n4NZ|Q`wOn< zO8Sp;!^3T>ujl+(dlvRr9Q~?ba_~2jG8oV{MNUmwlhab}U8bK2&#CC=CNnoqzS0Dt zse9TB=eTuHZ*m( z>-LMh6<3_}HNJ}BvRTDSvG6H*RJL|ct424WO-jN+6JMENl;TNK!gKhR-CQ}T-8zNb zkaJb9JzKn#5$wupJVW=MiiL#6Ywx2>gljc6-MqVZcGg043#{^1aP|oGYA+lbOT@`t zbGh_nT>{%Ony5I<<=Nvl?sI%B9^J04D4iJ>r8c9>ST21>lkdFC)|48OD*V}?gyue} zpu=6InwAoc8sRWZ#jSb8FeOa2P*krv;VXjIFaz8h&i%#$D)FrK4}3mYt>qhrZPu9R zFwf<3xthkcVuhA|qIjo`3gb&3;uPCu5_8ENJL)3lF2;PTRB`voBpvDraTcqfe{EyD z%xAW{$+02c`~&oeVz1s_&?7E+pYUXN!wAN!O~TutI5_{{De6bs5_y@{-Ual}n*e?i z@9b&t*Cqd8Z|UwVf}7b*{8)&IFX3}SV35YYKn&Qt@orsEjb}ZzuYIC!Yx$bfe`wX- zaz3V-7{0O;5mN+TlmdIsyt3XFI`X0KTW)w#(FE1<+FE%@!;i069mciuKKrD{p(}Os z7g)ZzYcE{NP=vXIo|>hkSvLNZGum`!qb4^_Um`IA?m1KgyHW52QLsq@_4(cqt^(S- z?aS%yl^HSO=8e3)Qv}&%#%u5E@jzZ1kflxhQ85#~a4acbBb5mZ^1+usna}yZqboQ2 zG2r6=mi`G{xp~jNw*`S5{+_N_{ei9&0m=?^RAR)NsuwT;f%!^Tpm)iB2*wTbgAzm@ zsPkP3@?`t3lpx@lpKsJb9Apf#0LX&*U(f|G(E!th>92HQ^cP+DMHl`xA-S(w-@iw6x($y*V9dU2VI zZ|{{y_5%|!HRld99Bb`sxlsmG0R?5BZNAj^#7b7tQClYO^;AaN^LOWyz6co<)^vD4 z%X7ER(Y!ysLMyd}M7Cqvslj_?_*X@FG8lZYPc%$;=A)Ir@3{q; z%7c*2aDej4sO4is{X;>DJyg4Uabl{7rnXsK300QvMcUQ!Q1RXY&W7#mYV03Xv@TNO z@hMb8iQ30P$TIT!2YXN~E%g0>v}=hz%2Cdi4zHE0hxtgI z*<@JCR7TBLVR$EwGpxPaN41APdGk<%s20ZvP=VaTf0+Y$fK(|s?YPx-0}D#~tTen%dCGJ$&I6UR63cP=Ckr9-ORdg!DOE@U+ZtKoUNM+5 z`(p`H^}dgdDU^RaMiq!NTdfNO!hW3#PxY|-y`+6|Rxt$Gs&+z`+rzjrJUNp&`>Y)7 zRcv7i_vSG^+!?*5h=g7P&L(oSeaH6De8T27hJ;Wc5OyLCVMG#cK?m`~xUwLxJatsA zEVpU~Jq?ux>R!-L3H3!6C~>C;5+o2YMa{~&IUqwL6*ap98)cQxIjYbgO5sfcpIG$t zb@xe)!j{m;;oUQ#dz92>=iwxk${aC1mH5Sv987#0=EmDqOs!Yqv2#Ikn<#~W5*tJV z6wC-8>bmYRsI|Sl>)pQ_uzzh%S+Xo}Qo_Miaaz)_AX4+l+@!2Fj%md+D0*?0o#P7{ z@`MA&=$=*^&h@7Y&XX^0D4Vd5mE!AEJ}jcU;V0!Z>3XUctKR#`jqR zUHZ4lj~}^5b-ZuX2(aZ(r+*s7AsI0Rj>#|h_XQ_~;BG}CWAYLE6?UdTNB0~tGO3+$ z>TNKb9J10{*qR%6cg51%Pxu5mWZ$L14StB;JRTb-0zez;N8SZT$zImX2Ko>BYckb5;(o$UEKKLU$8 z0rd!&_CSKbd8n^>uyy9QNhF!C+kua-HY-s-C|h8EP`0Q7+rM)q)er=D@2@JJA5|{b z5EwrwTYk4(c?K}1Z=C6`ELT{7{ewA^%~#H3_KP$9;!OV`XG$BrP(=b>yi?!}{TXM< zm;K#rv=-a9fLY?6SqQYC!Uy;Ob`CfiKff0;>B7>5iSKqoBKA2k1!SP2qQ>uATUl{r z>1@0#jw~x=tezd)j$z92%(-(Zzk6Pjm#ER5K%pG^PyyS4^e#`l!$=H+C#+QAn90L&KU&-VydB7uJQ21U5W}>N})*9s{vG_Js^~9RG9&<%y#-L5i%1 z_-gG>K1qg%P7XE>MbVBZV^q=lJ zJ^K05~r;TLm5(`V*QXSn6;vs)GI4aX^YSAd8rQxYsEl5TgNyS6Ue20H8U@R zo3WX}%de?v;!n)F>HGm6U7=(^o-SZQmsrbzt@`5&F!YE1VtMt!s2Pn~p2#u^P)-MlBv^JJxrSt^4z^A947(BK0zAP~WiBuR$vYKxgg^q2w+&6DPWhlAC zfGLV@p$EVDbTY@iK6hEj$-g5kK~_vs&qJT7v~{r0s-QtO%w*6eg$p#ioGV2=lwzUT zk)4T1v_PU;1B!6A*q7;r*?xa8l#^Zv%saleD;@Y zMoJ%u4e2=pcjay3S=Tj)pe393Nj2WxbwFruZ+wzfO-wpWRMGkJBjM!296^0krTk&0 zmd+R6M>MLO?Jl7O34;|FCMR~bHjvmP3cdxUWU48kyroqmS5FI^nzVgxR$5*IN=X;L zVj6a|ND_CodCroYyhJvD&Y(mS(jyotT*YW{uw@(CNsjiFrmovxYd{+Wzt}%zlsRP? zaKP2x(W;xSMKtBv9x+!9Jv-qUUb3)%V7x^1h7@M;!+Pc1V)>++&ms(2rDhzhEB*i8 z6wu9%3y_GvrGLU^Zr;7`Z9yQ7zh^T#e_%7OvB2gm&e%8E445+@{6CnpXd4@U=Q964 znX?@Jz)rx{xnPT0@Klfikp0e1Ks(KFf5%Q(f%^YG6(kO_1SW&j0qxFTF%@uIFjE=$ z%2c|3F_m9ThR`J`A&r}E3 z;3P4z@-hV<2-$AY9?cwwMr@RMx2O-)vB25uc83^GcXBNsO%+Ej5(gNTQR|+OzkNzs znD^{4_Q8JB6lGd7XC^gy`7M~o2qH6GuP@CemMTc$U$X8XR+|@gv}tr$r)nJqZhL%G zDG=)*h*~zC#8dM$>zqq{8AKuU5Wg~C$|5LjP;J66pz`EWO+!!y>%i+)i-J;Bq}csW zjx$joLjpCGHp}u=dsc1Wefy9@hU0d&T>LNbl_J%giN}nIF*11WXhxnPwL%QyB|~yK z>s69&4?-kHVx_pIXSrQXGc9<${oqWydWe3P-=y=t2!4syX))`TNMpD0tHO=A+15PT z`0johVR4u)?Xg2>)`zN9?3^Fx7;NpswGrNQ&N0^2^I}1M@@clWYON0DYrw_YgToqx z-{(v&dt#`W+A(FClNWwWyBCPR`N%MdGPHq8vA1%~!f#Mn z-eZx9vB&zeu~mYS-Ydzc3@xVOyd8p>Wkws~(zTeW5-$lg{ZxoWxz#U3Q*v(|Qd}(J zQ`ceY?jj>6VR*P*!s8P1b#!v3BQ3}8wjE6pB69;|^C(N4kLlq0$j1zk(){7rZ4B}22-o8k- zrxkEKvSd9=nnHITQy$zVy6~DRc`dYWf_z)%Xd>&~^4i6ELusCg>M+azw@8ge8 zAVGKNzkpa(kX(Fc4x1NxSEGlkp^+rR2)x682*7$BM zC`JtvvsSv~CYyVg1fZUeXm>$u6e?Sc)+f!(CrBQ-tep(>#yt4=4i~|Jj&C`+tLMre zHSRl!>gt>@s}j1Q5XuW(t)|Y$JH1(8i1{Ucdn3|@)F%3&!=N$v5Smd z$cwCk$*Hcrje!69yDcvS znEU|H0LTv+kaYz&g?9&s9EERyKnDiC;04C;AZRE+wg46|5Wv2{q=g!&kkdp4eE!it z+|3*Q6SM7zP%(*SVsx(^@+-K9as$$jaW#aGRS&bDjSw-o>WNgqd{mvfJnp}Ik2}&# z+F9Fk4!u5Pb*Du%w7?p%umcgAV=K_I&Mo**#uU!&Hm7@)P z##FA0$cKvy5h41tB($>NK}DoN54DbRL}r}3{_@Svtt#1cPYAIoY42o>Tfq|!jtsuc z4}H9w=KC{1{I~_J5n@Z5oxxG_qms*uPTp|%C)D~b7v*ZP3H0pCDqjYk>I>GYi}tW< zq3Dr_Ec_xZ{~>A77b!~v$K1vMp0A&&R|0~?j3612pB)R<@h>_GWg>Yz3s-Y(hR8hv zCHW$d-l7nOOq@>IB_*@kG;r>>zpC!qOk|h&ETf9jCGAQ~=pH-z=HznBv5C z9OXJfZNQnfdK8tN&=8S=HINOYqU(+=R}NebN!$M9_45x(1g(&AxIj3f%{MES9Vi-(4xQmx0B zxiDPj)H53|j^qE7DvZ+*wL32LztCENVcneGPsB zK%ni-IS`SMBS^xv4)j41SWOB%87{yd8p!9i9`FkdyMqdn_uniGoTg$AWMJ=0U8vcO=DY|-hto@6^;Dr=a-0*$bwHGS7o|=ve*?9-&+f+3p)Qpl) z?iBe42;nLtBXUdIm<&|0nX+q199+2J2EnB1lW=6VdX5CHOLrdK{^0b)d%1#gJK(Tc zFk!E`*YcDa>FG#{o%<(x2EVqVl-=`FWHOf+B>B8HCN+iBe)NyCv_(%D;NCc`GR;(U zt1oOhVd`x;04@oqM07RDToheJlEm5PteJ zN6;Llvn~n#1pdZ^L>nw(4ZKJC&T-D;t3v+2v#se-a_*klc<$0Wv%p-<*gj-*2&o(1Jo#KYw@L-%%Qr&OxQ-aPhyF1I>xOf@gC zK|_uR$^P?$KkY%_a0ghK60rCV`)3Ew2tMr1R}#h(te6dGqt?JQng)f{9XLzB(c<_5 zG5lsDZ$draD0!LGC;dW>_~Q#M5KYx;(GWFT3(4xIqn(H%GwaRgGmNTAkHB+;E0g!$(>6pq zfxgfcB&0%Pq{R{~tBX6erBNJ_+tse{$+@pDvT?Q?UC@CI9sZ(8#v{#5uWm+Hd4g&c zA0*@dUgh1$))0~%tcxQNb9qUG9_tzYp}2S{bV;y(Sxvci!Q8ya!3egdktJuMHcfcT zEf(VQQTY$XgTwd}d8pGawDnak#ugPdQ{Sba*)8+)Qm+=uGoJ6#<+Gqnyu)3pv(7bh)y3&}yrTpUgpX&Fnj zlPyB>Dkou$7UJM&_-<;1%+qzaXes^aHSMr+#GO&#FRPHmOWeLf5DT#KxhYCQ; zv#~avKdEFBdu!289X92|uD;EbKuHkhL7BdC;t_&q|2So^c!Tf|CfXst7s`(_=4G6w z;B#^2E}X4?+>3{Kx6m)PMzwdi*G*n#9HMeqgoa?nynyE+>ttLI<9fkfNW#&~NP~aD z`cf(~#65Pm<9(9s>{feL`4W6EXV_&AcV5 zF@grDH(ki3@XlQmp7^Mj^}N7x2(H0PooPg4f3?1S9>KoNU3X_P?Ja3P*OP-zS>4xs z zQneHop^!V3`p#2!Pwkp*J(9Pbg5I|>ls?|;L6++uw$UApd`!Z6IM+A~yR=2yS)N4&b>B2Bj1(!(+j#GozTZL@%V-z8L5)p~kpk}CE z(wOS&mF#@jq7|nu)u&a9w%<{%jd;F0DZ*(ikxnY|xxM|C#o&=b*HEUnLqbIzV}kJ< zqj=+nG@Embr-{%ajat*ypeE}p?qh8lxlS=RPNi4!HT0C5gRZiI9rSD!*Ey)Np6C(!o`iQka zz#eZ%!Z<=w&&7R|g}#KeeltF<{rN%ntNuvi66Ee6ji_hrG?knJ`=S*l;ccGC%LQbF zt1qc)q_!_|Vn>h`lB2=0Wbk9&C7P)_nCgcHJ0)bVo_%^M zE2I>wQ2cx`r08v->xP~2u_W|cqgzmK+Y&#e{ zbeA@|%Zegqbj{c7r#w{1*0En#neUSnQ~4^8i^k=Z9*wS+-gOj7f_Yv!mOk&okTxBG zjj5CdF72IEOgg8KE2wiKEbh|RR0m$1i*S;P7!DS_&BO-8x$j0>q1GVv;GXxgkMZJ* z#zn9g4s+_|EyW+gSXaZ$+-@BXOKh20VYcc^sM<_Wsuk4G!L~9mY>SG!!)5sz=AJ+AFgf=(TMv{PsQ;>CChY%sqm|5P=Cck7GA$cw^&%YV=U@DWBegK`4(!78F(< zGO5iiltNl~I&Zc5SiPF)#P-`752qP*bML@&u0@*0H5$I(40_~jx=S|;zOnqI+UOUIPGcnF6UvO0Bi z`ymislw{m>=_UIeUmbcPY~?E{Uh^?e7|PVv&RNpuh0t!aSk`2$hA+3+US`Mn_~K!|^!Jm2Wg510Suo^JAv-}lt_vps!f zAvgDN^C9?sA3nek;K!r+i$~)7vU3=~mRT7%dv{>I3!KoPIe}kXJOKM4w$d^)*E2SH zOu|UVKtil#q+zV7XQcC(CH6&eyCkbHZxSS0DhY3&}*8hSpz#ZG@yUT zz`#mxsHSH`LaeUC_?X1Nf{}#S!Uo7`>H|4-ormDEhu|_Gr>PI*)HF1-j4Xf}zh%Cc z0NcNn{HqSr_Y$V>b(p`GFn=#$`Ch{Ey@d692`dS)u7RGBK5$0hFA`#HV*|l&1>bWd z^xVLE2@UmcN-)Y1^-o|^z8c`#{>k_$BBaI$7#s*3Lteu1>SMI8Nn(0#Av+p4BCo-r z-!eoRLQ1_w=KOlR9o+)HS15&S+WeE1IlRhdafnE9xc-o!YaiLh{Al~m*2KCLVlMiF z6U^f9=VVoD#>V4YjHC$LL4)l%!x!l+u?SEh!g!<&$T`s2N>Nfe3_J{}^~P~3wlHMU@@fDUqiUkDfopWQw=AWp>p4WQpT|dz=R~ zj~Tf;MNAtfCf|&A-u#rXwi-0m)mLr&+QcRC-TdK)+^j9kg9Dy*+czzVNPgteYlw4a z@;Xd)`SsVpRr>egQ_wo~8CVI#4qT~!8$LAxy%?}g)a@Gm$Y{V0Jv6hI^%-S#=+baOB3 z#tX-{2r{U9x={Y2runL(R0QdokNX3ZPM**`stabyq+K9}T7QH}R;g7L4)kJOZd9CU z#8#P*rjFi|7s*@a({DKKE@Rp=h_J#a^j8BCD79%@UBe)6pH9g8FOOveypO5vp4u|Hi1P z`iPH=D!$$NTlon%NkdLnY7w_|?e-+6^wbY};*Ox1ry*xTjwF_=7c;;M)rMJmCF?>y!!x@v;#aHBwr$W$clB z^*x2~89k0NH1~)`;`v<>g$kUQE+Xk50SY;(do=scp%Qg>>6`=BpV|3gV)@X(`GlvL z$_2FwY{_`u6I@Xir=Wii4#REk`jnTUf zDH$8NgzVFbD&^}kd^?4frr32EmG8;Z`;gRhAgoifq`tX1prRxP1PSNd{bU_1M33A@r*7`_?&R_l*YGvzsL5R==&q_M*(-UmjIrNAJRV%a5rC`-&<1y z&)Q!IxSNmT?+q~kvGlhO=XYTO1(<-g18vE^>bTHA?PwvaVBHlm^gCdk3)m4_7VzEs zn9BP9;b=FUs>>jE^Cyv%v~j0$zF&6rrJt4|9E2RG z`ORqbpL;?p|H?n!{o)_L_{V?9KR&>rdWr%Q*7JbY2s{;hlYeZ!6f?fVh+ZeR?p)+h zKxuFS|4Pdj(xdE_pCmDl>aLxd|H$;X3Zm@g&fN(SL9^^!@;c8Zbta9Tcs3^Es@icZ zH-jqHsxX8=?n=tJupH*-JP&SK@<8tUt!|>clK&%fhClP=uR}-c|ZCZ-kaO+QLPyeXBM>uTtH5B9`1X zu9x-xEktAv*p7Bl(1QMb>B4qA!{Nd?OyYGtrE2=A=TAbH;n*c%ZhqQrQeA+#zmxTd;3Z0ztQ zVl^vjiOC;}^gbaTW^4DB%I?U#8J=H(PtgH%Ga`kky_0m()Ph8zbE?dXHxd#P7_RU7 z6+f7uBlgNfrzuXJ!+0IcREe6hUw-ikC;kLOyW-Pl?%P&|GEI9mOIH9Js zn^+9QGIMSba#ypZwYHdW8BHFvUS4H8a&M#7G!)^~ndCE3==#^rCLsq}j>!tBWEq`RUR2}i+s?i9tQ?Ap;_C{8%)=GLT4rK z*C28FSvsw^lp<@*3+l5(vTQ}xWU8E#e{s;FJV&^c{p3|tUfqZU<*ll^?CBa?ofx9w zz2gW?fMm=GY4s%G)57B1?UhED*@ua)uoNacfq@PeZLQo=cc;s*?%O+5%{QRQpgIBX z*S{;dNJ5l~27ov2PdSa0NE3KH49qqEdpBdc0w_GoFR2p|YRFn|yMp#Z`F zgae2G5D6d(Ks10D0I>k#0K@}G0FVeE2|zM{6aa4lqyk6-kPaXNKqi1J0NDU?0OSJ5 z1CS4(06-ytcL0h26ay#$Pzs<7KskU40F?mV1E>N}4WI@vjFA*%mY{eun1rYz%qap0ILAj0IUPp0I&&Q3&1vj9RRxk z_5geVun*t>z#)Jm0LK7M0DxbdGXUoRE&yBtxB_qu0Bq~}7pktCQV-nXTly!aUpL>} z?`?tG4D+|9U+J*ls{aFbc>lE-@PFbCFAn~9Ghko@_OCTiszCNTkxhyD`3miytJMD4 z3T-A}|6mhh2RN_^u@&(7N9(hjqo1FQcYc`=|2|^AIoA8(Wk?Tc1Y4ka?YILLXmzIJ z{CH{N#-1u7a5adELZTWS^5ljy^_dLv&x{|foNu_FT8dFHH5!AYa#L$KJ-P$hN$>>(I-pS+X`qVdVBQ~j3(hUD0 zrPVA9DnbK9?w_sDPM4h!UtmCYe60tLEr2*)q(mwWEm%YYjx7*C9>ia2xtYIZp;+$} zycIFg^ads}ZZvkUGnQ$=+^)h1OHpX!4iH*k>$lJX!7vRCHRBx1owog0RB2j`Z2fzs z?Lb6O{Ket7DH%cYZl_gPBA`xk^742mSNsH1(3z4<5 zizq^NvS*J_kyNr|$r90O%T_|+N&nB8+vS$>y!Y_>z5h&o?wpx(<}9Cc=6k+p9sBZG z*}4fsrn@bRS#w2B?4;J-b07PvNM)BQvo7CU6xUkjdJSiqN&}EI(4Btt@0v$zDDnBs@kdTybO2=h5peOPt$TrN+y( z)r{NVVEaUHad&mq?Tf>@RXvxm^$DH0SZ~HA_tKKD>$5Gttk2e?*Jqo5$SAH|pFKr< zT(bk6M`T~E9+Y*}c-dixE@^UFHY*bSy4cufeU#sPuCrTrDVwYwI*9{iRK5KkEL^w5 zDDq&(z7dZmOlhq!q*X!0fywr=Ed?!_MNYHJo~M07d|bR`{;HiH&vi?Sb56Ma*`V## z{MNbxRb$!i%A+D@pI;sOZZv16{Js%0@)ys|3YyT#z_Ix`_mc9~gWe@%&fk9`CBI`q zeCzBpBd>bZu+OB9u8g^uAMX-%Y4+#0NlC`7uEw?&bqaiKw{zW2g9LBIvMmzPyF`M= zR{G?ucN)G;uBF15XB+>R-ePI4vqPs%B}cm_4qCG1=Z(t zn1{Kj6%TFm*1o4@#o+sE9`2YFYL+%Oc3sb%%f0S=H18<3HACrk%|^alspI}Yk; z|6Wng=u$^fGmSrjZkWC;O^Dg=b^pDlX0;vr*t>R@4=p#58F+L-xWXsLebb^Xrrq<_t%Hw;3J#q5;9++N+;6qg1uLe;kqVwvu55s` z2l+OtH(fE@vUZzZ5wDer*1KBH?{zcY>O+#f=e|ub!BHd6KgV`=r3W*Msm$xk@}Fut zB>SYVoT;_E<+8!8M2l~DrX_?{zk1}}Ox4siBHqPD%WCeeii!#IPEJ+p_B1gfVTg>h zeix4kFZ|U_?nb6-`L~(!tjiP447pjF37NK`c@7pwURaFQ-OyI)g~tP{4=1aSi)W|5 zIaYS>PN~+1inQ5P$3_)g?HYIB;Kxi68L~*b*TUD2XTIkurkabM`MfV5q@bueT{uf~ zT_^L%ODbPDXewQ8^@SHASbO}4W>P=*c+XPQ#6Q45v2+RFNodqrKLt$c5lc1bUpWQC`O zGD-5|%n!fFHGVk14JQw)wD%uX8DMRxD*u-nAIoNScoa01J@C4>Uu;-E-H8|U z{M_3HRDHf_<}a7gYjjK9Sus|sa(C>vZ<6>}TI#I3>&q%_3G;&Pb3T_Ixa2cfX1{lS z%fPZRF)K3HlnuP}qVEH-`N^YW*XV0pd6U@VocN=)X#?Yx3p!+Z-#;?wddKB9nPX4% z%r9P%8Q*TW{WbNDOA-?O)nd&eGgCC@w{y%&%=H?Tnq9j}yVrV^7ra&4s&@~PRoaTR ztF(o=tF+Z#-LGAx9TWQbI(L6Eyr&#GC28 z%+;bdNc7o!WtZNem8?tYSy*NmIu0`I#7*ei46E-Al4konw1^%TnD- zk*6esZwqvhqA51X>V_L|Nr_4vG#N9FwHKaE;Fc-=My(WtA6hO)c& znpQVA{IJ06psl?*6{^`3l?6}IVx6q?!BVqm{uvR?VbJGLw^y7wA_E6<+7!v zMk|G6;k?}m`tM8IB{=xDG#*)|D?ab_h_qhDmBaeER31+*?%iMNu6e@qM9a%lXI^Tf zucI8>>9e25tvhYKtes|Vj7U+?JMT1YM8Y2;XPi&GD2tokXZ~XKKOS6Cc*F9XIm*CT zBy6dR+Y8S%nX9B;z7!69l(0W^*{p2&eV)D6xo%H)J*?z*;<@XwO%^%VHHJFq?Ympu zVO0EQV@K&p0`a!jWaoq%*-K+jnvYeyHQKwzezQiKg*DpUMn)@#f3rqAX%YV#Z6$h* z_5<_O$#yz&mtE{XZe6LP!?7#WgGu8~3i2LehUH5*vXMVHi+hXHMXRHNF zlMUSunMep9pOhpdTpefMzO>momjZ85dA-DH_prRJ>&_m|ta70YBnpD4`>TQzGVd43 z<`j09*?UhXEh(wm!@K;oVXw;7&)z>>aHq23V)5qYd73q-@xE1o&YBJEP5L<+Re|`o z{PmUz+AY_1YmWHEfQ2_I1s4QY$h_{IyL0&a85;73l1`0y@=U{F`^#IKW%1xm#wa4|2#+^p>pyG%m!#$7%7!p8AghknO&%TGOe=JLj^dCH~CnaUrZ%v?X`qvwfP z!H!N3ZZ6uTR`Nh|_XU?vDvsFtuh{9s(iuHP^>=@Cn^Cn z0Iv+uXGy+MN>a%k^0cEo@u4wqY(~w&-eKh6PDtx7j8C4)-4`XBb1T#L(K=rkd@YZ{-`I)MfPI2dk~-D}7|iX0p03Z*x~#VrKR7m)CCej~_j5ZE!}9 ziP_3mA5?E$<9aos?#bxgQOV2Aj6_%N3AYSx-+m$tmE@2+tj{Ey5E#a_RwUPE+c%me$aO%wm5eiyaj+VXN zHY?q#^KQh}>{pyy_5%+E`u7-`F-tkOdDeQ1%prR+DRsl%=d{Kfyr|mMf3CBsPAjRI z$I7&f#o`Bf4}T`jK4tJRPxy0*Xt9T(rTV0cMFz+8tySlAjlG`d+S^pC+f2#9QLNc( z%`IPNmpt6Qrq#>s1331(cT9gC*l~ z7wl&sV)DvF&vbW-0;ApnCsPM!ZrHlx;Vq>lpKe}%xMzy?rZk&=Yd4Ni+jGqS&1A7i z$qw?Hqn_Olc9WPCc5ht566bjxwAAeLwN_m0<(;m4S8(U>kDu>&7U>kv8=>N2?Qn4G z+7itP<4y+bKdN1(-!UU7f7`5R_gSJ3<$Y$)=&6uv7Hrc}Yo|?4JL2 zTib1cC&p{A1B3K8O=b({7#%(~PC8OG#$_0H{Wx96BD3;IV} zA2=}1|GxKys)#vyJ4CY;Z@AkzH$U@!S#G=Dac{<3A1TQ=m^FDm>%g*Zhn$O2V$_FB zDL!*6}k*A*`$ZuHzHyp03)4JG;Z~w)SkYl| zH4UOcVSxpk)bd;{gIse>Le%2Li$C)|KlW7*Y54o^g5}ptxH`%VG2(+*Ig#V(2($`% z!RZ!QrT~3KfD$jqi4wvO3#yL^_Qmc@t^ux=R#Z>Ba)!>q)vn0I_8!nS#Nq#D?39hg zA!*Y1-IMl=_F1Ub7u>2hBl%1o?c3EFmygPmGqxddy19bBHJoEaH zGONq0xI!UU=eOYr2AoSl0kiCV*sM{m>l4?Qm=zg$^%Ar=dr@S^r<@PHjV@lzeNMTH z^m}GMD&Mn2y zsaxyKS7tl*~My;A&}qV=wBS92R2MT09i;9vYvXTJNa zDeKb|w_9}Ie$A`2?b+NVE=rDrbnbqv>=Bl#Gb7LX%(be;bLPAswsiE$jyAmxJ)dE< zw#|pNVe_gky)G4wwu?<3+j;xE*5b+;nG361DLCe>ynJ}kq-c$1k4InHce~hqm_wf~ zYxCSyEsh^w8P@IgSf2s?-c}!dTp+2aH!P^4*?a6F7vt#p*zkp1o2YAceFqQC(H%H# z#om6i9`-m8F+qQP&%#kDCK(A{SLdaaJ*duUox4W=RQtSezlXl^qY*{Y4g17&nQ#1l^qk(;Pnc17}osYl@9c%e}NK4u+ll{PMjo}6Xp za8Wgmy)-te9oBGO9?&_<_EgD&am&ukZgndwXuq$n-G=d^o~!!Ab@h7QK~&Q5wfN;d zd$Er5K%lU^N>KBg>sxoz92vgPRAyljXZM!qz-6pQA014RyR2jP&GB9SAb#Td=uu`@ zJFi@{=7z9_uv$N*W0iNh8`&60^ss-i&{O?fNN0DYUTdo*rkd_`j1U_nty{i--1a^n zdpuY>C+0%0JGX*`7GHeBK6+O*YIb*}^SSY3%|fR5$qb&jymg0}UWUq>+nwboFL)v- zG+x>{w`_B!PsJa@?sb_XcV(ro$=G?q6}vAPdPD^+6&TUH!&LQqv3?6TC*E?qT^M~g zu7zvVjo!;d3J%Rp@esaOV5pL`$g}6DI9=~U>lWr)*c}hp-Z9@%{#5*O+UIhs zySo;qr{qWNnqm21$)e(bVG&wqBR08A?Y)1JT-){%W&`AQuedyE^J(jCmwnBOMy$V{ zUSad}_}knaCb8oZZ{5r7sV3 z>*b zMmf##`1EAE?QIu*(ZGuzyJzoDwiwmd?Pc`4Iem{7pU5c|yFK=_iqPirFwW=uDne&a zxNB-&j7TMVb#=>2Rrd4tJ`nFYr1iZ`R`Uh^V1>Da#w~Wan}6iRooN{stanx~eG;sD zuU&UJZC$I1O=_an;4%QPL|ES+h>%B)Vrhc0}b@Y4#^v(nSMAj{y^M-;Z+m6wrj8W zsda~r^QNXxGje=AF6VAlR(Kb|5f`c>m+bdnnP;SM^2)$9Vcm`lyT5qS%aE|8%e%y% zxpbia;e_pF36^8d%ILig@1wsw%=hI$iP-}KqrL6CCsQvo;vzzAy;}BYxhtoys*mMe zFVj6$pW0_+T+2DV@Opyz+syv6=b2jj#-?(Mr|-N>ofv}~lE z$lx5ajSss%S^h@l#{>n(4Yv|m={d|b`~FX5Q7oHwmS zRU&&HUp_Q-qk(ee(p_!b?+qVwEm%5XQTrD??5>4hbF=w)q}|flFZ&cfwLfq|aoY>8 zJs}&46MY_a8agg+VA{%S;|EVSdGyjCS5$wx%YjFkLHjGz`^i@oIq&Np=Wyx6ED6;K2&X_o*k~EwxX}jDb9#df^QFo z2UX}5ZV1UbxHx;&jP$jererJK8+85l9RD40pK2~l(l;;qBfnLb)1AjIPrhE^6Su77+*F%&?)+rRmP$m$E|m6^q+fp^?8ls%UqAV zdpU8&q|D{3-Wv{0D^iPhZ-4#5-E#+?r&?7iPuQe6rhDcIw$|ve(!P&|$@}h@tmx#l zphx_HPkM%FEdi_6kKbk~zjsj6=K==$8(S-QK% z8(BDuJofME^G2ob!0!7EZe<4;Z`%2^##Xz0TK}Bea%w&7C5+zY%iTLQRk^TtVNQ>d z?nuqI`qR`YSCntUrqMQOff*L#XOsySv{O#)8?$%I_HL~$qT{8{+p84jZitJ!GIYq1 z_>hpaZf`68NGmIe)g07!k>LDXeTT_?)d$GvSxobmn>i=>!7!uq;rpjd9wu%f=d67u zX|3~~2P;nm?v85S>fB*dVIQ-UXq{e~qn61&SsZXYAa%k;9mPF@ZF}68t~k)S?T+;2 zZ;Y328aH~<#~FSt6 ziqwjt2E+;3X7B`DisF?f2fI$g?Ht6%WPmgr-DrFM&nWF?lO}?V-!zUk$HdQ@gAVYm zY%&Uf`LyIf6EL9GtctcDam?-2CiX}ReKhQmvvP93Z# z=BK0%JZ)e&bK%i3?41fDDz+ZoUl_D%9_PURSBch>`}jIN<`|pa&UNf0++KL(!L}-E z)_R4M2#&p!yGW=ga(ig6K{4j7_O6j!usG7#Tqz2Umm<(M9^6Ql$Qh z2*a4x&k|w$YM4kPB5dx(zbC>NHH=ZizN3c8sI{p0t{P@;eja-wRfX2H8Y1_G z#>?Y{*U1}gc$V=X;p8l>E5nEAY|K8RciA}RN$RxT&tFdHq#ecH?j~HdPf}=#*SibJ zp$D(tIjwBE=z@~kr5%OCSjS_Z@9QalZ&R>kfNaQ0jL6>Lz_7Qx6?k-rH^C zyJjw{Qey6vsHa}(pmOAewUV;FrsUJ7Zd)p^DMy5AB;A?b{Xz2r>81N(_wF~G8fS9? zpP7(dG{pQ<_n5KoHeQ`mBy3+Q#Jc`6+i>>H5XqIF(uZ2+m`BDs;e~HvGOQl?v`nIZ zen5uRRl|tv;`N{ZxEh9@@}DKc_|-7cMr0W0Z^|%64P#{3cVw99nl`21Rm04^Xf=%N z>qgYDv2v{<+V$6&r*F=X;37zv3Gx8tKmu@pF@TTOup%R_Xjp*rlvTxCo$T9YQ zYi?D|P(PpJGGnvA(&*yk{WlL;o~bF4N?6%S;6-9e+nL=@D5`#ljoSP;=Ij&FC)2b< zSbgsQk_fA-hLIsaBD#n=Muq_YtR<=Qvs;qN- zC$cqVdcpiz3nvQ~jM>Z9d3K~o`NrybiF?cLJU*c5luS;O$#UX0CEqu>reuBc|B?)= ztA_DOxNDgV~O;FHK}WZ*pt1N`yAF68@l)1ONFG|Zf6WX^-p;;Lu+H=Dt5e^aQCB- zVQxW}5<)kZWCpUxPG%e3Y$orbp5MyxSbXxhUjJ(e%XmAP1)hxFI_bjrwW9`aIC7M3 zO0KzAqM15e?a8RhE>g!g$SVcD&9@3t582L?qGx5Ej$@yhbl`5*1 z_k4Ebv9=ly$k`z^RfCi2yHnfqM8=*~)T4r{#!*ve&c3?2+I?y2v}eP#6Xr=7tPr<4 zA9P5wuVCJlKXQcZE?$s7eb%+mOicdGAE#Tb=`|*C+nYXpJIYveRPykDWgc()ATm=| zyx-yAOqqV}E3bRDeyuvWpfqNiqSp$|OY@tB%<$U%dDP}4t7l<(WqV&{-g#1W%siceFf_d9>vVXaPjp6SJ*+SlGRywN_VrNHOAr5g&~BV6+e3)Tg@I2d~4Lw3N7GZgs_nG$TG3@xh1>Mtn3bKFE+g z5hL8b;=1C4&4>?1d@$mJ5g!eU52AJOX8bfHK4cj2!H5q=d@$mpVevt<4xacRXR;iO zzw5L)BR&}M!H5q=d^9dTh}Q9Slvt>ojHU5bleT~z`Ak#^25!A!Br_MwSaYfIm&x3d zvlUWUO-g>l=-+VqcRp5rk&nfa^b6XjAn-z2P4KD%9!$lsqRj2214saGNgwHllTUc zWKjI^-OYx=M-6fFcjI^laGW`TZoz?^34U<6l}(1mWyu5dLEKRkcO4z!k`gikSTwKP zR1>$f4bBa(s_TRaOuW35Ya}lRT==zli8%#0xOvGzbh&xiygVDcvZ+O~8T5IHtSFt7 zQ*5%2j1&E5MOjgH*kw-(cb3XNCEien*@g7^U->~yd3<9PC0)+z!~^&6{}T>2wM6y+ zd5x%3Y-#-R^P!LHrLvK-PbEgOHG~}x22Ne=w%~%^l*49eC&R@Rbb@JVkQGpN_9D|> zQWUXwrE|s?&rOGm0EFxV_Z-R$EKdeVZUxYHNEsltC}BKB>$%nuEf1~!W2DEjw2FL$ zVWuXkOq8NTYt(8w*J#&>)kslBbiQ-&3<=Y>j{k${QbkfiN(jH-RIW||m1_u?0{Van z(5Il`9CLvWq^yyBPyG-EiCt&#=#P6OCH|B;?hJzMM5LCKR#HMZ4YKHhXGkgJ`oy8! z>3raP^RyIJ%1co;u$@-xn+wd|@ZX0{B`JYCTOGJ)+?K?ot}V&o#-Am&By6%$e=EBM zrtGo4$PW8jlQ`|FE@d314_AA-k=Ulx#;z>|o@jWXpvM4W#6+x7B;XPhRXhBHX$10|kgCc!|DW zx9v-|b#>Gp%dzu7MgS3aq?i)huZ5FTIkcMw3rKtNFu-4+a9OLEf7_gvp~^)#u*ZJx#2NMKMG>Vk zbQXEVsk2tYi4BP(-K1TKE7F8pOdO&KM_xJpCLFIx>q*-`??!Sc$LlgHa+UZP7`u{q zkwb+tecg{XslDdVXI}hjFKSO@lX)Vr*;K#RJ_Hmo!y zgcdecF383!%A#JAt>tM85h@sOpQO+VJWY1)Ur75RN|#LXHa9~`$j<$KwEYk6+>d97 zH>7Ji8)sg*6u^BBgv}e2%jLMwP4pb>C|1V$yL-QFDR>MtN8%@HTN?IOZBxcmAOlLUHeNUGX`PF|`6 zYg0wON3aNPc&TF1{bqAlii!SN+KA8ON!~IE-uDQE`O?;z*rKEhpbfxnO_6p?W9iYM zedEfPL*j>6TI_3b_0xkSVCN>19;tE*+CBHprE3Gcn&j0YZg6nOPpuiIAMJsY#Us2P zI3YqMWKtCJ-W$d_@Q?H>Zdq0T^Yf4_=!cWx>xb(F_uuAg-L_C;O!UCBJXz(`^6O`6 zY2AkO<%q0drgUAG~S%BX9DwoZBdwJ9lTOM8C$`w+8 z*nHFo^HDP%Px7ca8};YZkl1ml;Wf6o1-dah$Sd-d4s7u!GDn^i41PMq2pzt(Lw-7T z(={|tjt6?Uafl>$^QSvByAhH|iQ%;rcqT4I+!90VzE6u7A1y@kkP<`sY)8Y$qopuf zOq92O;kRYwDkK2A)HbaxBj_}`e(e!h)jy|S0E}niyBWXDep*Z^7w$FJepHf8#xyBB!FbvJ2tJx z!?dv#x61rHom&f&YJ!)oFfzyg90Q-B;0O!U$~V5)lRE;puH4$tt+h;4j40JraRmWk$3z6eN^NiaG=k8JPr-(sCXkO@Ye*#3^+XAQI?4fH$6c z2oS~b1wfkcZvo=>CxA2th}0&H6SV5km4Ws-&ZsM?^*EC_msV30`P>fCEG>OC16=&w z+Xr>bOnYlQ@xs#C6aN7JViaExCqQb9~<=x|^Yab6Z53 zq>sSF+1qo1zbBo3BY#h4Z#qSGo<42?91FK;92-9$XJ0z?!)-akJ$+p~{CHP`{Q^Ba zU1&zilH$vfj&jP94wZx+D+xVO5*AhxdXOf;(%s$D#SMzKn~xhEqOG^5s~bm~l0+;B z>x2;!(?Nc|fdL%*05?}UL9in27Xd&JAlwQA%>WUA4AKy>EDm5VSuW=|O9P}9Lo1SM z4zvJBg-`N=?13r|kj^2rqp4Pa63`k@2HF5bg;D`HKs!JcXb*G%Is%;lH2`gBN*(9| zbOp#}y_!HbfP51fZD>jx&;fb?xilO#FWcPe}j$wjcbh)V&)-#>Q)?h9Gx6_x&yjQyWt6y62H?NHUx1%` zAgA`_v+|qf=5OWa=^IE5fPJJ&@G$ZWF_+5?8bexq#5`9wW%nJJ317>A!N8&-x}{k| z?uk;1WhR0Yd}ITpCJ~CMt@JNpRD#e!b1NxuP2j8`xhF#VMcf!ka3EKMY5pmS*b~>C zfh_>ZB^Iy6jS%Y}jh7L|hGBqVfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}P zfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}P zfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}P zfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}PfMI}P zfMI}PfMI}PfMKAiFi=%hrLL~7udi=xZ2YT#+qG-=8$#ix;n-no4$l6Oa7- zd__gYjEszA-tmjfadB}72!Z?}|9R0T{{ObH;^X6yC~)`O}zev*fchFQB>5YP(Oy~ z8;56XNKV$*AB{U1aW48HwKdy6tT$07A|QARC5syne2&Y=;0BbuVKlkiPtx^2QUCs)8U%s@s>t7#Gk@v9-xgNWm(Ne0JmHE>NX}n+KxWFw$o$eE zzIguI!s1p0;o;$bsrdb7$NU0{E3$skFQnNo^7&hDe_L1y2?=d1jK#Fz}yY026+wCV60j(0_(xZbJSQ z`HZ}6nlv`qh&4^R8jDqvrLoCItT85==KUs1W0Q^8-_A!yKJ!!e-^*YA)crPdlcllA zM(np`%})skG(v8uqhW2M9)||DO$e4D!e^}7%dW}B;t{MO3<|L^55f9igl8Sf*QY=Q@t`~5Z&n1C1t7zP*y7zP*y7zP*y7zP*y7zUae z17#SwPmaru4$p`V&%Yg>9Uh*Y9G)E?o*W(iB;M>vP0f?|aJb2ckIRpD$&QbMv+Ud9 z8R0J3(Jo~*$@#aFGs454;K90{^Mxb zisMeW;$D)@U*=#KU>IN+U>Nw97~l@TH8@U(F}iGw(uKs~Po1*CeA%C}<>tSXP$pQ0 zf&V506sE1m+5Ge0|0X+3su%_s1{el@4Fk2)Z^(Qm8z@U(COBd46PeT$XJc#=f>lP? zrC422Jv%#`JNL;3?rm(iQ=#AzOOv>(kZ=dd&;4AuWqGXEQC!Jg7haBAY0C}ETB z{|3)YDi{Wu0s~}?KKeH(vR)smayGqEKs_WISGXIOO$L>S4a~F^L^irB$sA_r)%Ye- zNH!iJ{hT&&^t@)GVKY6MO(AEwaUcyG-shaMvq@tIxdVebh6|CWbPkf7#Vz{437Lv6 zui8s;JDXb`xJ4p6`{#Tdit9h~%P_z&&=>=F`RL9JbZOKc>i7U1U&lqg{tal=VSr(PVc-WD_#&U7BtwDbuDT`a5eD~pqMIv$8I|}4DgD*X z1fj+LKRntZ&CLKi4>un-ijcqp$XzMG*4xw7jbrB-=uLR2{`?)zn+j*3@=uORb{Dvi1DwvTEK!}oqFPXI1qCTt zYPhG1hqIeEXSj=rv#%@OfB>>T*>L1N$!}-mj5o!>wF_l}D__bL*KSk*{{D9k6DkO0 zmXtDIlxhxTkJ!86`bQ5Xyw&`Y`A423Ts!=T!yk0>SHu57Z-3a?|8n>?kXe3o&JUzZ zfIl_;hto|Q%2UE$@}I~%nTUC=v6fexFL*}nrB13p<@d}kdO%b{$`#f*+#DdxwXs6z z)94aMOrEu+&=h7DluV!6a>?&YU2|tjfJN#WB4bHD)U|E+ioC^x@Sc-Hn6jnA@xdER z(moV1a7XC=2-}&K^M1I~_(8%Kf;F!UH16`N&3PDQO1Rg;RQXKn>^&r~_Squ7C!h33LOx16qJKpab*(bb+40XAohC zGmn>kFjwP(2V&0y&&~hW^E{M+P#_El2j&9{fQ7&!U@@=+SPCoyB7o(<3Lp|#39JI5 zfM{Sfum*?$)&lE*^*}7J0oVv^0yYC%fUUqbU^}n_Abrm}fnC6EU=Oet*az$f;(-Ie zLEsQ@7&rnP1&#p;z;WONkO-UvP64NZBp?|`0nPwtfmDE$gJV z=9PUS5`=+ffC$hW5Cg;k2|yB%0;B;pAOi>jEdW_S4rmF;0}6m5&heMv;iHU2cQe|1bP8_KyN@F=mYcx z41j(>f4~qJ02l$rfC*p<3XHmg#Qh5Gvpu1dvcI7k^3qDDHr4*X5xm|A3@|T&mB28_B@gO qO`*>O(fE_2l1=I-Qg`v%{{nRTKS(4vn7jeOv>a{BlmGSh|NjAY;M;fr diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testPPT_various.pptx b/solr/contrib/morphlines-core/src/test-files/test-documents/testPPT_various.pptx deleted file mode 100644 index 92c2744dc4ec866214d9e00a0a597ee246a21be6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 56659 zcmeFZRd6I*mMofLW@ct)W|m6K%*@Qp%*&f@KJ59&B}%)YdlNm85?Fta*X zdUT9W?+a@Oair@P^e>c9SgQeK%T2#$%G`0_*!wr;%}kDR{3I&Arz9y|$K{tYg4YMpdUjui;{*4vCsB6y&l%IvL+oVxiQ zX)1`fHreF-=uC1U7N0OdY6-&>H>#TMH!mhmGW~DZJR=p6k1Osz_P+uP;icHu*P{ZYTdq^+l>cTP~CdcJBbxRT3<6zqqIsPywl zca^@qHCFFl^e_)VOpbq0=x!fsz$p_l*=(Z29?r?S@1bow#jJ6A5s(1Q3{!8M1K%>d zof9Oeo8#bcvni`6)(r!m+6qG{jy(#*7&oY95pK1&f&kQJM`Cb!MHFy8A-K%T%cZW# zEfwpHVI|QkVO2`pGK2%OiJetzMF%0T2E(T|$QHgO&m3kKXbc;~+DdzImU-NAcBX^H znJZ$9yLirrC;KMmdJh2re0+ca6#jv1EFN77!GHjOU!Pn8^GP=PPNp`_4D^3|{x8Y= zPo|c?{r2jlb!iYL#IOsnEutx3j&*9{c^iAt!`!iFfH89?&IYbz*?doT;kKb6T~K$< zf`I<@j8~I;u}67|?Im=YDqK(({K&Ijzw=A|H?Tyk?tUf@C5Hmo?1O8gH&GHNtcl=Q zO;ogeG}x&txru5Br;IAZ@pNg7BoTVUV^h($FurkB-L0`(Wst1MvK4GFOY2&}=5P)6 z4+(8H@U8Pje~OBd{~CZn8_zhPc2?52s-#F5YIBZh@970W3G}V<>;H9 zK5_WxsENLQ&msSe8b?q7fbjDQ2L~4hXB$fsQ)h<11kj%cOn;>6KSQZ2rC)B45jN~X zvP;Btn)ECnaCTl7V2P|_z*GFyas8{U5*fbMR$hRhZK$}nK8KZXV<3o_# z^=W-*#VtjwD&MsDSKBYYDa7ZNI*@wSITb-3k8Q z3WvR>9F#6JzPP3$q|yN_)i>-dM-J4}`+VlF;+`Q7JQW9JKwIcv)iUt@9&^Qi25}M+ zIwyNa#zmg?LF|>5yD=YOo$%m;h)bGHkZg0@jhnUdJrc;zN3bL53ph{eqiyxpbhifD z)b%$K7je1ii|3()?yASbVo8BdB>x%XL*TFL%%5jK!2kdP03^^K82<%a2PaeKPd#Ml zVrg&ppAdht70{<7`n=(P_E!DlyIcq(BE)597@ zDWM)e!FJ)K>N`(!(5buz2R6vMn9+b0@!JsB0h|f3b)tVH^doiD z($6p6^)5>+D`-$GoddP`cWbq;{`RVAvhXw10xpC{1IRf;J)sE4+MUsaC}qIXE2OYB zX;+g8+WCkTke8~2fm|ZwMAuSCof-)InkK)&&1V&^caB)AeX#3oqaqC{6A>6Fi0|Vx zrp+011_Wj+92%GSHAlgFwqoX!7kI>LMVu~B3gNbanZ)Uba^*#B>quP-_9M)gt~I&h zB4^HPZ!!MWYnP!kU+Dgs3`O7o0Mx&R_J4ihPSgA2K4n+fMd~wrVjUU|vhbLANa`9i zlqF3e+zWsli*TTUWL;sU{9+}8yoB!*WG~HQq}ROXmEG*!sUp)Iyn?!)1(`QC5~2&y zboc?wbBk62>MEtItK^Hr*cckBr>4&vv;Dky5O8h9NCp0cL+X4CZnK@zmijP|Rb`$i z2(B>?k#*&j>iL_}{1&A?ivVuCac>zKg}s)?`S zw*ishPuLu?e|)QpkhL;X4(PeQxVAVm_%gadGcE4m?Q;J-1Y`Rvml~BCY=!YEj_|vj z$i@0MBZCUXft+7fd7jy$=wF?xw z1q-!s$Dnd0l}865cZ^qxBFk8mFZ)J?9ev2@^0Vn6_|44^;KqTsJRuYr-u(X2ZS*U&`z7D->R&21=Op8 z>~pHCZBNpM=EpGmAvmS{Qr|$P-2+(#XJg;h?twR`ln_pc@FSwpjBha0iBkXGGPzk0 zrdiBSFrdiSGN7kfD6nXl_>DoAv@Sb;_9DSKnRaWh^E=oQwk2oI4$;i0GrT~a=j*u; zY9n9aJYgDL-Tid+XvpoZ9-+I|CPnU@gbcn6TRI>9h*EQxNTSyT|I^b9KE-#|Qzjj{ z1#){&kecL!sV~O}qC*SrzeFW-PDZJJVe2nWf04iP{ed0Ia<{q~ni&BxKUu48-p+si z0HV(u`({X5(wa(%AyW8wF;j74V zbhL*uuEoJR!P~XE|K(LkN(^LO2xZuC=MvWYpS?IO!7^)sWR*DT3PWCQ(awwEPW; zeOQIv$)Eye=Jq%d=IZ*dDEndrVYxw-F^T+Uz9pEHX|T|;Vp_Z8XcdyGniJTk!7)_QHB9Q-U$tE`t7hnaSl`?Vmd?{o(SN6>F91|b|5Pe& zuy{r;Cak0KmhPbB@tI&V7eUJb;~(k!lB~d6DdRqPN`lk-xRVJDpm51AXhRRT>D^y6 zKY^LF2=F|-d5bI5tx%?6Cmdtsxck1&yG02C~ zzzqHeAQo$0*beG3aiF@KBhW<2)v^`yb;qGnx}9>0^u8v_7ux4>x2_qh81CBpj1UAJ zrg+I2>p16_ey-kx;{hdOsNEUq*4O#*{cf!Z?_hD!nx2RET)b6|^ zgLqTZ0ZI$^tnk5`UWi$VX_dL>T3?yxij$-4u&L6%stD{K^`)_tSCgvnrtq=ALIgc* zCi9VGq){+!c;6Z%9(vNEeiOjUOv-}T(UcYzF8s;3LDqKRO44Ra!t>CPh_^nr=)Ezf zS~ah*cN&0QgV(4>*SLh&$Zdj`Z!vYdlW^CA2@J1kKVX*YJa(M0NOU67q{-xNrBQ#o z3}%cG?|f&c$-?^<5*nnqF{QxK4t24qg+lF^(2)Ye%fc>1j=L z)gIVb_iLYF_23FbTUTl*^kMKA%wshoapjid7tx}mY@z1F-6z(&RdDoXIsq%~N5aIj z04^uwax>ciW+Sl*k9_pvg(Nzb5k_HAgm_|@B_zcN4fhYvpCD)f2(G6{5eF*q$t$Xp zEl-|W%p9VLp0lD+`!}yvT3r6*zar*@CY3EBD{?PvOS14mSp%veH&jNNneBz+3QN8& zI2CU)VyH}u2BI4UGJvSCl7gSHH_!E$QbmfA2VP;zVu#>iw90JE+-!O@!Kx|5Pz#Y6U^{p?lzC*JIg=5X zBm-%Dz>2Kl>*mKr(2Q5KT{8kg0yehvzgHb#>iWaJ5KVNh?F(n@g&hEaYiyc681XQo=83Ql_5h|Rd|xM z!;t%aYAxa2Y7A|RE|h;t#7Yeh7TBYD7M$BSm57H$2t#nO!OC$Vi*fGRbO&Dr<2$4Y zLWsJv#gQlxWpWV$zdp_ob#$;rduu7!Rzj84GNrFTn%6L6LI*Edd!l1juJ|zM8sWVi zTxtAq&4D_V5m{t)05@#5Rb&#mS_)m*2oq^zPKFv)y5dxa$ZR1ugg8gX;Qiz;2fXi; zJvKp`CQ$($>c75~`cTbV&-SjIGeQ?Vf?)<)96&BNZx=PxqSZd}OeL}}Qm+iFknXl} z1u3O5(ng}sTfeg!IXk-$|ctWZ{K}?v@d{ejE^EWx$z)Q5eRcS{) zdkvQRq0?ObN|vKD%1zj49!ZcZMmB&+^Qoowh;b`7%rE8*({l+`r(Ib}-BsF{l=wlhC@4OY zF9vDwbF58Qa= z$dz?~x(ARUc{p6rx>3DBh;?~fPE;7Qs(YXU;=)*}nmh}Te%vvXq7(|(qDuD)GIi<|rZ-Kp1hZmY(yj8-(H;j~8Fb!82lJ)j)u$a>;!!a05>L~f_3azmmF$!r> z>4QbOtsOQes*F*%|;BjC~E4$M1kQ98R;nNiD)=?fKz-m+pfzYHM`Q6_E*Sq!w;Sd8@J6P3`*KQ8TXG zYT%hhi1YVjyKAF!T{4#qQzZEX$Xn4dqUj2AM?+U7GrTr#t4@o+Uj075@#JBcRXAN- zFihOE$kwkt+(IhcLgu9)xCq@#fA1B((;)g?&279s9g05A-F-c5hHP*{jpboIi}2Xs z45RWnIGY7G#2WzNNT`AMmg#Oarunc+vM^`7C3L*yk0{4PObDh`zayM_BRk7RtZ*9T zZRi(C-1E^&^+|-ByI%#vLNM19T*+e1T~}?gdE#@qIfw;``vR0sf-c}=E~ZJkaF=Sj zYrh ^FV70IqTiuH3TIMGaCQ@gooy+h#Of7tdhGlgK?^BCl{pR6x$_4vNUx7GQ((;sqHOnzydnv|{aHdYRS`S!CgKn93#^{|T? z;3y-lOiNWWhQ9zpRFiLlWa-6VX{4?Xbon!=)(z1GgK$hXLj3_amBm`HA?bEux%UJ9 ze?AY$a2_!jQw1PWg zUxjEp()Tnbo38eey}JvRSe~-h>f;+mnjk8Mzw0y>v!y(4I=nZvCBh60etO1_2Mhu# ze5kug=S)t`5?iBD?mou==Rt;o99H`DrhrR2FD#L}T!xzynU=+J&9AO|-|HXfTBf|} zS%xO5tgB1TcHZ($ZnsVKqxQ@3@KyCJd{d-c&9+)iD7io&E!~>b&2=x<4aj%qRI3%Z zexJMonehjoYt~LXb%&-^WmDOWdk?kl`exMl z;lh(}>8`0l@Q|s}wGB&mxq(-*C@cpvHj`)i+8;;I!@`ze44>wQxBtG&YHF%TyC>vBE1S@hM8fZNvHd+)}>uP>vy2|xDR45_SdJ@x}U(I#kTKtAvE$3(ZAaGCi0EysgkCY%>t z9sl}Q?=L~@E7z@wI1t(PBK-&BoNU9VXwBIkB=_#httyUfP129=!_t36jT!`R`~sgc8O^R+GZ7O*PoA_|L85ILS)#u zw?8BfG5K8zo%J)dX1{|b0yE4idz4(rUuGXqnA9E>2(;BUq;0v8$O%gHcqH}8Eym|x z-7-pCQ+NE%)*5ENuI}htg&&Dz^VjZ?mn$@f#%EcNQA3-U# z2m7FCfpuo&nfzpzm@P-9Gp#1Z|3hh%F--6lIE!;DlQ&ez@qxa{pp@rP&l9*-pM!T> zJ3-He+LV2_J)y{toeM=H;+xJB1nMl4Gu?pnGCiQ?`9N3Dm>Mf)@BcIkhOc^ReElMG zt!*=oRM4OOjiJQO!&;3bF{m|n9v*fzCN-knN4Oh}JkH>Y38VBu-H@v^4B2yM&^&B7 zmQu~*a8rIU$xVvN{J<>1BMV_tesf!4k<<8^ck@|N0trTv-=V0^%t!vRVX{1cOc&kP z1xK7yK+_WPTLc1XbeM7}apZUK{wB`A7mCgz0!a`2dVJAJE-omgKFxsdFbp-puv2vM z66=Rf#bcVXCL-&VN0gm-e7yLNAD>3RKU=Abb4gb{pJzBe-BG%~w+g-(dfL0X{DW!z zmm#LVdDkKF{m=u9h@uxkABgvM^InLFWqP8Q+hVsMX(sF7>!c>dhg;dkVEwzNjP`C> zZ`Rnu$;h4CY(E--g*HJ?eCl@G_-b1TIg8sk!R4_xhwAOqqYs2h4v>HKi=l`9(i}{1 z{z)}#Rsf|LCfUQOD~WPItXxN&#dBY>q}rlrIUR5xO(mx$!o7+;bcZK%b5nUy53P`~D{{RHvQUtudm8eaNrz zX}rsWq%fl*Q5T%4Sr&h7#@ObDk36Vb4eKx4c!|UNIiFUmiw1>e{NmF!!^L%U=|aM( z#bQiD#`2TP%6N!W)srk8tEao&0*es^DD1u0?^nQkz0fVhHvkrIp z0%=3DNze*OX=X^1FqRPRE9)dF#~#Ha8t+l-`kQEi^sn4*&**OIsHjF3G4&uRbsnY| z6o~fq#(8$H1^cTqxTbA#4FPFO(6z&$-|y?kID;Ab<9Fu87+!V*S2lbcUnc@F#giZb z!O0u_=UR_cJJ_^I6bF-sktNv!5pxM81#q^^u#au;448>2wzx%im=8!hsfL2ceyX^R3wv44cO#H0`7N-#Ce zR=hBG)J=NuE%2n;Gr0hzP#Qd$_i(#rLt#D+c!|_S_pz7^dF-nwauL{nL}<4;XippD zmnbkELS)WF5-rbq;eIvC7+*RgPJreh^os$STt2)d&h7Zol=?0B`CNqC4j4HFMQ0H4 zDRqZ{#t4@sQPbk)BLv-oS3>!cmp86g-s^1K;3t0D^z(vU)v~}v+eofu-Qqbsd2s)& z@`I1lE5ueCHfwvxDX;DgC{5#lEEfqQy8BEqj;l|k4kfCnO%FLm133n?*1aQ}m#j*$ zgW!iC)#gGkLkjWjowmVdN~+kUBbkU;&2s-3kCy?i4%_I z4x?k1<-I!+y><6@MEKoOKb8UJ9$G4nt%x42@YijBz~Q##%KYWh!!BGczs?oys}BC# zXZ`U1J0<>mnSlMjW#s?gXGFID#o^CP!2G{B{E5SVPo-#Cq)&wZsUb%{Yj6L+53+{N zE~ZZZAesL%^fw%a7B{R02~i;~U-~zNrr(T`T8|*uy0L->YQ-M_$nyLeMMZA3_5xI} z$dqxyec1s~zh56|4m_T*ysl$M+@zVul29Qhq$B0nVfh*C8F&bn9H~6&NSQI`1}c2~ zxcv-fa<@#@<+OiQNlq5H$`Meus78adMb1;cN!LvEZSxQ;EWIe5oKWJC<0T~ z8kxC9o{{B$vddSaiQQOZJikP=%3o&lK~dbaW&Z^8|AvQI|69f#RNfb~3J?I`;S(j| ze>F4tv!?s^le1>E?@kMhh&?QWJQh={D{?jzSFA@RQiGxIuuwlkYkuTSBqgRK^f4|h z2aWDXX^g8$FED;%)xjNIdux92?WD=gv6GHk5*-U_Dn#kJ#}n0ev`|Cnm6?Isa;}oyB9#!?Am(8( z;YVwbO(13qB`TS58XY@4^Su!qO&|*}^gP#h#6dEjM?jfa6czJ5B&G;VqvFUHH5j1? zH-i(7qAd000T2D!IxuSH*rftay{6fW%wE{3?csIj-Udm|Ra!AF?tZBHV0IA*(Vxc@ zoI9XFf|hSg-5*&8>JT;yude|u<{SZ#ZrMTas=0H!F5K+7d`dyOUM&dVF>SovsS%En z8aE2RGxcieD&_Ppl=r_biCXgE4v7IxEF!b%wNw<2d&!8j#QIAp;JSks>M#5_3UzGz zNT`YK< zO2aJ{%&#rHT5}lc6xMOwLl!z~Rp4aBx4O8t%sNUr$vI1i5-*pCH3;oApLB8Pj0)No zh|T(>u)ERlM{vK<@kc^^0|D%_*PA~S%w5=F);X$A*=&|wagz_B3tgEaD!!Y{;1Y_J zwVV>Xaa0mpA>DTD5BUK7y-JWNLdnpGTEPBTq!UgCj+4+GaZT;V9li zT9Gkl9vWpv)t)}Zh@*K+uZbyi7P31NHiq7KnS~iQ$(2Gg^5tEo){tHKe0}}m^F5#A z(T0=w(L|iM;szdkmk~P-cnU>axFCZAf9EWyJfX@;%JJ%Nnov$H^Wi3)mYkQwoY%qI zJz7M-XFfWL#55Bz&V2U%1RiAxc{Gzsky`2UG_(c~G~e|PHk3=a=t&w(P2{3u3MN&b z9OE&Z`T5f^z9|9U3Rz9vayc!6n;Wjccm?ttNhouiB9NbJ{=8^6U$`N>gm|kZfOk`+ zaXo-7vevE^xcKaZ^L*HaY_W#EiMv&ma%m!D_E0Ob;ZMPJC!&KP>D+hBSc zY15wMP?$Tt*S0n@p6)`XT~cBKj;t4x=pOGjbB;6;tt6p1qq5+%vdaRfSP zur1>XR_1XsU2f>gy>I&Kf(`4MT-d_T%C{$?ciOmxgfAN3CM4-;5alS-5PK!(g)y6d zAvOo005Pt4bC9-E5o!~P(=VaVyLE-2En^EF@8pP6j$+Z$DoH^B2hFN8%87iL3#u+< zM;Q)6DZ;(JW??uOjb~;9A$$ZZa#_Hl^_?Zq41#yyL9*hqkfYyAj ztnxg@$^r5xI@}@aWK1+iK`rB2yAt|XYgg=5=bgcGDX@YHk1_i$iM&}&>b!`#G^?nVkxgQR)f)&J$G+-^Zb0XRbo5;f#YK9t4YRDR-kX25ou4qa$qKp|- zxlVT|V}Y%lX^g4P3`_}HRiNtf@)+(!q%}`BcTpW~<)jPWpd26NEX&I~mBw8`qGcc_ z4DKu|C3OgN*+DGP$B>jl5Vu*0$(K<_rR|uBi`(T+>w|qE2C>0)MCxINOsg3UZwBy{lO&E0IsafnwRxF&yuU^`*59Q^f z0u+a;zC~*4bRMedEWX)Auhwjk1f1Vd= zz5>UUeRlPFBmJ=y=kGdu|HCT%yFC1-7gVjWp0p;8=qs=KZYb?JVlk`RvTd}w(N*nU z5g=fY0AA8)mk16kS31#~>J!&XxaG3CJZMOsriWOc~Bxs%EIu%&%Mi?AW_D$iDva zgOE%_oGDcsm7K;5>+EQc^6Loe^I6 zq1L@UXIf%rW3W7tHI?DUJ=~h6jSJHT%7ap!=jQBmQh21wz}c|x1hJW~nSoe3u==!H zl%xk2%ck7zhWw%-r=!@U=K=Zk5l=o0k_ua}a_+&0PeHbb8~XM5m2lgjqq0Q5#m?p) z&SO$|CN)TNW{cHZsDex5JufSWHR1h=URu2S7vus9XCU=OOW=D2zmC+tE02h*sxy}z z7mP|OjN-)VY}0wzb@Va6iK);ow8(ywVq{OVEVtNOo;==?IjX5v4hR)yF{}IZhiHEHAnj65c>aph4y#WwKphw__4PzlV>U zWWJM=3X9+x7t?OQe+t_}lW&U2NKi+yp&+Gh$|$eVSIi5wgGd64gc>dFKmjcXW~D&i zXDXIqCt80r=2*>bS;?)%%Sa{Gs3Q}i-jv^Xt*B<46G}+u5m}*Qn=dG^U&$heDHvro z8pdY?+fzH|vfFH!_aR3#M4L^+Bc^;qv zDj_Ip38-Na#Heh;AuM7IxQ9ag0f1jQ3W=BpaGM9jQL!xdn|@xhfkIfMo1dZvi&P49 zI}Na%Jjy^7ZLgMkvkI?hHjcdWpvMJg-ZKvcxx)rU3E?mr@0dfRl4=Z>Bxq=-FX@!N zQib`p@`V;eOKgKij;@uAxE8*qbW@b*O`avu%Aq2-OG?C|O1uGt&E@bbQFEyj32u8^%$ni*RgubHi8lLoCvVg&H1?NE26415 zSHzB}P91=Ce%gE2t197N<4UaEfD`@xmT!w_t+?d*uUsU z66fBYWp=IJ(C~S1?WG37EQZKU@cmQTEUu=Ez3f6EJl>SfiJd{mrj3Jj=k)gdd7jdS zGu*~)Ty@Rt&1bOCY4-ZQ{cf|@|2PY*YK&iwzjcVT2kgasGQGxCSF#a!9dCdHi)U3p zIxZptD_S%1PG<~3xR z954+Sak;l-*`TiXAefP@p{4X>mdkdE2L4rVId4hR2}S!Q^j4_iI~d_bVa0>aW@DK@ zeSWIu*kmS{D#zn}1ph`<`z@=uRMl|vw7)5fM~qejGm)JYo6DCxJv#odwb7ipHw?1D zfg{!@LC3=Fwj|xMYCw3yR`vCDwz)dYeNtRSrN3AM4qC_`XbX)1G!FxTT5CzJO)saZ zN0WXs=X}W)6FFQ3kzgL++O{vmUaLJ4LOu z=$1gR9I;FD3`sA;^)N;4J*ARJA?EQ%xl^ozo0 zTnjcJ6s#b7mmq6SQCNdk4>!QfWlAj5lqM?WFD&VQQtr2a7%|-$E3eZmdFr#=`2G*76uK z1irj}Q?|({W6FWDM~fDsr>jt-jRbCHC(RI@&E~i=e&VdaoEU2R8Qw>PhK9iHJkb$w zx-2rGUVtu&Z^Fd6!aHDDu1ddxEZQfvDwwvFo^t}h`)efzrm++kJ_;`nw!lD>h9y~a z6R46OVNRpUeGj6!`Q313M?`=T*6XWb3L(qh^Cj^Z{6+mJ&$URd*>X6BkekOK{sj?i z9Rg#~DL_IZV1aPa($O~n$rOU~90)=PI@3FqD29@!Zyafx6j#r3nrH&vVGTsQSv|`o zq7j6b8yHt2lK-WSAq=UMzw*pC{*b|8W6<}>f8(*R`|7s})Af|)^b#CH!2zb;uLA~D zRIXm(NZBXkb28tT>}V_+_3!xb=vk80@-9G&%d`|1hdq^A!oIP?*Y=H)vY zO}|sP-}pA*e@t1R3bss9F&A2IM^BH@@@|Vs?)R&+i8ZU!kUOir%`xj++YQgwgE&<) zd9lb3=~FywL3g`)znAAxgwK8*&p?gE;ie;X!wB6PJE`_Z(e?5g{-(m%aL%B4v@$J= zpAGlm3!9+6EB>Hg-ao5I|4ca2b*@}LKGjt3r=v#wS5@`5n$mxmX@48}o0nFd&~NqU z%BkS);5lEl`9aKLIR@id5r$H4fFVt!r4h~~&e{610wxry+p5$n^7C!s**UZPad^`s zCTw#D&~U%_??Hp@liR0jaA{~T31wKD?Frz{^CwI1y+~=@q&W)3ROF#P4QVe4RG#ZE zMaI7rD14__Qn7CWB2P2HM`RbejP#fbw3!w+vBtdHfLuGuyq@gcPQC4)NY*YgMLZpV zO|Ro<60ZA(1C}1*e#%r;(QX^Z-wW^w)6zmD4S1S19m3EL)M3?bZf^Wb#rgLR{J&Y8 z|8rXS=fbl8PDN>PGykN7PrH?f;lE_1|FB&Dl-_@5ww`L)+x-$p{m`p$um2g32J;>* zL_{N;=2m)Y*F27IP!Hk2vQo>Z(5PD4En+{b-Tbyxd@A+4ncctyWfCb|LMj|(zh4wVJf8 zoCZ#&yLX@Tw<8bS@K;v2-z<5O!=%51q&WSVVw;Ng(#{6xCOv^{EFRVp<97UZLL)A{ zSrzjMo>n4?$s{Wl8>e)Q23l#8>SAf6j9m$4fs0;#GO-Iql%4>R5SFA zH@E}?579Tc9>6jSk}I%#(XbXm$#cfQCQ(;w_fq4LuIalzVHChum=3~%SJ(oEC@0sY ztio(qwPu4y;c(46x8K2!&7Oe2Y27JxrFP7noviG;(i>BKbEMZ8EC2MN)Fnk-L|WiV zGIC3IR1Ha~eqxCg)+C_Po}{Qhi;F*@B%5@~Md#htREufsN)WhmG21RMKj2L%fwXZ{lN8fRLVP|Z)KsUJ zpFhOz$L!}YfJcXTDYtxIWgS{CIn2bHblurV!apo2OI%hLx292oRZ`$u=}$s@yq!`w za7hELt`SH-;#{IosvAm=7SbfnNNj7X^xL9sy=as{0Wb6*^DS zO}&H)lM?eo%{QfvG~N+Gw`bchOEhd7hThFUp4$!KK7E06E0>3AknbD9f+u}B55cDD z+BX*5L65FPo{+@utY+=3TdL#}pdip|h2%I=kA|6T!%R%;1#(QPF9wY5=3SR~2=8_o zt+rfpXpFd9O*R)t+V@&rgvxZqz+w@h^|L9N837L%!gf1BW*)fH*lgZ_gi@2jAX#NcSomWoZgTuHs%E=oE>Bqb z##rgB_i495J+j``Le$rW7-gAiY9Y8Acf+-YF_l0Eu9mqbci`5TRd@$Lt<)zSXxHhK zE54T@xpZOJVw<+!-NqRAav!C0djf&a#6H}t9)_7E&3Jz`6Q+Zz=AfdWdhh<(df_>} zG4VA2)=CYUmuIEsx6bXj(Yn}*9o)xb`%TDvmh&puZFlcB*0hB?aKVWub&Fo%26-0# z_FLz-lREwP_nnh;|NHHl3U@qjnNcHD<^|h|GIm@bPf1x<6Ic=GRwE4NeQM>zaS5>C*apu zxEi*mqMN8*En;n{X(h;uRqbwn?sdef_!d6*mSQJcNmI`of@^5DFF4HsB4RRpVQi6g zH@3?bJX_l^HQ%D@#M%$>VwMAFys6m~Ew=ZQxPif_fl_a7Bht!P7;~YGx>)*(bsFcu zNP~XV2zHfP2|Ku@h#x~LN_Jd=w`w!eGlt;Rd zNrD*4)x{lL6rMohCr1Jzux8 z=0N9EhB0(BCZcsP)V}YaS3QIkuG>kAFsBtb$&9mhzcg|LK8F`wq@_zegM__I>~nt|q_3jtZy84|6i zWbl4XU!%5Owu6y<<=<_LP)qP5q&sxlnL0#KTEq^?Bu_6!+l%EX(A?+@O76vJZ z*sF$rx}zfJx~np+(1}+);TXH|Jv+1#beYujK{(YqUMXXdT64cT-;_-Iozm>1W!5=) z=HR&D)CF&^sno!E?Xfd`^w5Ww18hx)R5K7yf!=+#fG!ovR0aYrD50ek+oH2l)$4Nz zi_^sAM>r>>h0M~4_gQdq8M$Aoj^nt-jq`)AA0KWo0(mwPXXPosMq z`oE;}f8nJ4Sxc{0)BgN36V$hhFAoKdy?E;ZP#yuW@w5578N%oVE6i>rGPR^S7Ue9f zYMnXNCF+f`*wW?&m%HIUt}YktqY}ruH7rrk#3mD(Ti5hDFi7avl%|tCcl8--cra3- znjuu?f7sm3wcr<9ch`dRhv4>)(nNg*%zeHziqr_W5@Ar|^~)#Dma!!}F1# z@6an)uGqEfZp=B%Rap%1nKQebaN{B2RWXdJL6E`?L9eDs2aV#B2tt!!i2!4>URfzy zr>|)Oeys^m$Y5PwOMqke->ky0=8W}{Xk?HpLdMgmgqeU$22djI6f^=c{u%RusZJU* z;L^mAiUV!AyFzG|I@p2j{?*K$>O_b7I5GCa9l)62Ne>v``XSAi0U%8p)U@E86N`+W z7m;wuCwy~q-2jt%UDUPQbhI~JxnlbAlEHN~)i9$BufaizNX&3<*C$iY9bLuXqbxR` zyn-ap(#)D)6Y*j3+lEQ6$Kv&>KddLVz6W5$+Fzo6O zy`#inIl);};f(C`C_zO9$p_eE^aXt-mKxqUl-(F5nJxDrQ*m;*t})(O@L=HP9M^2w zsBGcukLp5dB?7f+aSQ&qj+X4dc!lF7SXcp0UNqtmXla16rKfn3fK98*ed*l7)tCS2 zw(b@#vSs%dGds+!X#@Mkw)ezS`5|02Qvuy%KF^T4yt zDirp1?}mN2E}MB@2WKPS&p5=zO52ERqQ^(m5QyeS4OcNzZhE5I_pzKZBq8^zQ^#dH7m7?6v z(82CM>rJZ+M_AI$xvXLvHi&z}Y2!PWQ<*G#V-y!0r6$&eYelu>aSzk38DAT@87f~Q zZ;tMb5;&_DyC&MJX_y>ai(EJJ{XRCNl3SU3_tw2iv3yi)Cn%l!~W%|2xRdT6D=O?*e zel}K$=b-vw=()ADPE(pWZP98vC8=*5B9c`~coyLV9^0!@=SX>WdvYCsF#v zc@F~1Tr`3s)`JsnhH?Ux?NXbqVn2QCBa^bZgOjVcDzRx*m{Ewh2otlXo6Q#^qqxtZ;~3dasFX;2%0lNNTAXt5o*+3Va_3QNnwX!W zq*UtxD;i71N|wx5dl3o5c}rB&efvGKp9~T|wB8Ch;7+ec6vt zWKU;1WC?tm7V*muD6=o(#IDw~fzk=xpem;fZhyOfx;xOdvg(!%gZU&qM<8O04_!wL~IoA#cdvEGgZh6i8 z#;$?ODkV#Y@!oP@#%3D`qxZ_dIp|>j?t;;(+~Awlu6m|xeK)wuJ>l>OZY-tA(!x?7!`fgODgg*%~`@koYbfX=C8Y< zoBL)EAUeO*1Da;3_OI4f&+zMY+hl8Xp=!E%ewvPn#FJyVvMC1Y zRoCZ83}yQtYVE)7FlzHlMw+`VaP^g1HZ>84^iJW-q&)Z`ldfzW(8;d0PhW$7x9^(l zf$!#8X-9PdcNh;p-%aKV_`F>&!&ofdPHvnq;2-g=b^%G=C2}lVJ7?_F0UbiZ(S>2Z zVgV`=jy6MlRs=BQKEVGKdljSk_QHL#SL44?jr+4N=u~6FX^j)LXDibeBKBde1+JAT zPcU3?C7XR5^%)RF2hrBJFC8SvlY3-+#xIv&BGD)Y>59JMkeNL+md}6W^IKxH!=sa) zQVWURNe!O%BG#HniJbQPPKs1{Zrlt&e2T5^4l1~2nZ0|Y~u+dp?3 z4qS2X4qpO~7JzCY&^oM$zlBP1l3Pb6c`O8ZImOxe#ZWlLo=$4AZmj?&Czb!+|;Y>JBQGweHPFRjsA$HZNOvf8_sR?>(TJ>bkYjAWeGjLX_SEN(Vtjnl$N6 zh=@ohDjfn5>0Rkkr38e~tJKg@Q0ZL=2`XI@5r`1NpZ7iEl>eM>jC1b&$GCUgJHE}> zZ1&1#?KRh$YtFfzIoHhSu#@8qaYiquTH)0B*{ThZqhHG)aCYt>Z3SF&(xMoXcPKf1Bk_}a5vzZX z^v`2|nu!v+uT+2lgaUQycZ>2R-P)k-vzzssOB>?3ixIQHl^o~UW{{Xo(dKbk-R?=d znN;R?k-eXb56}N>6eA;u_!IiZ>RtVCAwb&B+AY{`3!;FHvear#zcJ0wlr59hQ{_vc z;#)#@pXGLCqe=MTKUj}&jDBm}Cdm?d*8hGz^7n3+PqyRFN$}!orR2pVyoP6XEAl>% zsk67+Mx$fopBK8G(ysPtExc6e_vQ%Jzf=!JI@;kQFP*8J!(KCF4{w*9X`GZMo#Y$6 zQp~Zk|K^g^+blD1_%42i;Ncx!fI7aEWSRI~o=xY==zO-#{MmRhtW$jO-cXRYLk&Aw zYB%lD;=N4C!3bm`Yf{a5#$wJcp7D&vmTrsFM#AgYnVkZKAs|4l0f*QdYh;x(;=L4BtF$UsmhnD6>;p6&7IS+tG*-1-hE zK!d(_T3#{sryx31`<)DbYX4+oPgJf-8^6kI<7M4gbJ*n7Wr2lHuJ4p=bh?M*W!FK% zso%}aZ)<;&aA31VBqWIBBVxgc1S!(6coozbOy&Ji5Hz;pTq z8G(2?x@(MBGCj&c-EScW8}z*L78d&Zw($4DYQB?g4|$rT z&gqRm9^zjc#Ok)bk>`+(7So2NOgy!7J3CX2yZSSzzw9B4DJ@GBOVonZEne;OJi+ZE z=qO)k?tSlAm!%VN_=|VV2c!^+5_Zp%-Pmd0g#dR!=calGL}9h)cj`BR4X2)CoB&yB zf!nn!?D!}6UUj^!FF|}l) zp_ffqA1*46ONG%YTD4oYE?5Mhwthr19qK#2cWc2V>*)xv^)Ec|f4k5VKy|KR;2IRT z%IwyF;*_G9sYl>sIe)1B-u@-7p4_2qbz|Y((iMx`XZit+-`Yz5ZZ_d5Bbde5Ws5c(BH zr&5okxtr24YWXt8WnpSDH&_}c6T~45S#PEa@yEY^-xA{MWeLDlm9wFWlmxci2lWcC z9d)s$M!ao}1#we|9~IJn-B6$0h=s^q?(5FM^1HX8`8q?r3n8e;I%TN-yy~?uqJTRR zeb|E@|Ao8AWtv=il0QjDRo8(+AvDVFE*Rl%icBy6u@_04Q&zjX>q7?fxxyj1g>z#{ z{i$tF`TJ*QG*Ayuz6qLmv-V)67u77rR=;V|Z2-B>YZRJ#U~&`1XBO&p;iK3UL0?y` zu+hFfbMIGN)->&mE2Y5EO+3H;_Tg6wy8F-90yXfW674#XhJj(Xf>oxS^0VRT0Zv3) zc^Xr9U5cBH6vY{N?I|B7?>-it&doTpYEy|%y5M{kc|w0HTtPu>WK*?d$x5c@`RX

    3Z^^6xW$J!vpp`?(rmAh+(LCH18S!T`hlE>(N7>J*nIm0Z z7(zu#4DWwns)5?>zM(sK_%^lT)S3q3cvawXc@8uegnFk-P$&)gcxv5NlkDV?`yU(%$_XxluIg~9aFy%^n@wFjVXO_Ajn zA6^*t>-tY9+~h;Tlxx^NH`6VSr(BD_=6^}Zp4&pKw|?fO)mziYpDB9B;njWXW%g%p zUe?DuK0~rm+(f=KpVaONHSGCBii+}k<0~m@L|)2G9d*82S`(0Djiju#HWErk5SvAs z>tFB7P(Ra4Ftl)>kM>XgG{Z3w|ALm>BF8zZv)4L#DQhvp@~GNLGAFFVQPTWL+wpzv z^T8zTyfF6MSh@?HKb<-+ALk%EHKHj*u6D8LR30C5C0%t^*Pa&5)zq88>|}jyQGto9 zK$QC!iT{GHTv7}t>x z-zz%9OAJrAJJ(&~C+qKR_y;9kfp`vjuouQwmMHIz`hj+so~*SjXdP7?9?H>Eo%=E~ z6jfL|{J7K=w%eeoNx)L^;^39iH4=-+kHFUet3cnH+H^_SR z1mwjgvrqm0;ic-vH~Wu2Rv4Vu?)#mJbSX^+I*RG=f*$ULr5&iAaHoM7MYO{tt*+rI z&fElaGsOgD@PRKOknAtaLU0;W8nyxx>}yG=eE5a$1D!;8@dvX#g-i5xg4g>y=B>VU zN`CY(n0@n3Jg(K~L%KUY(?L1wo7BUcPx~QjeKH39&oV%!AGz$Th5fIkI~sJ<-4=Ge zt9>pBlAJ5Gs(fpeF8QKy36@=+#?der^TdC6iA9J{@z|aEOC2EB|#nlTqPIGLz0P zOI-C1qlH^+3@O%#IF1kFWlgg;ciYC!J7*p@VfFF@vGZwJo7CYi9BM zyQHAoSH&zFUSowT4EWAj`7#P9D!#H(UMfAToU+uTWQxNq@7`laKqcVn=-brT9}2HB zE4>6iZrXf^1tvUn6Ys2Lj%|&;;A}$Cc3jOXRTQH6Oles1(upgtf>#_C<#O9Lwm*KI z6~)>oIZ=p*R8x| z#U*J5^ZXY3ln{Q=A#y*`;Afk}?UhV8^9Vg?W%{60=1q10xOXo;ntyq| zWwf09EC`{$DB0w!b8qUXt~(5FI2j}Rv&Z5%qTHsA7L`HU(YIuqn6S-%n$# zbHbW=S4zJ3&PVP0#rL~|4z?lxQ0U`C$1mJT@#>Kz5b?j85Pxp~+_U>P~Q*B4H!+&~=Z zWE?@&QnUld2SdiLm*~@G-zPiHX;uL#I;=&@fNW_Lo*%UtJj#k{$Hb)NY;X}({;Vnp zzqjr4c;pmyffi3lPOf-OmwA#*wcnrBud~l|rTPVssjG)k{Rcs!F?VN<5NU^s%ILnBvD+IwJj@JJ-X>}Bl@QH`$rMV>Oxty=nHIaTv3_ilN=MwIT)yWd=Z zS&Y3h^fc%ZD=$Usyxp0f7O9Wu94J}WkCbX>|H;nb*OJH(BKp>*J=&<6>Z|8b3MQ+a zb?+ySaYa_LL_+^cH9eca?15PBtBvdP*LvCTKb0SU`q@d5N6upGZ!~uQrde1}b&8tA z7%zaaHFq|P-oZU(4$$YB@Xi^YUem}6uw6X~khcELiy(V3_ZiCm*M&dy?~6~+-Ao7Zd; zL>x)l&ke_evTc(kMH9(^UC`d8#O;XjMWtoAE&nb?N)NG zS7nG9E;WYu2)X8jFu3&Iym;`VRVYji1;s4!zmq-W^)0EY>|b3ZygSURZ{5$PZ98J# zK}UylHdKYkur@@#KUcXodt+{D(=23ebrq!2vDA>M_3iMS{kZg*+aJ(^z}l*P(Lxgy zdHZ+zueZG>_v!81Ht8KF_F11p&V8)&Ygpb(Vlr1pCk|iU{1TI}XWT~F>BFUjqFK^# zd_nmh->t3S&0NiY9QNgzS8MOb?P4Jr3$my~Vr?AMZS+vFN)>XqQ%$f;I5ih1XDX)t z>@^=VM8+n#gM5}+`>@b4b0PYPW9zq(XHZPWyxdcWgUKX+n{Rg?WcBMB%cB7DmFJzi z)iP6u^fg}gH6$7tJAVM~t z`i?U5x5@f1b)-c7|G+l?kFjF^+pg4q^{M`c?BoAZ^tVsv|97EcEfcmpT_i;&O!n`) zbN`MW{MS&if9Z$+Q1tH+h5spJ?)a*paVSfPO%;%N+K1OWJi3RvTt~G7322{*2YIw zhQ-YK>E=h=ZO9CBrW$)I|Lm=Qz`WhmSB~6wOFxuxBr^Z<#S;{yh|lO(3L9)gq+M z_$xuWyR+|OQbwPD?S}vD_U|tEX5;l&BsUTKHt6asDpkbcx%ai>fwaf%eVQ)=jsY!q z?e2)*BZYFWmG(IyNC}e4fd&$7GcDLa3w9qeZC(bq*n&5Z*bM|%S9*p_M)kX)MLJn% zn%rwkk8ATT2ZO7|Zd)wyDt(E13cj$gif*n92(qJk?geyn(WM_nlW zSv5LpEXlrge#ft7E?pwGaal|+uW$76M~g$J+SS9GFUOog7Bj4@2SyY0$F(!&fG zhc}O@50K!&`SsI}1+=l)9uqdcZnJR8yK(CQ6Xzhh8`cs3WNP9a>k6g$HzzKUE{S=* zk83^;WQwAe%#;UKStzKJF%(qTiKOWUoo!>QH%Y(VGaVO`SH6jurRKly!?392e){9j z`BN2c-J5+J9RBZ>iCMQqEQPn~-e(rx7~S75_I~w-d;o>ITE-b_=v1EEy_$*81JDF*5Mo94hQ*e7upgf}yUhZP)zRNS9egSk-=c@Ymh%~Zgu~Rr zw~EF9+E+t1oqiWVE!uL}kmI*;Fu3oLl|{q55$Zj;Zc1U`mb`zc@l zCP%L+5&1kh`#C(*O?cRHGj>D|mM;2NVD*2M5Pv3URHH~R&I6KF|F601{oP%CTtR>T z{#zjI*_lFCl-Q9>i$PcYAueKpywm}Si;elU#(YD*aRMF**|rv?x26rk`Ia|ckPUV^ z=a6;n%Xv#pUh48M7IlcLB|qgj*1y-pJ+41ZwbC#gcUKSwy+CaU1)PtR<(6m*3V%0P zL4a#R;)OMt+*#Xt!Ul<13`_(32#KN;DM=;TYPA~a9K6OQFQX!6CWl#1;JfkK!QnLn!~82C+|;)l*57z_DNfO zm57(&?&*)5+Hp>b468wl=(3j{Jt%j_6^SiAfJM@Kr}~BcIl54)7vVsK)}~AkrBl8m z6O(3&rnID%uGS~DXpG_S1=yzJ*kpV(TZ z=N}PXC20Z_UODTql|PZ-aRHXTPvYUHspeTXerb3k@Iwxrb{+ zF0ffMr~~Kl^>7lo!BlBr`}u*<_@Pk!(A;@y4L@R3 z^?&_}BiHJ(mhdO}4Py-}l{!JyA$ci2f|9ja)~l#D*zxy8mh0QE{aRKcG_7&#_a{qt@SWDc#HU6XML75D?k9uS?3I-YFGp+o)ZgEK*i1!Jr&%h=dsijqN!L#g!IDqi-a`JtHquhRHOlp|_MnmLwLaF*AIFf(6wOYVmz1VacM@9}F2}-*XdB9Q z?XL^4fwKx7d8gW%>+K}ITj&g3rA?iTHy$RVI@!HX8GcC;oi|`faegVtb9wp6n9LWu z=`2VS{g+giY+V(pb~T+tqqe6RqcQY9YJ~;1tR=e4eiEv4Gm%%V#+(US7CHxFPhA^0 z-I?%D;Zkzbf0mEm-)D)F%+#GdpmV=swEx0AGR!VF_U+3g*F*3t?Q_(R0=V-uuSvwt zi&*{pi)og^mzUz29+w`lKRWV0XcD&5{czXP;K3XFM!j#Z8cOOScS9j3_bEzzs|~EF z=zl&YbR5`GrQBKlmU;NO>e}#{m*4uqPpJ3xD05$aB?e>Pi#tP!j&pU&qp2%JaerD#vEzMiXE*tTRNPY(7&};@v{Ek)o zMt$Fb1?oRJbk+N6b^YQ9`Sv9bYQ0I7rQ3W)d6|kn^J*T5@A*fRL7kK+wvqYt!eP{= zLf}+E$6mpMZhaRw&OmRU(d;~4jj-!F&h7=k$H^|AHfw)ziED;lPF;)aB1%|HQ?Nn7 ztlah!ksrzRBgUtsS5N<8MwtIbY5g54`cI|xPo?!wrS(sx^-rbsPo?$$1*P>j3inT? z^-rbsPo?!wrS(sx^-rbs|Dw_&ZO8ljxCR(+dwT@9MYE6JchX3e-3%-M?wb! zd>;8adw9!wK6Q1wSiD#VFyGTP&;^i@kpUi(egGFM0PVl-{*R8oD)~RE;IDta=mW4^ z2K12u$jJl&msrTiS;#Ji{z_s_M)@ywlL-6!AiG3PK}kh@nTD2*^gt^!;1U@*`6UW+ zN=gb+R4`dM>2m-D3nl9{`P)=%X3o@t&)Ba=zAm{ebf>YO!+Z)WeB;scmo&7TT--do zBBEmA5|RpvO3Es#YT9>oboKNN?pat`S=-p!*}FV;ed6Zs;pyie5Ev935*ig96B`$w zkeHs4nU$TB`zEintQ=lZ`L3$Eskx=Kt^IvR=fL35@W|+gk7Ls_vvczci%ZLB%*MCx zKQ_0vcW}QBe;*y6;QyTdMb}?+{_F7%>0%+#b%}z4oPzom2$XNhd03uPm{-x{%z#Fv8yZrOV8W1m{w|@Ym=VH_5IIkQs#fxC<*GAcs6N0>R zE&y0_b*XRd9ae02+Oq?5y!hU|Z+rO=Cj!HJr!7knU#p+NE&%9}3&2Y|*za+1@`FZlNu22Vlxc|KP%e}3VnokqH2rV4-KnDy{ywPLtg;8Vdy3;_xjVj2m2R* z3qTwnk;e4`Fe6_O6~o~+WdTOKJQg{)Ts#T`*b}7ik?WuRhIxx4q1vSc^KR?~ zV9?OmWoZ6uZaeK$+XS7lV0RZui|t7M$*ZEa^$+x+0E1CO2tJa9ItwQhJ5Loc8@7R~P|MRyecC9zC$mgs$+ z86h9h=OjGaeWMPlM(WB^SifPff*~3tuyx9QDz#h8UxN1v3qZs8bi!O}2&UN!=u3EW z4}z;J+St$zUsZ?CAIWopf@h%~nEXxbensm>8W~o-%hV6K0_(qp zgg_{R5AWhYDXri*8!#0?=geS7ix1)ti`S&ad*BYO@BwA^P$9ewhF^OG%uuVz{WbVL z{>H$90N+Awx0D;=0Uk17qZX9beWPL!ZJIAgdO;4X|8BQWDL(OR6xo#M9?A}DNhO6@1sm%^);v2*L(R?Ug22<52rr6cN1VMSDd4HFqStB9J4PZGEKO zV`JQuX(f0BM33h}!(ugD*G-eb99~*2U;@80&i8Prm`%$S72C_aK#<31Q4+uW3U=v> zV8x9RSd*;qY*^G#JB*&l6TFG(UgMwJL%KtfwAcv_Xjr_Ba)u|2=0L*`7i<*4Owew@ z*&O@S!4Y^($mJtUmTTS*;}I+Q+)&1X-7rb>vU{~gBi)T`2nk6JNkjSJZ@ESfgg}pr z=$N#r)m|a&;nbgFjtwaz2djqHtHbOGi8b3}KVB z-#AF%)f%eh(#7f`FcZ;EKoUPD_wsL{IQ;7=pjwR!{(SD@uEI4D=0r&?PQ2Et+& zT#DKGc&=?o8k7~%d#)c%FnILdm_oXzcv)k!>#Gb)x7UPcU~b6e=O;Ry3lFDdTK_;b zJ5$>G-^f@D$#2<++4)e}vDxrCrZ&TlK}}osvu!}yV9-W8f$KJMy>LwuuZ(jzroy8) zm{za^xvr_-pk?9UaUu;k1e-L{t2yrrQIwj%n`&}b?mMF=7`Fu2ruKer;3I^mpwt*= z%sK7HZs%~cDT5bp^4gWQ75t4e6Ig#x3owSJjPQii?B^ubs2uVUTGvWb;F$9-PC#Le zKLkwh6DQMmc?*yX(2x+EkkNSqfn(h?wuq{2(sysK+XcO=wPbPf-{J;{Ul*?j!lV8K;YAkqnoFHgINg1IMOhz@;T(E*N%S z{YzoBWBfM>LK?vrpML?Mjt(9jEs~ojG7=|3VFM5tdNkWxqi8)8Z?ACOa z)xMaCC63)|-YiBkb7!VViVVy}w=ifuu-+_oN%b``fbKyk$_3y~38!F1+$GtJkcRcON=&6`~4HU!3tr~78j z*R3_hsa6QH`zbOCYJo=X!O!jxlxIP|w~)T3i4h8TAHVer0DrKJ5!PnN^sDwdMLdX(g-dhv98p3>I=#x^;@{PjiEEpBSFm0ChvY+{E}LzP~rs zZ#8O4qs??DYXq)eEy*E9eFtuz38xujGq07r@eXdzU;s@wV!xGI_A?2)8d$^_mLV40OUVsHTSX0Y`X_SF_L|K5n9ULFnew>5iCQ{_8>-(l!w^fNrV|h$xBeU$1&lsfBQ9Wcg+#=P#66Q6dskP(79E7kX zU~+J#%85hBhT#kDPpn|n(1tvItQbI?0Auf609J0SyN#qJtV!XKXUeFEWhZf5K>N@{ zOde=o29vg;Jb!daXfx9FD_M+-+mNl9VTOF+%ol>#JOIbYZ(}0a~S)s z=2cup8ICIL>?(0WQyv>zjPbEJ_B!t(h*SihtZVJt;q=bPhmcs)1H2f=K`KH083x4a z6%+x&n{I$9wa@tR#ilKI_=XpZZqXCTK%l8|3P$Lhm%-ZhKw2JBkoX7%N2=4^JY0Rh z>Cy$DuW9=NzAm0jRCS1cO1XTuuh%_5#ABqh&v2NUq}IXK#C|XwAfL!)D#;YG0?mYn%LCH+|DAQ|Mlu znCd4vd|Mff+1RhWV)9(e7+OV={B$Nj zs#qjW%b{z!mx`c0smQPU3pFrL>XdGE1m07fN#Mwd9DhWbf9$P0Uk2pXKuObz?*6&@vPxqI@1usHMF=wj~+%8 zQTLIv!|N{t{sQnCd>QGQ771h~IKPDoB^|4hnw=CZE3N>F>}8+T2;V?1k`rm1*FiCB zyp!Fk8~lEwYaG3^0flbIj@Xgn=cmGx%PYan16p+5LKqM&ly>1rr8O-wlsdBe(E^Eg z>tK!%)UbYq2pT8_z6%XZUSq{E)`nrgR69WKR(_@F&Gq^{Fk0U=f5UjP6DQc7(91y_ zy8ti}g|}f(9+J|q#C8Y0MM3p@nekB@S*T$c`_iIUTbC~T&M%OcIXkH6jF!+Uq%!Z8Hb=JPm@b7>spCf->IP;RUwUW z_UM@aK9QRsk7l9?b{G<%!-7-0W$2%}ukqp15{&C#^vZY^UBTKU1@wdt7-l0ET5_Q} zwZRah_bq9L3j{@JC&6h&@a$=oXtA^6ejQvftGqKMnJaMK=9j~rleWO+DEprCp4ypf zMQd~mjZI$fOI{0gc&S^zd9X5M;jKP6&?5QBV;;e4Q+1Es>rT$W{HuU6l4X)A7$;TW zcBfY2_Sk&{qO?QcDY7T`WtW@T{aUajQo{Af3Wd$foG zm{=|J+S%P;>LH8Z!(kNG5+wk2#T&e}$1`DKE7#RwOU3K8@Oe1e5sZ00ES#r21bd-H zXVQTIU#7b$xN&QWAo4ai$PC9Zdw&A5L6U|$_Mn7}`wi!^kVGUs>m=$Uj;n8SN!mv` zM_E96vKEu5^7~*vgg2;&C{LXQcQb$UWN~saDnMMH>>0|;USi!jwk$3d$Z?indB4hn zN4M6%yr8Ev#dD_b+>OOrQAFK(=#k|6Y5%R67&ZUW0u>BAp+3IQ`zB(9(Gne7Wx|5s zHH*rUG#fs*juA66@l6FX=;AV+Rlk(;d*Htox0v?nABjyZY$y-%rI^#K(Vg)D`S3wz z@25Tm@>Z0e@Dc)fV?JV!UM{;YzgElcI4`a|Fa2sqR~B}&wR@SB;D^6KyWNBw)&iU- z*%aHh%(OnnS*M7`JbULZ1vfx=fLCyCDLPG%Ku}`mt3;D(!M7`d(>Dx}-BUG3 zDn*lCZP|xg?Z!=p>SytfQ}qt~Asl$vkn-}+{4!57@G9PS)3|k>RlJTL8zLxk7;1C@3O6vswO-jl@sr3Tclu$ zG35)&N^RzA6kM$K4iB8xOd;OC6y!b!`ANIiYt$U?s-^dVu6dfV#i8e zbeYEfk{-_<80!d#o?+X#zz_wgQg4zgNrf0B^A3giprdg9dDh7;;Q_1C!3-?}BqolRY{E z780~>A^x15R}yT_Xz+T+e{h*vtVplom6>A={3b@?w}hf2A37)@LNM569khSdzj*1F zkPEDG7>$ZUu|Os2zCwkuLd`pAL4NC>U5s%(Ka&HvyKhabUlUNQ{80K8&JlvjwsesY zpF*gLOCCp8x_4M4WvO(&j%!1hG9paWX3`Wqm*ssGX3W#8H>h^aUuMGZ5ni8kxJuk9 zZ7Fazv4WeR;6+*_iTwy~Ed#rcRo)BH8Xi?b(f?Ec(uxfuqfzwsBgTH{lT^ZOoMb;C zs1|=X?15RSrwT?5_A+12Q7#+=3uYEB0-#0agGPcK7|q3(qnZ?Z*XLB<#4hHZ~*)uIUI!yLmmknF*t>)LPe z=_fX-mTSJ=lEW3e9xX#x`Gub>(c!d@_DEnm=b{<>>h70IZcsDj?Y z+iUK@tK+kuzWx(Tc(38gq0pydpQ?W#R(rWL z`q$4(tKu!6-mBw{v2#V7FlO^_DAwvx7wT5Pz*!TrL(W)>VadB85%n)~dOVjz#Ynlt z;sTz@7Aa2IrQN9gY_s!qoWX)CN9CJkp_L*3ojTQrJBBAwXywERmEbsg+!Y{~CeJpK z$y1+yTW-==WGPfkM^}A;d(32Vb)wZ;T{#O}&3#lg?HR^WcvMHqvLtG?EcNb8I@P!Lh+lt%Gxmg8H0! zlSUIn;MmZj3C=NWT!7xd98lrAh_oJOg28^nMXWf@sOsARa4#YcPAjhUrr~WIV^sCB z&2P^ z(;z4x+U+G3UU7Z(7D0V7IYNGR!92%20|QFqg)rzr-j2R??-jtx!fO`2oaAs~JCOZx ztRE(A6!d}_uf5)bd;uYf27@vN*Su@OJPS~vS9mi4T#bzK`^ z+F)`_5fW{MbBe*1Szyq)bXk>KRN>R{Q>*)iqoz43l!6Nzm^u2&&{lkl(xYNB-W zapDDF+ho=L<2*i5wNFQkV-tB6UD>P+;u%}Xn1j~R1+sYeo4?@iPF zzkj5u4yanVUjU&Q%gej}F2ls%jW^v&ydzAt*I-O@I!j^(ALdBv5W`^Ig+T#D2raCi zWV@}$^0uec_3}VPGWxtQ^@}rur#4Z)ktxsGV{0|lbn1pn2q5G_ohI|D-6$pnjJ9gw zYS&b4nSVS}1GPhZ-BzwVqrN^v!gn4aIiVH|7XShqh;mk~)nvGW}#n#Zj-8oK{9LkQd0NLU{e~pE@5dCr1(@Zbe#pq!7uM z@l?kfw$uZk>qWEP;RDC-b;J$j@ET`*Z#~Ya*anxNu-hoSNY1&C=`qQ#F{FH62>bJu zw8HwI6FOTfbonQmgG&r&T!GTqcRy5L`ZXfo#2Ghe*sX>(?Ywz z`Ym8vpDzFpnFw}n`y3a5>B2H1r!s*_)vt$+j*bz1sK(Xd&!30tZw?QWregExVAYeN5crdHkBZlM2ClE)d<%37ls1sN z01S+f#3CXr?$lv_W_rZ)`9(| zT%K#3?`LHR^3}n`Xz)NiZ7>(f_o5nv@W_)z_`O}*>10^_lJ=}>>(rz~saU2vr_(iN z+|Y}edZxup2f+RhPL4lkIIUyH^SXfgw9bj)`j^Gn`=&flAU+H;+L$w}Mb>SA)#}sY zg>un%iVc3l29g8<(-Dlz%igycyXv-(T%?W*XiGRw0&;p;pL&}wdyFZr6f+Ye%`C*L z?36{kkb!=&xJ<{dBAmZ7Jk;T`b_f&sbhsFYa7XyJ(gW^~<#3*`I)nFJ!(!sCVqD_N zA0>1M>KpY-XUFaHPjFTzgY_%+OtHMk_}U1LVXDdt04?hRhF!_Vi5efYalnY5J|VKC z2|%sp2rLyOh?gIq)GwffY{PQJ5XgPXwn(jI8<-GiuMI}n;=VaOt0ayUw=Yd=$d~T| zv6f)A<#F^1gQ$<5Bi3dw4NGpk01d}HX?;&Z^9D{vuS0c#oGrl`pxT{728Y22cDzq1 zL8$}#Andn_hC$7i*>-0{{;7oyBg^x8G44Z*yaLXzFz2RPCYoE)){bB+&7pJIkR|8g zDm{|Jnz=S%(5jaEGp+gkN=gZF^FdSEvLKI{LQ}7Oulhd;BX>;9%5opwtkYuB4UqdS zudroVW#UAQjY9{KIEgGcXLxxuji|f>=JHlNpdtujk8`MM(QmrdtL^bQB-4Ffp_rd= zYhG|h!zuv#O9p~zD}z<+EN3TSyU;lUAnGl*yDwD;T8JIxoX(-5KRO)c-!>(T-&IPe zDsqpom>9Q}T^1i;>6(^HcHwYkPuKql}q)ee*AP4lEXu^`bE zZ;}#CfBApKT>iIbrLYJhEzOxn#3G6s=7Hpes+TRyOFnOs-e%Tb4;p4t0y@kH|i^a?u728H~u8XKvj|^@uFM`Oh{q1pE=P zGs$wF8DmXG(~KD0Aprp{5{i`L)^$qB*JLAXXr~c58xrSE(AL>412vDaE2x=Pjz5vd zz!foKdcfIL7zVfq`|URMp9^L)BZ{ZMxj^UhCfK$ZxQUv0?L3a8NyGnZ3E;0MzZ~pq z|K*p{iJ*iUW0eOdV!^FBdmqp4?5xS27x&(Ty2lL`v7G1n`;ZUWbf)zer@$I32e}E? z%05;T7nJSUz`GHA=am|c8}4p%N8tjiHfU()-ea* zgwP%=Bb?+C+|d+m!$L-pcv0#kqJY<$P~1IS12%bCHm*Ea>bM2pj@j26I)$er6T6j5 zwu_i&>l7I-h4Jm1wMZ$?ffSx={yr}VD3-1qYk!t3X_=L?oDRR&>c$Zg#>=tmuyh}x z!g_MG+ED+VXI=*0awXb4<4j`)VN&p*25t^ld*7sEd>f^e?IA7DW7GD`$DwCRge zy?wHhoi!@tv|^GL-zvcycZ-E!WEe+nI3q!1jqOl)c2&8DeM;?74HoRtrYt}1 z#XgljWOn>Jvye-V+R@~Gp`scI{Hm)F(U*K3RmGbI8qs}_Xp;Qy*MjAD^YlWiq%y0i zs1yoqbo#;PF#S36$_WlJ;DV&oA^DzGo9Vgm?(r>> zaiZ050oa--`>#dS!slA0NIA{Hv&SUUzy$f2iJ*PPL`*)#`yH2@S8A=7A}{rd;`z{< zgP`T{WS*R6!7&!x)yWx8;L7`z(} z@>!DcpX;sst)l}|CSE=#=@Fl_1TA`~D~|d${wQpn-*AAmXoMR3tfSIkSR*Dbg2qC( z9%I&qGn%Kxkx8HA%JjfNhL_V4jtoU$9KjIn8I5aI4?KA(j+K|o+^r$iPVRq{VC}&D zqaa)v2G#G3n;67wqUY*|KYQuo-@?{OW-7;i1Z{A>Hs0m!g9jeGZUku`myMzf=Nf`G zz7V5sUPO*X_N$ww_VUa@IjY>3Kf#!V@u{aLBsU1-cQA`76OBoI%lA|!o;1%w+PNRt zj)>fy*Oj`324l8%5;L{B@&Rb;LQj+8R#GmX=@=f22)D(z?#AE;TnCcVB^-$(UCt6; ztTs&)=KMD#14K6r2K;yY-dR>k!0yf3d;<-jSc%IpmR=r$20n@u&qK+4p4&|W+bAZ- zTMVD9UBUne%QLy!_yH zNafO(Gzc*0Kg*p3$50Nrxm{<3_(+>T?L7`lPYVQxdFFQ$PeqS={?$77WJrjonxA#1 zhK4*0>N!rDaIp;HfJet+)x=YfSbT*sr9*BtK*8fN-gKU=vB-UuitNzwc3h9MB2{V8lzAT@vkM7LQek@Ss*>F7&NYd78*YvxB4 z%k!Qzrb6l7%21c6E14Xz_BL{}AnD5DB&#|wKKtk(V}%JU%c#&cUM$e-1PR0)?x~f+ z)86}-?{z$1_3P5?nb(dL55L^AV80-i``%VBmEU{di3l*qOjQiv0GnlC z8;8#xVWTVI4I=Orw_19$v?R5#D%0N6%~Z$%~`x5MWUK6)=krYmYrf$`i@GDt!5U4$`bq zJ6aJ`_5P%Z@WqlYH*$EhEZxF|sZXMHU5nN}f~ztZ5D;dHk|#&Wjig1^@xX==sf_hq z7%3FXAeOuj1#UEyKQ<;8eLN?mVwi5w21NW!ZlvArf-hlXejR1$+s;UdHU5h+1KkFw ztFV|32hXN~`NepZBR{P=DiisGvgRARf>fnuzufTYeaw~9)_E-7ot^VSJg-vX^Mk~h z7AGafziHo>R`1A^u~nzfVd{=ckxU*NZ_`gt`wq?Ydt+8U1O!sT6y@sIQsm)z7#dU= zM!z1}_ktY|KNWh4a1z*5t_zj)A8~NKj4l?+^BU`4=)BFSYh?4plnBiXT~&b5o60&= zMGPyw!-@{Wg2%FJmL{4a*|DSzqo82G(WZriM-tPH_;`&KlflFru`DRzC@w&-Vm_)} zUr}ix5+)R_n!P@kB$K?67Klp^-0IkR9E#k%1ROD^iarh~j8>%?^n8h&w^~D`XGgFH zXUy_KM1xBk-p1Of7pi-~`}9fmiWO}avu@`^Bh05RKEB|7lv?^6y<8&{eMs6$|3aZ& zZyv9@V-Z)`CEFC)d(A;BgIN$kos8pWkZyMGgLqEUG;16QcV6ibr`N9^qa3Ioxd&bd z1%-gnIl`cIyan*PCE)p%a*gfm7%`T&4cISOqg-6|RXHDx0PUfUooNHbBj8yFROk3` zW3?i1UZ(*#2WUO-Z${3xmI$^56QgU#w1Lncnxj5yVa~M)eDs+zYITVaHPJ{8>1^_Z z6Tr(B+l}A{F&yY0K(Rbsy{<(Ka&DzQjP4^d6`_=Ira57KY$;C`M;1N3x0=K5cT%vJWie#X9u|=YfW`ABfDeV#{?N|D&SmoN7s_ElnuzF zRu&@Q23flWI2PbErcG`(<;R4M2K*<{3_5L8Ae9Wo+#02eFmei;tB4Nx#Ei78%v1Cq z)iWdRpR{Uu;1{2-=3r9*E0tHXlB^w_^T2PvXcNe+=&^5LG(`N3+c%d=F_e z!(x@+7D%)^wNOu1r4h?(duFzcnW*Op*NHhEg^9C=;00%`O0;-)XgL|kyqlhJbgo}9%sh&i=- z2}47Sy~~awT*ssR0tBM5!7GY;`_p=-+=58$26DMZaJg_e+zvQ#;ZeED4{uNaj^PmW zDKh!>fN7rDp)|&PfU^hMa0b6OPZdLv`KwC=YftK!lT6@a`Z>UMKmvJ=9heq4CQ+|e ziJWUG8rSP{lCViHf8foK8T_*@j0Z1$|NH-i3<%dh{B!<)|Hr-r_9d_{fqe<=OJH9D z`x4lfz`g|bC9p4neF^MKU|#}%k-&Kk!UME;M0h~044^hZY`*WkU#XWxfKT?wI=q#9 z`S;wzd!?b~3sLPhJu|5{O&oYFiD;06LuYoFX|nm&2-Mv2gH*ZRk5tI_Mykm2_OelmaGEZ*B23v@k2EvMR}0(e;p@a4*duMVwHF7^Q^gLH$eii^gkvCRe4tR#LzVBv<{&ydjn?LF`C4?JB(E%|Dq|XB>e{ zt61=rc_nKwabJSKVAzoy)#OaUJ5NrGdR^0^T{+1W zRv)l%*(1QzslVMzALjUcz|vPH&8k#dXdEhZ$RUX5M9pm?_Xm3J?1cjmqpUOJqLzy- zcPSp4`<$RzNfn%(U=p=5bRkT>p)f4hKVgxPb5Fgws3sX!9yISxxl~(mAl4-OXy5xR z>RU$5i{g4F7bn8}5XD8Ebk|O2wjN3)G^f5o^ikqR`3nhtn8kAXT|}#tYkYSo9%U1$ zb9B{0sEUt(qeQra)yJilsTu7zk=?P{Z@x(~DOJAD&WwLQStolcqr?*^G! znNJ#C=34jRw@epA6x-^@oqteL@{}M2B8IQzt;gG4MR!G==AdO=8mwNRhTz~3wH#%R zfWC%ZfP2P^VRqWtNK7si^=`zF3al&v8ZYS2Tj={rM zQfKmLkKY>VSQ!y(x4peCOD9nIv#_Ct`@n6zMHRU<#n0oNyW;^iwkv=qL{#g&yU1nbe53udp2YtBW?te#|GA z9}dO5Q{$G8T1jd8^eUGo8N%%kCXb5Bm~s*!vp|y;8C%szw;$EuG!-sfc2Mqq{ta)m z@u#uT*Z~A8nA5VK>c?rz&?~|&b^V{M)_-6ExqWmS70zGe$>E*VKHf{`)9%qxn(!*z zXOzVLO#|$V%cH?+$S~!q!uMOOND;<6%7H4C&Inn^1m~)G$=TS#V^Ze1D1YM`wW!Nu z60u$OY=@Vke0oU4Y`UKwh>%bRtJ*U;9MLWMSZ7y0;&#oPc8#bm5yf_!G7VNLd4j}S z?yP;BsNaJ5)Jf~f#jx+i#LkE=!?Trr2`WKVfs+heF=t@4Xe zWh$yE(lhX|fZM}z&es(_U|H^?7eD-Don=-De%{o9y3Xjws7E6Cb&r4e}j&py|MsFwI%az$Xlwvi;n) zLR9$qP~iu*hl+7*3*<(x=#1o}biOr^^;zE#arD-AUD2duaq-}>zj0@DF!fY-K<1kz zvLqH5>Y`;cT`GFvOQ{!c+F(>RRFgNQ{D94KwTC7;bDxwIC#EB^3rrWtI+ryRYhT!X zx=1dPS)nF+Z^WwkZeyL&&HSrDtD(FLQA$&DK8F3l2Zlz6p5oO~hw=GeWPgaNM68(B z96ldQp{gKhdEcbn&LJwj6R|2IEPKH?kq@1M-!P{X@g4t@=UiB|N7)={yX0e+pgiXT zafXK0m-D_J0jWK#hP`(}*{F7146HN5Qt?ovl{B}`M`KPXsIcI(T|BN7aPnq4t60#x zvo}fTh~JhR-1D=e;CE)pDh8rXOWJ-5RINq8V}C8E>u- zU*dQKOG<^BFA1IQD{*+B`!d_}q=EWZEEv+gl6V^=-D%7R))yf43m#AgcXPqL?gDk) zOx3edu{$DKj-pY7Ov#hu8x=H%myASjO(5hxJ_S{lBNMMPn;gbb8*%nlVFvX1#KhBD zZ|))`dqPPUlei0b58fIdXYx{H)|cazwwbLwOcLpX*H&aAG;N?lKy*$Px}eGyeusbP zC2y7iZ?h#qaSh24k@?$7raAImxi`A1d|t>^^0pOzA4ta_C4woNu8_|L)S!jot@3)TDxG+bv*TBqsNy37FL8$^Y41NvF!v^ZPJ+KQhPK_e8pZFKS2A@ux z9lQfMkdRNK%xUe}=6s8@^@%+VL492(ssD8{`*}>1RcE4u;Q6MAFX~^VAKYdTRnu#L z#*YuT|KwN8xBc_RxoD7~7cyhM_-qB=%JJpKtXRkXOA%3;D^-CBB!ewIL^ zFP#DE8^Uux|DL zWx2d@G?)>hhS)k@_O7M0AC6N{gBO-M^&jXCXcQ+V1K&A+%(UR-Bc$$;pp5n|5hCxK zr|>@~FZS83fWA}iHD~>6kno_N!$I4~%I`Ui&lh)MO36BddVC1F{5?jv9ju<7XfS(Cf^dBVpZe*^p-f_; zXzu_ErC#SDXPb&EC5H}sJsHlDch>fDeN204=54lh(siN?SJ%@a4rd=#E8H`joLjWi z`RGtXmT8Rk?vCg1pI{ChizVs_O>L0Mzr~tf@gk?~ONfms;pH0=EzTEv-`C`=@d@}T zm4Sgv|2ho+q2~i|@9B3o(2E`(8iUiTe+uyf8j`7euEM$%3i7Buu zOBpvI5oAZSG50h^r#C^IyQ_@?VQ$7rGfa0AFV1=k72H_GPk@MJPb|KZFOYHqA8jyAS0ieP$2^b9S}` z>j_o9m0VOO_!=K0FnQ%>afJR&Yn9tcnafpR_$i{g=}Vak1O5TCvbL|dnM2-6*Y&t? zzj&DSEV4;t#PsCgYvP*l*So zd~cXT>jSLC@)9?K{pJK>A}wL>3PFmGxzDTs3Tgo4G^4`Xp~L3aukWkyw%KN0)N~pY zA$v~Q52x}Opa_kXUY38i4^K?NoMUr+j$d3cW ziAtJiDIq-2(;-wo1&a|a$j?qN7nvj=)lCY)!6?yhAyJ@y?N; zW~hGaB)wM@lspn48YHN~N6&cd71`=Xy?CL~-gq!miD;T8SDFhf>Iz-%T`-MhsIzm( zx_bWQ4@<$(&F^%ZZqW;=>rN6O-p5S{A6@@?>$GT!HY}BOefIggtO$;f{z)-uH@#~= zO=j$8Qd#X&p4^>iIuk>9$8y@Qb_R0vr$-V#A5zrI`b0I9U41+Q z5F)POV}&;vKQEJ_iiNn&68IFx6L^Pp4FjdJ?DZkuw6v2ahU2?lge@G2mmoRg>w!G^ z(&MLb9^oOskZ{7dcL#5;&^y3WOXsN{>ABx$zV@~Ya-m970}>qB%5dgX>&ssWTL(;PXb*xWjo$iyX8fAN*{IZ7X7P;LB?6?aqF<)Q?9+M%b4*s&42fomdy zuksT-DDykLn5~6dEN<{1+$2-q_6_-e>SLM1s!?s1NWmu%1j?cO3TWuu8|sq)YJpBs zY%OWqIU4^$8n{PHF3478e333O;0NMedjN?XS z-|a>W+*fXdX|^yM_m04T%-w9995;ruC+yyZu1B80c$9(KQX7?HwwoU?#0?lO26u3- zF&59zz&(U@Hy$3{c6Z4t>KPnn&>N&bR z;O_b-;6wn;+R+yI_aJY&hOIxdWy`p_)+|F%Wd`u*f8%`;{y%skd%1E$3_(;j;Elb3 z=TX3>b~In{zw8P8!H(w5j$G9{1$H8 zG=D7`+n@Zq`QGhk=N0z2%pIEa&)jUva!>V#Si)oO01A2nn2y~7MJw(=|5DP;3E!cf zxB`8%(AFU{PcKbsbci(7HjDu)B}lkwa9XsZ7mx0%o%7necj zTTc$)%6`+H!`gqxZCVU2j;{4{=@ft?qsATVCEdT{HrEhb93)j#A`5WxrNExWZrQ;N z{*K#R>~L|oCDgaI(ZcxexJ?I-i^DBIy^V7-{X1^ctK;JS?82C??ScDEr8nnyPmQj? zDLuM-LoH*39h^ z`Jd+wcQbICo>RX?-=PA(Zw=yJEN;wq8%No)h1(SY#_f(9S>5hF`A_#!xIw<{ z?#VCznX%m=KU~1>K-zY9JUp*gJ0|Q`SPi%P?ij|-?h0>qbpI!!abDvf;L-5#1c9%& Mz~hQT=f=DL0rCAhZ2$lO diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testPSD.psd b/solr/contrib/morphlines-core/src/test-files/test-documents/testPSD.psd deleted file mode 100644 index 7cedbc21a7a8fc3ed3813ade1c6f302545148275..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 69410 zcmeEv30#v`y7x&!5)wcbt3ySRtxy@G#IPGhF(^VsP!TIdsc5RWP*fTzR>67;SgBO1 z7LCDf8U77yPs!Y z&UsGG8yFfrmGFr9p8y*LA^hiwC1LyQ-@w4oiIW5mO#5~u99tg#y0>I~03qv#6Ta!X z`ODmb1v}uYS=;!=b-tQX_RZQ`lCpF$4%k1Xc|%?#gkAr?f25(WCM|qzX-e9Xxm}N% zHoc&_Br&LicQL3gi&=}V{y%{l_>V#rzw~PQyrnL3v@HEs{AE$|-w$2eEc)-g|9^P! zZ!Kfm|8I@|f6Ie^YuUfG?Eig~`L~w+Tg(1MTGofylIO?(GKNeev&mwzhSSHjzJ&ZKRa^KpyaHcmmQwC{N1kLmJ5e z@*Y`3UL#Y8KXEiu!u;|Nl*DX(nx);PyhU?+4mt9WTCA8(vkH68kwhbTOf6%w=hM7` zS6@y1!_$X(*H?FvCTcl#|5}=zH)~n8kxJ&PX?pG!x?w|!mbM}<=I|+}ouryp;d>uF zjPINHv;q;qcU48=PSQf17JrmMvsR{+?5A|$>MPe)J$Q7mu&G5`ma%YQ#)+rx+@T%h zys0fCZ6o)nf$XP7+C)zGYEwwRt2#B1-$~l&z{Brs3r}5>`$i$9c^9wbU%GYt^xgXp zuWZ-08?+fGdo|lmbf#u08he1I364H~+Vle)b>PELrqtrQs_IvT9i*MUP`6^^nnkac zESXQI?mYQP_T_6uRW&zHRXlEPJ+#<})zROZ^5>t?Q|3%34oX+?Q{ zz-O^B1tgSLs?aI(~L1VB8_M5>z27#C$wL%OA&}}|8V86+9 z-iUh)_%wjBPSb}#9PQYr&=@1GwSfJdChQsOSqto#3C2KMu-}TOTXA=%2^wb-5QAx? z?bPCW)s+Yf0~uCZT_2G?GczikE?$?j^bhJCdnz)Hoc-$Sbr&yR&C@-EppI8OX>M&l zy<@Mo?A>LTOtdhKB~XMG6O1Ob*bEr3E1c`Vxn@9VNAHX?;nAA{c3&&bw?b7Iwyn5> z@xqLKGoEfnYi8^j=0?T~d^h0^1LJ?y(Y|ejUXn#jN{d-WlQiLPgJuh_sdQ<+0L;DG~%^=4BVK7h)_;ldI zn9c5CY;49ghBCW@ZOtgNefDX_v628<{pC zBP%;$>-OChiF*$gY`*x-^_nv6wd%TChb!+teB9JfdaTXRs4dC(winU{fFR%&;|@dI zh{oHoZN`=zcbK%P1;?Exz;{g783$?Cy^IJZefD95bU)cc2t%1!AvKY1CWe9~-NZCA zTe}i3X(ulV2E3?VNn>KdUJak0vOMYC_qPpbhDo6JkuP;2i6K} z%}k^D34@77v%Z^Y?6wfL?G`VC$y#Ksu~s<1F+(BX1e*0(bfILFVXdg z(N$G*E(E&E@8j;pMazSuV_r@SO_(|F?REK^litnWw6%0WdBx#RR$Z!ZxVQFNRc-x_ zc7x&6k(So>#=Lc#wxlna{tw;Kd9y=}}D$M)&0k(opVj<}J;@A)_dPKnZXrKA7YgWZ4uUNGW z>+7rEXaD)!g|AED%a0##)ITcN(P(I^D%xIN@$TEpszKXh6Fp2FNta3ewaBO`wEP@} z-DcQZaDNY#Oi{X(v)crJs+6|1Wy&!TlhmQINs8Z3wRZxs-eOz#fbpTxu*HPAi^68l zTa>DaP0!BTnEGE|oI9BRpV||fs~$Y6FRs3EZ8vAI)0LfJp`dZ|%Ipu)Q;;#;qK)*& zr=Yu0U^d@lgB?VVNZa0}jZE-N7#;9j%qw==mzm%W^2ih%3}}I!VOP8F_@(_G(VBfu zj}`2Mn z@Igkg?k`vKw|{n|O#iqkUsrSU_NmfFNd5@dP7T(lHtpGqq->+Mt?m4~59xiN%+O|T z#~et_j>?Se6OhlW!ff@-mKiI|ALgbQE1sBKFC&-T)sq>pU}`=_x`I2q18FyDO>{tV z$>R0JZ!F0zT$Gccty}v^%8spRN$CrHeO;v;L}2UeIo23WV9;LU=DoANV4CKwHA{|t z_W37iyY{cWarX^C7F4fRBFv`+OmJD~on3>@+RqvtEfa_)&SvqS@K+VR>o@dG%-L10MDDyEn%rs2RcgaqyfRV0oxWBx!IN7GF@eTP}Z?R za)(Y~Gx}vUwRQFD|GcKYu(F_*Zh!P(%bgq9-(EbMfAqjdNm-du$?*X`j_^53fxWX& zK)5nGI%x5#|9Ce>y*Od-p`&kS6ciS(|Nh49RfkTTKDYMhx7T52MLPYXODjM7y6Wc9 zElm*5`P`FSo48^1>hu-M)h65zn7YM306tesYiO}+eO=BMrIh}urTNIC2SvAU94@_h z=ES1+*2g3#N8$faSM;^~Y@IxPm2(#@i&IUVnqKtBol}rK&p&tJ^7j0_e>$@KzrHxD zyYx-gr5m?CS#|jM*}tqk^@ILN)eaqJkhG7>f6t&Hx9{9HFJ|$IwKJ#ay^WRk8))j1AyI|q^E)8g;q=ut{+WO;bIl};Ld;5{L z)|P?`=Tq`?6W?6AaOR4%rLQlB3Mjc`VaS94cLzZ$@ep{(V?rZmB!oZ$Z_ZfsZvN)* z@iDU&&PaH#Xh%i**=yg|*5|8?(c-B)+uN@!E!iW4KCIlL?Q9e6U8;sZOeyGseuqkG zZ`OZkFjN=}d$@L&vd7g~m4`F-e6%eoKW|;mI{>UoePdY+u$T}P;59(nO8SUx9X($R zh@2TZd)bOqWh57EGd}i>th^5w9zj-hIX}V}Z8^2{(UaPQx8Ez`@7P+z8SLSSb?Xpq z3~h$3>OH%U9s7^#h_I(<2fkbI*-RVoy%(Qmj4s@wo;BA#Y}{ZlRGl}p?|J;V;Ja_~ zbZ0+b`qB21;;m_$-pheu%q%M@o(9znj}4P~JK41mM{!?Q4-YSIBAEu{61L006 zTqUGp!VbB47k2xw$FQrXzBlevtUupo*n;?SrYZmay}a8uYYR#aA3As-@AxN&Qp-x; z*>mzl;_HhSh9oCXTbs2wFl?B!Lld!+*xLWjUFI7aIcf9OKknRnFvJ)wot$^&+V>Np zr_5fIZF`0J_4cyO-`3o2{IR^av9nE5Tza)($ChomyBEMbd*NozNGRE4;@m!5YXWXJ z>B!ofb;tL%>CPAorOj8bHa8vm@sU3Ni&H006jfcjvi8WwhswUcp7wtJdvB%W%q;qF zbK?8?c@v_-luCtjUqKUb5cG4D2ZhWWp@g3cf8*4jzf6LFc?<5i`~ZLan>I#9MTPl>DO~$GJRyAq zwtW?0NlW7bl#$W#+UwReH*Pnmx9_oDTw&gxhUTK3Tq`eoJJ-x_ZFv6{0s+fM zS%TA!{Wjo=@gsaQ`L^RjvCo;#nzw9ZsiW8Vho&IHTEWO1p6|dOYlj6J7G7A}?WWe4 z9BOG~-HrO2H*`C+>s!j8Cdb;_b*HMouPSJ5Y0kg=^`1ho{*fKjnM?9 zua7Kbm`wHph(?A@p0{yuR*ajowxXi8bW=l9!+W~cmX-}?nwmCObT-pty!i`%0QyXB zj7WwM`*1kdO|)6UOx|p~kx7|Jo9**)irv+H{vP%jb69IF;EP!=FbZP2i?zx0fo&gk zZQ0%3Pqbm5G2=UGUw5;%{>tWKjcvtkZH*Q8?>8b*8@}Pro!fg1hW4G{&CdEO`>%eR z0X4{~zW#0g**_o1*B+_Zv+BUH|4d4Mb7}1AKa_-aM%yIh3Honh=U;+r`bGizD)&sfk6hcTD!v+ZNB;9DA6d$acTp&f@RT3e4^`SHgh z_=>}q9^5Z$Y%}b6^yp!c{=w#(H|jRlR9`xN|6b|YZ!YKSzWK{n=d#cI`HR%uW!uuX z9zGIgjJ8qbm3_2hYfyCbxU}~R;Kki1q$eeX$P}KgOY)XSjrAZTK5BJd>6YEc8XDF# zHINX5ANDn?=#%$3Jf2+JcH&nj4NZHSD?j!{%#uZkM(+Hx<{_)~vtRaQkLm)|Jbr z3(kIiJpHq?=R&4Wk9&FA0qtj>l}F8)6IrtFfc>Q8g^QNNf3#y~qWTTZ;_QZ|Z0=a! zqUrfr2}@T;#VuI9F*f(=)zXdYvl>l^|D52NJ~J=JMg^I$G2)4>c|d|4Kg6|e3t|Fi zTF%C_ndo~9oszpa_fw%W{bxe|n$o(c8+EsI+jqRPyFvE_qQa4u=HutvS}&b%YlGyA z>mTZi9^Ajb1D>$5;m+-iO;3J&RJ$HJu(S62YuVSUPRA{OL!(*#m%n^{dFQ;$tR<&U zzj!2xnLz5P&%bylt>A+X-agX2{|h*}ea+3~4NXnQ-#IpM?dmzJ)@@JwIqG}70CR6l z@JP1d0n8jQpEDmb8W^qJw6U$(HU2dhIG91VXE1viX8Ei&raC`@6Bp?3Yi{1EufKh~ zV*id~$F>)@weCOD+FIGz-j3|9?CO*H;>SPMmm)sbK6r3{JJ&v_^;+4T+qz2#fgA4L zy_Wud!3X(8w{G9QvuEYDEt{{uSamWD3b1pfuDbg0s_hjA%hxv?W^CBl(7Fk>U-m`o zx-VKc?Y^4V!v&7;%#N;W!{p5H16BBD8Q0(Dh1o+gzA(zVJthkW-3G(BYmS@dExWfH z3ajFDb3;c(wKX+$H*ehF?zrB(v*Xy_w2CvWrB#Mg$WRWRX=#8k8!8&x%IY6KuHVqe zwL3KKxqtuY6&T5u_V%Oe_aCS@pf$Ez6qg+Rpi=R9-_~tERIs=0R8gbhRB=^RQDbZC z9wa>=tGQIm8H81pkAFOVo@q2&&z50Kr4a(U zim2X>^u2Lib#=|@3ZsF);npqQ!84q}t$gG0yZ5R#vKu#Cdibb)-4_*G8d^(>uC`vw zd*e{)xf?g@P66UQ$`pt3_#+ei?f7Qu!ki7m<`e2tTUS?q6P=Ozx*NCZZ{E?tb_&++ zczeC}eA}r_7``jzTKg0O&atD&Fplf>wHwcKZM>aT{Ps)7jE2ES%XS>oZcX2zty+Hs zEYz*vU0harEG_F$DO#+pt3BBZ3rfnmurL?r9s+{{l-63`sJ&5t3z-}8nL9e}(17yd z?Z?((NdMHj9aW7*y0)s~-P-eqH*VKzGc0yAR;|}^ZT+-6uGQbXb>~jymD{=_yLX?w zvajM=-3_ju)O~6K`GE=4A3^9XfZ0twVGq?c;5{^@?q>b%+FQEa+Pz2a==N2V9oezt zknUW?@jG|w>n-jauef!MtKeO!<8Ihj*WI{z1KhcDr|#xWCb;U-t}a{=wZjO;4Pd~m zjP2a2%&lH0gOfgWKrWkxo z_7fdxCJ)F}a)H#7MxK~_N2*CJIY=_dYVs!jG-L(ICtJyfWDD-7!vq7AW=RK+&l6!X zLOI#+3sVf@dgc#ciox=QJ%^?#25S=NdnG;Rx~3S!Z%(1b+ZHUC|JPFt5>DjN&70?E zZDgLD8G?WkzVIKIVqm}g=sKFeHf#SuN_l+D5*8pG71%%L==$HDU*K?b%fXnH`9)bJ zl=8(wkpQi7zzemC0nWes=jRtZSC_r>gY+dIuUtw6VzIzdC=>~W7%LE2iU<3R@*VX* zGqIpw*tFczw^!`R%Ax|PL?jmT#bO~N#7vCDvfr?v@l!$uI}ZHkCl)+cdvxaJ!bPR( zY&uXXagYi`5{cN76FLx)5ZsdxVn1MjP|#<<;6cy-mM0e2Vq(FJ4GT6Wtf4MeVu@73 zNv$N1kx(QS3ateKq+=Mw7Yc102MlofZOkgLyL4qn!uo{O)XPdBwh~IE#3%vB#o`VK zx>LX-0*(m#^mlOg3;y-z6Fjf{Br;)UI`y*dXT`U&wz8H=q*6<%1a%Qk$Rq4a01f4c z*x7&D>^X5$!hXBc32X&!iqtGdinW6^NU`Fi7FJTJgM<_Fh=>zYAx|Xa1lGjTX^3Ks zt-#8mzmwBQ@6@|94I&pkkp^EVZ`o%UKVwaDuxc|0i5cH|tH6E3Nx1s~%LVTx+F&|4Lw3IqpTRRx7 z2|iLQJF%6O)P^adQPM|*)CYG6iItlyNZyYlRyNNK@D7XBycRY@7Vz7iJ7E9j%6Ce) zrc9loj!!K5Zn(XrrLZ0OmW9L`GUbV-oz?@KAXJeU3Tg@S6A6%4fNoQ2Az{f`Kd0ch zw--CPI(v_uI5s5k_r5{OiDBVW|94Fs=$E{%Jbkq~W%ah%AE;}Qf=KP`EyZ93(b+X z-Dv~QFWdj-#ZTXf3koBaoPgGY~%H42^c=tc79;6xy1zvb2GFC9q#B z7C{A|l1TcyI6JWP%U;@V(2H_s<)j&}>(kz-`zF|6g~`z`(@J)9d%Yaq_d(>3>>V zhuqiJiqY4npPju8vsw#2%+`Ukey%T6n1vIlu~cFyl?nu)i<9tRzJzB9>G6oL6NMei z0Uoj;{qbu-dv~W7$Hl21uBMHeGQobruM$gqHmz5~;}A#>S(sLqGdImLfqLvE)lE1rk`5 z82W}34(UE}eNM`fP?1s0lL$B=1)@~&Jh6kM1jL3An?YXQ;mK*M(tQV6iuvT{W(e#Q z*VnE4W+$ZsFeWGHYb&%Ai6l}h=e~XW+1p#=zew0h_|OTXb$?=G!@3z#J0w!3{uztF zR^S{JO03}Hhy(0E$k^3KI5==@WUM2@Qfwcx+#)+;8A0Wj$z}&EM)}bW1IP&E z)KVrP#{W+Qym>V&TP@S7&Jzv`P&7=cLdY7Oqi0fpp*x+{0g<+TX&23^Hf& zFH8uqxRL1TWZy}o@D~Dy-~Fx&u!GTFY-M8)(iw30QN#~hPC5Vr7l0@$sQ`kPirB8e zRtyv*=rs`=qqRt2>Pl|mnZuC-bsW2FL7dY^s}+))31-wR?NWK+S&Ev>;O+JdP1yW ztX2}SwG?T(ofT&-7TZA4qyQ@B3rQcNH2@-3Fr0oT6}XO2j*otM?qp}u$1`YBQcOhN z#yu68*D)~j%jN&y)m(XHv2w;lxyt}s(O@Sq7fB?1gALcmO47&HR$^<*T%jXpFLZ$R ziTF=oOCo_t1kDp$GL`@zqlCw@QJ5Vku&`ll8Ym$){hVI(k@sZ@w$%Vx>^mRrKYHTJ z{AD`}($b)2z3S_1W&#_oG|x=vsr~oAai=snB~2X=9`u6mNC$xs9d}D;JZA&eTVn&s zTG{r4$iTOLwzg7{1p59Xu@MMGmJ$iN(k#!n172bYhex5Yh`{7|iGU5)2#B40*6VMs zeQ!&~mP@sz8#k}ru7=_D!t0mn{(VsQKbX34@$7kVp$X$yCmq&d<>COC{jA|tEiFZG ztOAKG+gRfR-NXbZDY~`53MK*tvSwZi-oXi@6)Y8Sl8v4aGjpzb!=)>0 zH)G&6E4SqRxBk9zzvSk%ou%bzZ%3_8$IM9~2ZxE?jnGzVWo3;)Bk*g0tqpA0Mr37U zgU}|jW$2kvBVZdenh(iZN)Y@QuQ`DOLL75nq)&eYY-RT8E9;6my?FIT45;3^mi@1< z^gEa;{ijy1h~KIlGT7D{swL(u(L;c{GD;@WYuBaX(a|H z1QL;uPep{^N(O|4El5qxDlRHXKXvN!{KGok`Oj7?DZg+rE%Mc0E@QJX?Vd6=8}I3< z&~;bm%;P}+d?9}TE9CFous-X+ju;5eck$eKFKlfiUz+H|ff50{mZhbYr6nim!@3L@ z5r&0JO)0jGu#X+aGpvl(HW(_FNX4*gHppRZw00MYMTmaLl%_0-5AhvDNJL2Dg7oEC zd4~?o`{Ij?m05>A{TovmSRt;51OM}dd^=MizvAfiSR~82Wkv7XA%aX=o4L}4BT{Iu z$y38G!osbMmKfKtL-{AV$PQLV%=jK+=)V7j7;n zSUE0je&meAknotq)UekNAI@I=#-i@3B`cnG6UfAk*%qcV|B51hKNRuLm6pzSC!UAD zyL15L_A5Agc(%!#A@f86ZW^%0_=k-V9@)my5+m4_7?))6hA?grgfvSr#w#W4gn8gn z`g}mq)y=yue08Yk`t^cOK0mcJeP?KTYHZ5NO$&dHY9`|Dg7v?)c>kQac)xj7Ug@Dj zsYuSfkKel!U|@o0@!8S!c@Zmj;5Vbzi0tt3=%wI12K8Bj4~~eW0*Me?2~N4fp<@pN z3=VHpm6v~W<&$+)jaj<$U*;Wuu=cn#y9rk#$mEv~JV2t9#b2{~+V^ zKV79C*i)t7ynOk}^o)Y}PLjU;VV3dh-s#H`2XW|(*Jg#Y&XMzg{(Z;`fpRNOXessz z3>rDwP9PAA26(; zYY_}wvc0Id=upb)isHY&O#fW(GX2(sgv43vHWs*YkVL=?wK{C^dyD(C@>E1(TAE@| zUt5O=^}BiRrL4SSD$^gna{I74BxL_KCjO+D*;rXO#e3dxs?E3?y13} zOuu4r&!MSIzxGv{zrE*NSDAix@iJOkvT$+oUzh1$J)4gT{-Uf+SP9^F{ps%p44}_H zKky$Y(>uL!BA;$vmwgyz`ay1fegk|JE`#Jgt^=QUbR6{GuG9}ay1gzC)Z`<-T4{|9qwXg}QZrDp|ekvn&mD51BY-*qG5iqeceFyhg}<20MGW zI=c*b-pgyS`~OVA{>AVaAM9Jd>Tq^89Wo_a7BayzY}{BsS%AN9<_4XDb0DZcE#qluc(O%?x`vX!^7me--#93mr3BCijJ)Ts-7HKEnq)IeWPc9`M_! z*$=;TB~e|dUPF@>1Q9j0v7NLFN|(dArI3{CpM8-af;J`;2(u`N4yK z{mQ*u`N`A;Nmw@`EoN~@@@sQLX2wm688$Uq=^r{~)JU(u(aMp5o-$uBT;?SU8LLzb zbN}s@?uQE80+(mgl+2kcrY%oh5;L{EzFojfTL$~SIo$S6N(ozDO{&&{yeFI`rlT$M@qtnxtE{S@5QP`~b8L|FTXT(m8R{2ky5T=y* zD&#}FVQvannKH!B*+b^_!f(EKk9qKp0ZJ}Y9-Fy(<%)##Fh(#xwqmJ6~FyN;coz-5kz zP0b8n_WGg)k?MsDKwWtJ%hQzsV@E5!VVPsSm|}V}_c&A;s`7Job@LiL*y*=j$hWO6 ze`#u>IwwBlmGGdc_O2jybRWmy*wlo~%(%CppVQb+2N#9Un)~YP0H|i*SeT=?+*cu! zDTC#%egVNUFE17gzgC@sBMFzx8pX%!ekE>9H`mx-@*%V1!hbkrpI(xaxMvjsX zbw`JG@Cz<(FZ}PS==V>}KeRe){?gpt>OU^24VgJy@SuhP885HR4={HL0;lXk;Ur$%BA%0^5Jr&^*qrHFK zn*M+lhgWSX%g)+S>~9K*eOGU1N}Xepwq=`gzpfUT!B2z$;o?!uYaJk-{>)8mCham-8@_e|GIVk zL31e$ztkoPiKXHfZWqnK3wkS>^@ZCKg|2r zEbhC`ed}~ucKX?`*Os0uJw=^@l#0;tpmRppyoE~>vr>a)ecQ>{!2uD8sR*#=ya6_`77j(*o+!GX4nwUhq(Vw z7WvI<0c`m`x4z_ZX)O)#_472L9uYM4YHq5D%6|&RCq7nHi&AFmF!O z@}SVOtA-5m zbRYcu&s6%|_wA{#%e#(Bf1r=t!`II{SSj}zA2E5_s#S4o)+FM8pl2Tg2^}yrBsL~( zY0BjJt5PH5<6^^*KMzBaDUX;M9VG|+aiL>|1O^8V35|*f8|&rg?<@Co_6{C7Dr8t_ z?Bs~?fu6n=$^f6|pMSR6KWGiB_NSoeICS`M4*^b>L|A<-DRGi|S?c1T$cf{XNErfH%7C69!zS$IX_F%q%F)3AvO#b)ezFO# z&3R?|6z_nEks`ULm(0WI`Dcs%gG@#LR8(Wv$c8(ca8P*p1qO$|nPEl%;|gmmqUKMK z;dl5!eTN53QYXh`zV)VZ`qb#)IqIa4I7IeokwFM{fivdKo*AoDjSn5|?E+^oTsG&8 z*OTYY8WuQ4In*)`IQhzl4;c6hrT^z{B+9(y!yOSKz2PH6SETpAx12eD&6=>-(4hmc z7C=AGU<~9(A`_gp>~(eQYjfw!9v>PPJ0)_?(nac-p_8K{Cn#jB@8vf?Gc$eJEO-Av zSg|x*HAWdIb9Hxhb9;6MfNO1isSj|2Z+0E-%1BWPI zo@Nhya2@8OnlLePlJ8(gFE3BGXJ-NgjTz+=;Ooo{cTuJ;51kSl7C$>FeBt6HDX9P* zlltb$X~1Jf;y9l{eXtrpA4gB+=;%3%Bj=(%qY3HuiPs& z)ql)5{|OP3f@UYqNtzozf6?nvix!8>o|BXS<(xJ@7MXWntOn4>#m7e`4@0h=K571& z%a_boqE8YzQ8~|;DOv0j7~nT4?e*lOfie#t79jjE8p5p8W7xRKF;in_1pIz%kWBXX zO$oqq0C`hKj~_Q=RIuOd4n2q|#!P9N|CrIru*}sfmZGc9T)U5lPsqqXzY#8; zN|VU^Lt^5T0fCATufK0%09z2CIN+tgfY6YLwAtSB5grO(zo3^UMNRe(89OdyT5Oyw zWQQYvB3d8&i%z~K>!B_cP~%>phP&k#lGPXDrC$H3UFag zSw3vs#IUiWkV}kYQuR^B#RUb;o*gIm8y(>D_stL(bMsPuUcu;zLCcrJ+MPhKrz&hb z#w`2;{Uaw$^6~eVhrT>BJ}hB!=*;+_xY>!*GS|eY=P!s!OpF~7EOT~r#DV}~M^|rG znU7*dVtQt{->ClZ!2tpBv9LT>vwXe6;u1oX+-#n2h;bG#&i`LOO~BbSP2k|9AZ4n5 zfUgY3?d~@^Bm`Y4xvNSQ=4F(>82Qp9|H+e;F)>q=(_-aw)XAZ*Bus*%Ne@-OHg(F3 z#OP4vP??*fBgA6i=;Yz#`kdELS;V;b%U!K&NX%jg&DQQjwxFQP;4IWAr z0#`?;er!UUle5CtH#jV8d@vW!8y&;FVi^}Ck9_47A7}61&^&^y*vTO=(Q6VkjGc#vL&7Ku*%ybW(6&V>cIx<-1je+I9DBlfq8mb8JRfbNTIy^q!U#XlrGvJpw z@GqH1a497+UhXCL9y@u`OYWRvlzd|N1b4Yy<{_7P2L^ih1qHkM$mKqxMve9i2t?o* zI!-l45iuz`U`$w8#9MEzTp6m0jEGRhX*3~`Q>MH)VcD{!sln5y$A-*ayl7_d)c9dx zVWGi;{pGISL!F$Rhx!gy#KeSz1o-sSm4Cgt1miH5K)Ym0xKbvIjDI=OLoWAJO`YP$ z5}P5zo#jYw{rvpl_1$GK2G0>A6uxqk4vY#JJ9N~jz^EmQ7cG)=W5r_vW9PpnpD@8{ zOgJ+E-|+AWzLDIl=SPPnjE0&|4ogTFJ2gIjoIGrxT;?K^D}VoU)c<;O3cSp73W~GW zB!-8{U44AKBWERe4;jw#TW9YPGIs?^B7O>2Ip;^T!L-z?AgIkjq$#b5dmSdgU@!e z;$OQUK;NEe2HNFmF}|ba!vd9@r^F4(ijORKqL()(mwL!BZ~+gd@bKV<+dIQZ6ywK- z`FSBTK-VkO`$gEx3+{d)qepw8Lnil-DMyZ!17n^4h{)PNEjaw91(f+pPp+l zqGzr_UQ$xp!UcJ0zFywLJcfJ9l_9R)K0Yt_1fY5d33+?TNBB4m3HBSpDS2`k7c3a* z85rp0cd*-1-+zn=Y0Bl$w!U!MnH7+_C*iZcicFYho=C8xgp$`!JZzj9$tf8mHr_vu3)04 zZ%9yRNSG`%ZROuT+2ENq0k(6C24H>7xFIZk^9)dq8LEs7bai(1a33ZQ3UVFfDH}2j zK`SyS@seq>L8vj40m;SLe0k->k4ViZ5XkK30(%7Voqle9t4J_DXgG*dK zPxq^=3DBPm#7hU#$bXRaB#)QC^WjN&uk&8!&EdVkdy(hI^XI8~(|OiBAG}T=n1tX} z0>g<18A2wIND_hX8{|jwEjdHZ;qxuIjY$d1@xp-_xcZXmwF7uh0sH#_e`M>{r$=`Q zFX?K}A^)2Vb`OcgThqzWdp&0@7E!|%yT`Ab)#WoiVc&BN`Vitucn1SMBh${0$`whVg+&2Kyq%KSaw3C$uVkqn%FiX zhO%h2+Gwr_7a{&way-oX!%-=0+R-NvjMC4 z5Z;1j*pcC@F(YppJE^b=D_2qO4p!V1(3(RdFZE_F5fiAN3JeAyJ=L}lqK4H}Gm^nz zU;-F=Sc4h_tJ)~98hq#3=NPD8l|gU`k?vhP@iulzqABG+RsxG~i2P z;wPFb2M{Pe0Wf~-)5bB4mze?8fI;RKvHrY?g{@B;V~goR*-YlfuJJTlpr`{58d#|o zZED*n$uXMd2D4kC{jqxd@}KgwOmaPB!PxK%NV}sQAgHxr!6g`L6=|uV0`5Rc%H^Nu zRHv5YfKNc5r^x{gdd4XD0XK$FD_F!p=-Q}2i}NkaBG@qyi^Wsm4PUQOw1F0O3LNU< z3kZgFnY4mw9+N5+Fq`UX5oBm#rJAm$+PawXv>_9Zp7I3fFxCRRnH|h|nWqPLaZZhe z+bB=R)DSlbVCV$=9n%14PwoM7C)iL6gY3jzd!4_1h7*=1or#5@0KyHgh;YMAZVI%_ zv;-6is&h0z7FI*Vs%i~h#^`3!2?h*P*vey7YAt2l=>!T)bPxj*iiJ)=>gr`HIQMjVd$_|@Gh1qHKfL4$Qdjl6TQhDvr5Fm2C4zWc46W1>_SSq3OHdi%W)T^9pC^6y47+Vi?8fVD^o5$7 zY7O`eRteyT8O(CVaRwV(W>9)?4(BbPC-x?w@D&=yYMiSEZA57w@y<;(rKP;9Iqh(aeVDh zTkg%_E~Ls$i*vxHU^8ZoAj~7dO#`8>(8}1#tPZBa1EN#_?)M@NLdNTz_%5G!FwS<~ zaA|?nbJwY;6F*d|sc^1&hP7tzN9+L@t_zef`p^Q?R%XtOv&@|^uE7Zl;gNH+U>Umy z_z-)%)TsNOu166AiLO&+<_UgjipD{`8Y z6CT3*eh6PxsE2cOGu_7sHxclxK7tvdYGxlgFo4>lMHqw$LKnwWcu>PyWSjyzuy#Ju zJO;YE$)Dks2^2fu%_%hMR*2URJyk>GfL%TUOzOs*wTfGC$e8p9OxpY85%`0axGE~- z4ChO>O}~cMjuGZ1n1^HzzY7_&$3Tw7>(;Qp%^(!{73v&hs;0>Clp3?0m<_hWBtNNJ z4y8Fvx718am_WLr`~s7}tZuB#T>pDm1bb8DZV!ZBWGzrw|+K{xtzq#3iCUE_4 zG7J7ps%FV9DW|j$Hp!w%FIc_V)!lZNUi~*&#(cJg4h9B|`-hb4F_Da%a_B)TLPn<% z(Vo*$vEf$9wztrcx`o%&k=Q4cg|%+J!G}Uo*%6BrU3>#j;?~QB^*U{@Cwxt{3f&gm z#M7`lS-4=HR|^HgF_a7U6AiXdD;uP2$X4)HXFcn?bg6Y0tX_OGHT0+}d4WiN(lEdv zB{FlPN9l0p8NX!;&n#!a(>+{ck%3`^sKsM?!j8Mi$9c2g;!SxBZ^gkRdF=+M7QBY> zYs%~V5q5X=9G4c@0visWnE0;)hu**xhs@sug64O6x9uJn^_S#Pm7%%{u+j0OUOt*bPk4v0fT zg*sI%1PhsfSSEZvhzC_Ya_$vQnuuT`Z89J$Zgc+pC>&)66}HrzEy*6)#25)wpdqGa z`Q@g>-_*B4PuZjb09e@7CF#vLoX$ZX!NjnxJ_By!=@qzOW|aypFy^a41WU@0Mv!WR z&g!0)afUexQ$yX%fdrm5q-M81t8fgRt_B!qy6!-;TDI&eYY^tZ!U=W@aq~|!tceI8 z-ovu=MkrK)Gx~ZQ3$tgbsOjzl`s8d<8{-JGk8-#emfYE+0SX*=di3{s3MBJqel1;kk~@p#IG)N>p+gLXF(eFz*q#r(U&r@h^0GBU@V2s z`3V@aDq-=jt~R^*)bg}B$Sw!rGY4(Lk_(?u``WjxWTZ8$ThVENim>)IY1XtvS zX54NO3k7Swg;=<;Ek9XG+TYE;_)r9axNY|gm)Dl4r27Q_MfspY&wpZ}_C&Iz+eRhk8G*nSe^*I;@Lh8q9(I?Wfv0!yO5oy($ zYN}&{HQF5Zsw$o?2UN#Wdt8gvA~(b+01zq1SXViGq^KNSV|y*e!k)SM&^(@7t!X17 zwYCeGz)h>iQqSdRTzRe?SG3rhw5z`22lH58shLqQu~rNtl$h}BT7Mg#Xv zsjx6MR;PD^kD^)(W)i-FD)3?~xKJwvMT%O^zH}hgsvLt43W~2-3XGn)`jChUb5yDv z3VHypnt=oF1gdhiim%q`*&wGLK`;kuolXPV)t8Cw`M)`_LD=m?tk6%>QKe5wPF`GwWMl4{#bI54WdBNp2(-lc7)Knh+W z$g!lpk@A|R0IUWffdX`If)L(;4ojs?Ll-~RK#2lF<~nsXQVKAm z_cc967BE1rH#Z4Uo5`?OYXm5TfFhp0u!$L?i2xPsT}?YxI0h-8o}9>WQY z4j}LtWPA&cbhH)u37jn!euv*A9Fq=~SU2teGBp&$03OT>@?w@I!8lrB4n(BUw-Je6 z<3{asbVLss_PPR<(NbNxrur!@Dhh-Jv3fnm^$JbIP&k5K#IPC;0V&`nUb4rYGZDh7 zFeXBDIgAjD{vLWl4Icw{rfZM8w(!TCu{EjjZwRHF(ZMS(gnlU~vImJ9RUx%5&nYYi z9fev=IjjeNprhx?L--$xgtSY{CYeBs1l2lws2OO2P-xCGbO8UmUGnIm!YmFb7(g>J zzQ)hpGfGY|r*I5joppFPbh9}Gs-|Be;;E^qphc)+0ZWLaP}@u_t5J(1uvAFi5@rgA zS6!}e?R6fI-BfUpy>%A)QmBU$pvbYhX(@+~?jB$j*2+X9F0aNY zpFJ}^VDTfUxsLp$0%VP*BacXIn8XcSBiEZ?aRk zIF$7?7-=>YEi4N$L;^2B(I2*S!%z~kF$xLxoCjeVCIxXUTD6DUG&wjnHpaXL)$!cm zcK2UA#)dK2M@E_T{kw2NQJ%{9Tk&J>eY-$9T81WN!e9H z(PbqqOoFGo{JEG(5&s`Pr55=W(pjw98H;fq;KO4z98_yym&}`kR5o(VSL^MYnWCYm z!z*XBSC<1p@Sa`kIt!O%LXdR{`8i!453LczNPz;)3#v`m5pZ$xSt@zs+7>%ci#XBXJFV>-@C-`)JNs)&@K_N}rc|YSjAg5yP5(A4^ z%8_d9Rf-2D9b$tz+QLI7m1$vNj3`vdDtRiTtgKRqToljPH?kWPw?QbP5FU60m?wyp z^ZO;cDr)eeZYU7?^T?t*zv)T}L@*8dh<E1mI;C81vA2FSNM7AS}qwJT%ybLTu z!CL%;*o<`E^@KK|y{5agg|&%RI%|-K5tAoJ?-~>t4H)3(-`h6wWOw5lW?2eNS$Es4 zi-P!sQxDPZ0b()zopSAEH1~+MwStxH25>3VH9a;`t4`X69Jd)x_Q|$S zS(0{-e83X6N92!$Q{6&SyNNB7X+g@aD|;_8EGd16XIa7kWJw41+wo!VbZhUR(xxv; zO1AGhd-3wc8dOi4K5y!{d1%*y(ypwthzL^NKLV_TZRalE?SN`pwq3n=@!@yZE`BZf zqUP?YvrV+=vytdGjl^(NVb@cei2xWlqu$Ck2GdzKhr#rZhmt*en?m=@iEJ#pu${Mp zH;E_X74Y8Tt>aC@jEQ7k3a^;=9`@eEX!BZNiJD{`4j%c@voq4GeL>K{Tq3!c)7f(yck{*Zvn4}w;uaxn1`_%i##w0 z?41pp0Hc)r0T6-!!yQj~U~b1Ko{XHtn;d=_N^e?L{e+gdpU{#CT6UA2CR%|1Gqn5} zP}X(RlJ)dm6nGm&LC<;;SaRZvjGjZj=VJXsYEk(GEi3IgYf(uo=II|(%d6S@XnyIT z#`Kt6b=v$HhTXs|ziLvTU(1jN2wBe&wG;8&WlEaiXuD^0E`@y4w z#Z4{R3wpGDf|6VV5vph|(lL1U^3L4zIEBuVsv5mOeogK?PQ{;>TX~Q23SH3Et=x~( z&AFuJbk}VFin9!;ly=md8pub`dEs~)9eC=UZL3q)JfT2qdhCJ$m;ZdKo++4;r6?!I> zh(dO(kN!89+1iMuq0dF|s-nQLdRo0&(`xBhWyMR|VZ@z1lrxn6sv ztggP{;32&7$WT?R#XFn#9@lkaXkwxb6Kv%Le5l@q%ma>i*DkpL44}f|3$!K|!mmMb zQ(u{zW>oQV%iE|J8mCgl=3aoC1eIb{*Ifp7C6--5iNaKe&1DZ3+F{6#cOjQ4X(4Fi zRWjZ6;VK3Cno1w+&V#Ykz${n~h8YeRPA+hj!W#M%DBd4cDMX;Hsl-iBp`b#{R4TW* zr+MfY8USK?Vx?6RK!S`S8lQ!XwZei+*p&@fH+`;0j5esBRmV%>hXw* zugblEZgy`rU~G$8Jo^4hgheNjTMR$ma-%*XeP(9VI-0gFXXzi*JN8s$969^d*X!^i z;ym3$Xz%d~yhyhF^p3sSy$5P8_2xVl`-Co6g8RV2j;PT0Mme0iFn)mhKobbCWpj}} zkV+&iE)WV&pN5QM9;H2(%YkDW!lPXwWCt>cX?6(`6`r2WII+Xyc(^hbU%4h`qd7K= z4J#r5#xJMcjU(4*HU-e#Euy?<;xEQdx*Ed(Bap@oFlTm+m&?kDKENRtUI0UIn6*Mk zni0KBa1*y8dY5B*71UbA#M%pQjvC+vZhwEYZwDD6>Q`{>QpBXRm~6U06aH3SVnJa} z*2l+=f3o?+>Gh{>-no0f>@<5P?w%{H>=pDUi;f?^{PWyIp*an_1c|)dG|*|smkD@f=AKxzb0vqr3!6|UDi+7CSU>8#>kc({%e1jdO z-7VrQDx-p2Ec?Tn!c<`sbY2kdV~?TFnIz!ITY55V)k}vP$nfq&k%@xfft~Ce>p2 zVD@8dd1axB)tGZJ0_4LqfR#wVA`71jR6mcg3e3EKBvqBhTnWxt*yUE1qe%-mtK4}| zh+IKUWiHcp6_?9{Rh!17%8?4;1{i`AO5PhX(_(%u6GIP-E-t%ts03-0_5#9$4eX!t z8$)V#D!5v+8Ufi(?w_SX6viPwe@%I zp51LUh>DNE8)z?l_l!tvOd_E^#BN@0noBNZ0#XG&&?f*!1%@3GA8eWJ+JU<`Zviy* zCdA;ga+x@A&ISBql4MrQt}*pyTZDfe>$F<90CT3rOw^43X}Hl8oXs%@;=)-0COOW7 zFaUvx6NP>TkZCOz3P2?F$)(RUj@RAM7O?84E%%l}7gDhhJ{*iG&@qOhS?2sOO92RV zWeJS+%y?mOJc??E021m&925cRy2xGQU$vzkFZR}z?$&De{-5@~1+J+pTR#x;4iKPH z3N0lCNy!*VBoM$z024)05QG#2d5j2E1TAW;RI6aQrD32_saSEiW}r~|Nd8`Op-M z8)Q>I2!z?h9TPraBqI65dYCB%jfA^!8Vq<4y&7B~=YVb+pc(hhnfNN)IZb-Ed7O7c zsBZQH@n*ThgcNh!Fp8AL1_nV@N^KECO&#_XZ@>fbjr1BJslg0q#x0cP@`} z@2SGXd+-#)k|~9)9;X_O5V%`|PWs)6Fi$*%MhcN5I;3Zk5x7Y*6SvGW@l~O#)hNx^ zYN}+83j1gWCZ}hwiG1;;#J6@?Jk;ng_ZjXq$99h;L!J$((?4-~jLO8_H}xjOGb+2N`8^x&WrT z(|jJ`llguh7IW~9=|U!MIr3xQnPLSzuK3!qm{D zg!EU7O3TxJRl04baoL{wL+@=o+tqum;6lg6uA0%2kz_oG@eTN>A|HJqGpw&PC0O8LE}>aCZ<9B-1xJ$`7y< ztevE4e2)G*424l>M`UE-_K>LlH@JGs#P5w*TUf9$S|fhK*_JHMNq2ZQL`6l-P|S&v zE`9N(jft7-HWnDRtgQI=cQ(EI$@wn|u`&H8M#ONft$qDB%WC>YhC3>%8e0yuc9J!I zXaY2$;U3Kl#Pq?iTDBAWYh{28KQOn_zaXq&k#rbgFD!FsKNx|j`l}pcPq-W*4q{}TV*_!j8$4?(D{g0L-B4SmM9@}!lEiv_ z<(tY{_8vweOx75pq2EEg{UpQTHlRf0FX9|07!GP_oex;!2M+h6$7FD8M!5(lSs2Kf z#6uEiLtupX8={Q!X$HJPh^-M2BhEu7M1Xh1RzEP-0}MA`h-{UO3le63=fXIOV3Hht zFcQR(hfJ_JY9XPN`Mhyr#+8*&1V?Q_{3~&eledrK<0jbtZ0@R}e@j(|$Am|geVLNw zD|5lVlT)qTL!wksQPFeeL`BCZrA4n?`*Q5A`a{hbzp2^#T5D%lZ^@aD8un46k5!uE z#mY+TmReE27kj0i*}L;qs+EocL?>C{$Q*Y_?-bMoNylqHdH`^b zcrjmWaz4x>k8>9%3CtPC_z;si0Exh1@}+y1Ch2#lsU0r=JM#ivV+f$w{m=j+8JIva zG54Yr5&_(s3Uh#NVdjYdCw*sfzBM@>-nf0mn#6>ag;5(*uIwwzUXz>UJAG&N8#UEGd=Z`xd;8yOR_1MaYkg(g$LC9{-fyh#zco;l& zI&7)iv9)D*_{;&a=#n`tArGS5H1NoGMjKu#<#6Wa4E;t#rjn9bQd|?BTY~4f>Kwe2*;K4EPO$w;HfAI%q z1;-z{x_a=t)6op)L-GuwqnK#eq-F!|4unG?7}DlF5D3gAZk7m*C2k&fbzeRn@T>zS z=X1MO7gw%bVc3!N`udhTmG7;r*{M&~KC>orCXmzG-ltDwp$|?04(oBMne`KCB2KdqwBv(urZ?jXL@CJAC?r87 zU?4ZdBM)2|M~Hz3;zLIL{fEJshp9I*8lq1kI*&WK); z8NL181Ba5{EYCW6?oW+{&Hwp9<+83sDILXp2y%XpV0|>QHL$?XD!9oHyB+Mq!C=nt^8%_V6VaCN9xTuM1PI4Sz@QKSo z3iOCq^tk4OW(K3cbYEb{-gCrn(EhAlolktlp&jnlHN=^_vuPC)^mFSab#Rt>GsNk#T#=A zkDffUs%B69uI6j~-;^Lr?I=6cbG5Jko72aUER`PYx_Wc4qNR@-VfO8Qqi%1@0kUKk zp#TlNJqE)LL@ji@wtox8@1P8{W<0Uhxn~S-1_%}n)ieb3%}DTfbmF19ew<>lr?Yqb zI{+t)pL(0Pm+_Nep*wMz&MPLXc5?aKCaX^JigIH6_*LnhI}8T+?Q6)sr{4MM_O-6! z4>xz!HkaKnHQv0jz2{QlA3r)(dicOQY594PnTZOi6WpMZZs#ghgejw<09~A&`v9cwjNvFX%L3Z9LR^ z>f?fAfA0Ubqo$1-;SA^K-~HYDCl43YShei!Y(?HWE(Mch+%!G)(uv!quJr3B)9%sX zrrU#+H*a+8yVUWS{?`?;7@x{yMrP_ly&OSC5XaFoz-vxa=w?Idn%Jc)^##pGK1@S4 zkw&tK_m8bbwW2hua`*4v&p&$MQt#3I;KIm3U4HS~)d%uV4i7UAZmsWX+I!%u?-Oz2 z8nVIVXt~w*jW&%Am){<^)p+A-_r`s@o;}pxX9VpEy!)LKl-94oWWkOknF?xAUc6xqc@zyn~s%rP8M`y1qcs*<9!R8ZX?~~f&c6C&A zQsyf)BO@0!zQ;Xw=Cd!m+sd{Uw6;F52G>>eC2x1N7EmM3Eu*82!$Y^r+Rv;kHKe}0 zYIy=#R5KgmprkSxs<{exd+f38LHCeFhhkSv6=ve)_^e-*mWR!WPF^0L^hQNZeeS6X zUtH`eRgXonVroZ6&#l^3$AD~X*xqt?n7McJHZ(=rA6S8Fpr`BYk&*h5kveMBt@_rN z`OSyc*S+)Ww9=BI*I&cH>YTN!qd~>oNQKCmixs$7woabUDzpiqDXZ7zD793S^&H*W z{F1kqHzMu%v^0DyiXCIT`R&D|SKp|xtl3#Xjo87>wG|=y7#SYfxx8t2Q`64Rp~%f- zM=rd;<~4Vcjj!;sm;4$;v*L=$^ZJFi`?idXbetI(t-E!r>`$MSw4M53)jL(YDtGF) zz41CUC82s(Wh~@0OsA2EUC_93VmSysJVas?6ST6BMK7$`l(??}Il@luSQJNFex>)? z-1MAR43_K9jE;>==_}j^i5ML*R&)#x4G#|)4J~{3_Tzay13I`M3^`VFXD1;tU57$U zC#~H~CC#wU{0AYk%OvmZsV_b=JhC0G{NzCC^=l>FmoJv>I@ECRKuPO+4LQ}u*XoWQ zNqsSUxhgX=wje*-Uo+j+9?NlYZ0#O*m&il45#>97Q@i(|YAlL7zvSG7FXl!qNXaU+ zJx83`Xm$A?JG=Y7*;CnfcbHRYJl|Why?RSa>!k@I4R|=yf`+`2JY7dhZWemCZG@GC zNW32ajh9U`uU`JDb?@Pij?C6n-jtl1s8h_VN{l(|{H%s4m zW##J$6>pcPzFAr_H&UZi`nWn^?>Bq;6en4LDq)5a)-Y`Cu@C={hH1B0*5mePnEh|p z?mBY%Bc23-$Ys zo*AybJ-Bytbl>RcP~j&Zf7D)mqwi|bi4Tq%ulIH2@84UuuC_FPZ{4z;?^GpZBOchc zRT~+pk!yT}Q|z%e7v0vuN0YWHPNCFBCANId@4VFAo3XR@x6LOC$)qz%>Usw&YN;WM z!YXRea;SIj6%*4yYuC{PdZ0&Kz6*nv_)c)%sCz*+Qy^DC{8* zPc!$ZkxQc{jC<4pq^g6{XX;P{?N_70^3bPWk9Qp148tbo4)w%f^iH(^zz7{K&gevT zE=3N=q!}j0W9&fgnBIQ_-*iKP7llHGdPjS@8vP|EEkGIiz&CV6{sSYKem2r^&I`L%73$A`z<1k`zw0~j#pAc{6iIk zgBJ?7*ECgCweJ4Zrr#aGMW(`n8Ffu5WJElSCQAxsl;8A@pY5j%Ldp zQUU4J2@v7y@y-PN4u*8YF$2xeyu(e}Z(`z`%{v5Au#qOEsimmXvCLmE2v8F=;W?i_ zqw{Wmr@QAub5&X6$Z$Qxc}q|Cm1=5qR^Oq{FIzfBN8fF`aK7Tyr-f~QIKAP+}LcAeXQ{*UV+iTPiC{>Rc&A08-eX{@i?c%bP&(sEy3 zrQ7uHyF%|qStq@2DEd_(*eB6wR4yhV-O<+G9!rji6XHv@Zr$G0)ViG-us#F}HqiTK z;iZ(hY-|lhX4iKcJQkm4Q%NSH-kwH zmOuDeKkDXaR}02335gwNR{stI9oPo&%|ehqKy0j;P0er+J7{;C@i!O-9+Q#DB+{Vn z76KH@3AaGNEimhkrqF?_m%AHk8tR9J4xjtxn+n*>LuYSXukITjsk?de>x%vx<(Dsg zwYBrhv#r;!8BcxoX=&SMAOF|s!jm5!&)Hr5>)f4(8Z~24*6Nb#cWQP9L`BWkzgY(N z<~}z!Elnly@f5BoSrZxJL8B!`ZYnWu-`&*PyLn*1Xx!a9@b(Y$$^=&7ZeZ9TbHv7R z3^Y@^9#d=(44XItLnp0YFEt4uhYx%Ly;9SpI)mFiPBjBjfmmH2Kz}XW%qo-v9%L1P zd=N%q(tzzE-!rp`B)VocN%e$HpPKmBx_Z94&~S3-;K{+Z)g- z)ARCIoH+4pV;WJzoMRsxe@$QZ)?2SO4(>k=d%tgRa8K{RKe}@aps5c2(Z`rpt(U>EewW*Q=<}r-m+6_jI?NMJ%%A>eUOm zZ;r(L)({M9@F)6pa$&xWE#&F(}M z2hC0zWbpoRFb+91uCfPL?T7m{J4(1#_&{7sYC$@eX6walMqq)!#mt>?6Nay;1(=NX zbmA3o9ekNUpC+jxOl%jpp_*qlih>=4iEVgN?AEPo-Ip#?Jy=_-&bQyKdd;+R{gj+V|C6zkc`}RCfF5=;7l12kH;BjE!1V?mGNdGpFx* z$EMaJM~>EY0C`lp|J3Qzg^g{M)S&H_p`ooO2M1dVPqq~{4H?ee;~+ABlx6_0*@Ny8 z?$Lr2nt^$Nu_ciP*hiY;mOBLfgGBw>iP{o=y6JG!kQegV#8G%N^Gy@jfu(-JS>kFm zJ*pM$xIEp>{5gVGQh9rba~A!m$XG}*f#cV(=m%93WMF(Ig0}Q#H3%Y<5Wbx-O}%lw zyQjMgKizFr4P|?Wk5%-I9INc;sOTFSszU?}9t|2>s1Zg-^Q~`M&rqWrDDdf$lP6Cd zJ~=i@FFN|sA?_Kpri)tsc>Y3l6E*65(AatL;`9GpaMNN(k*phqb?toY3A4 z&LQN&7jEJvj+hj8Zd|{1wY%%9uAcg;n(C^C`lGvB%Bs$w9M^KbvSVoIRP`Bzq7Z`f z2&P7n|MeAp`DN#c`mqtqEmy8s96U*lJhNwO>(y%=Tgi!A&VGG!wCH&K_TC|5#rdHN zC2Jdur!QZ=dW=X_v5}__SJyPP z?98oc=_qalhueyGS5_OF^!W`&EaG|;eW9eovT!yT2oTh1KXTGi6B-m0drqqv0{p3+it z!Mv7R^SSP}#@)M*l8xiKE>T^yYriMy=rz+@BsINh`i+TdzJh^vn>?X}15m^3J!GUV z4B583Wp86o+rIkh#+sUjw$t^kWa~Jqp4R#+7pQv2b6@o$M7suWKxDAzD`bb4uU#83 zP86srOoCfCu617_pLBPBb?Hjam5Wr5)mP+nPH%U2Z|~)<%e}Wz2EBdj)}0*_-zvx2 zSNId)J@PVg4^{gRzV_|=&{^PXPj0^d=LEj?=t<7O_jBceR+QoUt52~ zXeu|pR9pvq?YkwW)TwgQvaEA0C=C7-eC@WwC8pBCg8cxxUUb36li0_QN2768b(L3i z{q69z_9wO6}GNmvYQUk$V$wpF*JJ2-o+_&efjAN#tn_O09%zuQo2I>utr z+1S1to6w@Rjop-)PGN-`3yOOG&%m`)3Ste$SJ&0$SD4bUfi;U|$-*ApU=;ap!e;PJ z(*x!#(5$LB_;&}_J~08VO(@@yWy~lu$*~(NmyZ3KS!{}7kKKGRqZ~XWh87mb6KJGPu@$99$c}Wm$PrnMs1pzTzl-KY(ZmEENmOjWKeWIjqT#)6Ku=m**iv7Kf=_S zYk;r?KN(YNHjw99OXe&H zW7|$~nO5-#QEOI$am2^Nh*|(Y7#xaEa~$K-ZTT!F#iMg=d9W`Gd$5$lAES>km~@)G z^RrLF-ZNP)@@3^)k3Is^eutq1Osm^eT2zn_9{moF z!6#CRE$$Iyuq+tNyYQiWdwI-~7o(jV0>V-YPd`G@5@bE9O!r{z_ht5$$>w&~`uN18 zSE7`0DtQcxfrUXBY_1cZZ$HMT;XjurVDWfdYhnh*I5tcg5)!0#G~P3k0NE6Z#&FLS9D=O2M*exS87vNr5})F+tr4Km-~)IhQm12LXdFj3S9_AW+i|Bn^{h;&jEMi` zVn-)M#QM^+|BATTCzlnTDBi#J==+;;e_L?!OS9`t_%nVkf= zlTdKb$te~la7UcEIDjC2Ue8@@hMnWoN~= zv*s~w!>3q5xA@2jICSV5j|msgrnr{YTx7oRI;1iU?@w_aBR5aSYT-{j<7Ca}fmC72 ztHlF*_E|+72ur1i-H%#IdEg&JX}PZjQ&LLlC%3@|JGP0vR0C4QbY6d?uYI z@|M`59$-Ss#B!RhsBd@}4a>USRqYoz&4mwY`A$w#Y-~9Kft}M`0Ua~W0prP-0IAto z;#!b5azmJD9@iE=-I7kqJ{D{m5jrv+3(XTYi^=7hIJPXuK<&!hS4+xQ#4g=fkl*?+ z4pzHxp7a+kOhmvO7jhp{tgUUP*o@h-$gsB943ZLHJQnj?8sCz^;Bi=dE=n8%9>r&| ztRZw<49c=(&}_!|7!VpxXnLGW7tT=5iCVa1zK~|)84!^cy}-D&uD<;9|33uw@$2Un zWh>+7$=sZ6nNPcb!^k#aajYpD9>>PkmSby6%)S$4$FPSeGA+M_f-&h#Cd81%CVT-= zV;l>Twn7&vx|KEI)sq~W^%R$9r7{Q7T;e%P;$M4b|KTHlC@pBL+5Kbjua3Pv#>|!a z3`JPLRQXJMI)lbwu%QQ(H70_O7tAxyb_ymCT%2NS%Vlz)1>c~Z#$d8J9F&$xQ9=L$ zSsV&qg$yG;qdmc)lXerGCXgk+_;SG;+t+VD3+P;BQDe)G!@k;}yLm8YYj#R{Txim4 zQoM%V@Z9V%_7pyhH=E6b{ibtl$qOIHL&>d~JZo#Xccv|&a-2dMxCn8##Ei2!2mlD{DY`w}Jh5{$8%Kmn%EA-p ziYlpo)~2l&FI~QJq3WLl?P_mEyB^!LE^(*Q>uFm)#F#~~Q8R#z9OFRk#sNCRg0ZsY+1Q#@m5r&zio0lOFfHi}s40iZMrTO8{QW-|i>ZG8p2APh9o?|S z*ic#7^v?uvwIKl4gIh8qlXX&MfYh2o_-Dyv|EkIP|R{!D5M&recot~9Xiqg#1;i(J4mE2k$M9r!3^GZ{4Wb?Fp`Vw*T7u06s%2pTba zQ+NV_HC!<74*xEX1)kA4Ook=k-IhZ%XO(7IPELMh#jf0A$37@M)Yf+9{dFt$w11?J zKK;W0&7UWj)tP`s3x#{8!-})nJT{x6+mHkst$LU^ zt~tB2X4nW&jOLB;t?4jXTow!)sZjIB`0flA6R`o}$^}`8D*011TDU58S?-$rl7@!# zvjejV%wX2~!=D4VZ)d2kcvAph98r+B!5WX}Lxs$)4h{wOi$BIj zDVdFchR)|wJS((G*hnWV5M6Uvs6Cr=ITF_(06~0(nAz8WdQ&nNn zsX3Y#4;?DpwD$NttQ-()4`9jxwgp7_=LKYa+zezrZ8WC1(>xFT>FfdUWlGuMLn&sX zj_4nml(~bC!m#xioSikBjg~$eO+Dh#X@qSw1P^Q$ntdE{V7%+ceL_)jzWm+x|7xiC z{PVK+J~*~Bw>C64N4IjrwikbhJ^u&**2l&HtmPX^j13JrNU6lF*LoCKp%@tk%(9d4 z2`0hASl*@a5plsWB00rzU?NCb2L3U*bPfZr92^qD3*sFrmG)~p_U!rW+TQT!oDOHz2-pC>{+4lFC>SN4BOS&(S|nFU&f;tY?jDBU}lhjPG>QlJ)iY*;6V(Y zbW()PQ#%mBnA=>pwWGIu+u8oJ1uN<*PW)u3>drus=$XXIFjt75U6j5ohX=ljLc&AD zfoV%z#^}sxx@F6=?P;Dgt} zFjRGCQc`Mi(bh5{h51s%XJlxy-^g|(fS>SGz24_32V48_j9-+EiJ+e#<^URS%Y%Rw%(*WPsZiDJN6Jj)#f66eRqYD!_4Shb zxOvK?qF1{r{+3YHCr?y08}hds3jtM)4Gos~`imugzH%>Dp-e8x`8z^Yr+!`Z+g+Ph zzn@=cI<1-)JUuu_I%{TtL^MMteOl-tbd|;w=&=d+{~22K*~0j@_7!i$uG*&D1yK^! zTu%-5HJ2#-aX=xY2~K0v;@t5U$JtLGW0%pRadAP^E`v|xCzlK}&drBDAxhgL;FZGr6 zv5T{orH3S^%?@7>6%j6UTSq~7kL>3)857e4~7qBQfr3taWJ z3W(0h%*o4(%GIx05&2@4COI))=NA*Ni-}VE&6}%HO5{E=FEK2-k5Hmi`MP>YMDj<- z)$z3z@1p=8S_{@mp``L;y1Y#r)+Oa;>N3(&leCLdlVD(Cqa&Y_!aw?Y1tDN|mHGII zJrHnsxq3#g&pq@AzWROmYTv?wveKfoc%?|>$U>=uB@9`bMCHxU<>ZB}elcsAHe>lR zFgPr6;qyvGNRUzt>m4E@HbYF}k7>$KwXdtYQu_$SO4gVDD=1dm+g+QI(lTC8R4ob% zh_Mrb*+Di=fx4Weyu7$qU>#z~(pXtx$xD8jqJUxW4}ql?%j7;1i84?o^i>2(L?ROG z$%LNDjd-B4qU#Zs^@p*n_U8(V_r1F=At@(qS>=~0`Lk6*6gI6K0+;5cQhCaZ99WOs zwQ(fJO`9E7QCZnlS=sgo z&iWJKtd2*^8aCxGU1ivv@tYM_Dib0W=jMkfiApeSJOktN@?!7iDH5|^erZkEBG{dT z)a2QL2w5bv!-K^njQ5`z;4k*{ou%~fAgR3E(_P}_8|**bCoDW@=_5evj|8+jpFgyD zTXkW6Ra1WL_SQS83;dFo#Kh)q4h`|NK}Xz4s$4pjM@yjc{8M!4i<2X>GG9=jo)WCm zEtnSQ@8_Y!)VPUe$h{;oALRHhF5($-KYyWbP;iLSRiubmRe1Ve5!Curw&C>FLmN(? zDA?3ebiGTi@r#V{NnWx%_T|k>13a-$Q{t7=&PZFqmd%F7gDP*2PnYYYWBoFXW#$(y6N66NPk*&frZ=P6R znS1J!0^@1p2~$CU(kFBdxF4@cU%o0eKPOP)FiO)rtq4!eNm{>dO;ldqhRD>Ug=*ch z7{7UQ)GE}il!$I-_#jW6<$x+I?2-oq&V7Gq( zgzKlq8>Te5ub27W!vkU#CX+a0bB;#o3O_iP^|VYIpPibQ7oD*zeR1TPRmzyC2t{o2 zqJ`6xf&OAO0uYJ6YL+s<-vbGXmn=ABR)FU;nYUCfahJ`W6Q-V~oIZEy!5@IRy6>y& z8ff^u&ScU^Wgc>0aiCHrofAGkZsW$d&6`v4nAIwXM#-E(Mv#M09;QUR`$Af3 zM8@iz>;Ub&IZC7x3X)Esa7-xDER3BW?xPF}R7jqJ%kh=WePQvU=NE_-^R!Hvmq_B_ zs{20T^(iyrH3x{-B8j)FnGPS3uYX|J%j?I9prwv&wu)RjSAs=Y101{+5gD1$d9S>z zd_E>BaB)VODh^?HtTq6VpnrUNN`g+Qo)a1*c7v7mmMmWTV&;HfjWX>5PcC6{@7 zUHB`w*T=4hVF$W zi&N%=#_1Ml7q807NC=%DrJd^|A=O9UrFnU|tCQXR{9&xPVd`L|zeMORtgL(({wloP zWt4(G_-diIRH|J2k~VW??7CMrC%nA=m5q52he&EOBO*R@niC6)!LktF2Ix2BYF8sT z(?%n)3(MJ*yEo^?uM_6xsU6sSh#6RgM?%VX;$J~Ej$dGU&-R-e9ubg|x;Sk~*wU;QBeSwqDT~vR zAgQrSb*K(FU^!SDH>p%2)1aP^8?p4|H7k}XQPk1SQ>KsQak8cU3f~C*i&LGLTM`pAXSUa@K;M)#0e%W0^iLeFi;G{Vj9Tz~ zXv#0cXGhGB-k7&J65L$1I>T2pUFd}MU>%&Eo~qF565^ET!Jyo+U@VWT_X`eEYVtO% zTZPg!@n=#GscNRb?+iCjrFjDVRMBxNl}5G9_~THqWJTCY#dH3OP*u1-MJ$`);Uo7A zcrGGxzMm>&wklQ^Cs74UGc&W2<6jI-&v-$TxjcJ8sxD9=b#-Ks6=Ch|-9?^$0jV&F z+43+9 zPEJ@6mgOWA3nfyY_|)9IFyC2@FxU!3q7G^)9G`cQCN4>(q*5&8so^f)*j9WRmc(@236HSu@EMA-xC-EXS)mx;9Nz{UK{1DZmr1^3p zi4lce|6~ioo;BtK`-G^(aIBsIfq_EGdzP0hC`jz-4GS3(Dp9H>VPW&++ULUjBBTAZ z3t~buGFPrvCN4^f%iX+L7rSIhL|$s@0&PGj7KHTzH_3vqc$&aP^t6+!NF*anjFE>^ z$qYqU;-Ub}Sh7`;MkC7oM~2R-33RqCOB)tGeXiOck%*tCzn|PIARthJoJRsSif8)D zLqk0viJq_#e&ON40nxGXiFoQVR-3u*mGsz!3nT94S!r@Z9;ngG38WG&f}*KK?6?4#cF`iK=`?KS0Pr;R^z{B?mWM6?Q_aSDQS`F$jI=qJPTa}^*noi zSYBRAWJ+?_Se|=mvQ`@qqz#ma(Y<#7Uh7GhX+8?MGBhT}J2BBusf$@ zE47I-kxU#iKjJxe%4e2rUf5iBnM~p#lZgHOJ$wTKg;JSJI%`&tr@|keciL=quuphI zlpG#zwq?bFFr`GIO+1)H>Mnsodd`^PBbS+_YnCcx+N@drkt?#ZvSd^UE7)JR^aa`6xxC;oqK5LYu(@(A zmHb4ICMgK2HeZvJ6cUq|I9sN9QYLYe$ds}lV$(kYpjI>vsI4sAoEoN)38hl8HaSV` z5N%3wa*TxXXDTFN3G=11o}J~3 z6^L~@FMq6N9iWYl(gwzbXp@rWE8~&_7cGLJOOfM884>*e&;FT!wOA20LN(1fQkx=` z_z^sThe#?BK~*LGK^_wDQYfLk?Y-a|Jj7Dj3i=kAwfXE5bV5$Jn;4(MQXJWYhhX zlqcsIM6Oav;5?C-l5ssGs2{?b`*?Uz-gd4~Yo9rDG`=E)bV$}i#m_>+r@H&9f`UXy z;bk5Y<;#KrL>Sqs6hHmPk(=rNbKjQaPt)V`O1W{S+jg0H$L8; zS`|?mRdmXHAGvD3+saBA-25 z?cqsB+fGFVKkW&~h3ILa(of|k1dl!Cs(?_HMiL58;ZIr=wu;Jf#s+S2UL-pAR49X| zDYgDWSKvFQ%K`#~PkBnbro+u^15(eLk+#OR?!(JMLyOx=OV9q;MPaMtBCpt#yPWhxh)M{YB8B zitDWe{^R8S9LOBF-~V)Qa&mG0b#QZWadPo+^YHwm@$&QX@bdBU@bC%n@$vKX3-I&s z3J41D3j*Ky|GLRvclfIp@a6~l{PpvH?AY%F9p>i*arkm^90PG4=HNQavEK`V0F?0j zb9Db0FYv*^$;AzfgN>Ms<_g;j6poFB<5$WT~Dkn~!($Uq^KYPx={DOt0m9@>qYu9f$ zIyt+zx_fwfdHeYK-470V5Q++W_%t#qIwm$QKJEF7^o-1xud)gXi;7E1%U-{!t*dWn zY-(<4?e6LA>mPXkVQ_4GVsdKw%gij1wDj%!kL8u0tJJOSU%zQP^xeI`=;8oz{ST)9 zld^wB*I|GzPHt{4Zoa?h;@}MZi}1tTJPKO8B4!SJcY;KZX+Pl?JD-|g(XBNngt!;;s7R(>o5okVzW;a#Do4TH#&n< zO?KV~33eEwtcWJni&~PA4N_|Ia8Uw$#FuOSmc5QD!q2YUb;QU^$veWfOxZIRddx6p z&?(CoN4Z9rad)#jOZQ`ttP)h3+v2+l$BfQ|JXF{4Oj|g6ES>0>te3t)PB#lYKrk6j z+r`?Wc4&XlRjANDE!5CJ2j_UZU6l|@U3q)?C5&Lfoub~{+~jmrv0K zC_TayTcdf_jz>3@tmdk;Qf7xrK4?^93*CHY!<}#H<$AWo`{oOsh)}=$$;vGH*!jX2 znck&8@lDW9xlm$YHuZjwK}sSTMl8o4!gg!&bfo9s{JPLlTz@vkrUvcD>K5sEc8s+)-J}C)_NTThz2;+6) zAx$R|enxb$DFH1>gv3lgM8?$v>|fbqVwB^*zkxr z=;}a$wNo~ne#&b7&bdhoT<=iZxOd6V*&>F{hY2fT!i-Vg$JKHfaMNs{h&b1~(tG+JD{Dlebo2s6AwX`7oHxR=)DpC+Y9_|~8J$7M66 z+~xDE1gtmY;7a43eEOnowjt`*knnNOr%vx8u18OuIF^)JkdA&W=sH)CBH#baTE_nVR0OOJjfEf25IcoZRagFzCACXb}11tVUq5mU!e_7Z*je;mcPj z-K>33(?}y0?W^Bsg6RF(Ax z1{Br0-96o*K)*$t+b)=y{o?x>tZj%2An${g7WAT|td7RuG#@q&4>!pq{{de2W@x=D zdY}*C|0T2rKV|I=dsT`fudbG})7;Jn#z&e)2mH{I_E&BAgJap`NRarVKdpx6((smOg0j}|LU7k+r2eyw{! zhYC&(>GeL`Rp1lqR6=xcDGvD(BP2iG-Z&q=xHN}8!%$>@ay!)6(n#3{bysqZdOsg& zPP0+Yg1vh1sl-Mpb>`bW4euc>rB}{rHp8&b`yfy3%s!~+VZoZTe$TCAV^r~}`6DH$ zigc1sk+Y)rNl=J;yX70%oZ*Zgk9DEm`=I0nX;wiy=RW8#6H>gEg^YnXwA9px(|ZFR z$zI}|&*R2vn3`*>>w%9-(^iQXd9)5izZ(nIjAYBB9?o^MrCl!7%$RQ6jcR;D6o0uc zWwGnTb?)p3rJd^_2Me+PuZ?3HnuPVk$Cz-_mkO9%lND&QAkt8!Ho>L2CEE6Q)42bS z3{+ZZvTbB*#%bDCSQ@NL+qBKiD^5z++l@UBuT{=%HNVCFOb}W^&`x?iz3P}oY&!Pl zO%+UItox9=hIgd$rk?SXi7xvCLbb-m9$}GpgGOZ-a@#z!NLZ$ zk0_+deR5r}PJAPNeB0rMm2Mup2g?&2V7Ym#wV8^KLpy{#Dp_@!ttx!Io2kDeadq}x z-G>Ho6N}o0l3W0*@-lXZrM&=bRKUJ%5_ISRrIG1wS zhxp>=cL83g2I=buZlcCf_a~<$w$%-3Us*3hN=B=K!3pv*RLPj?`n7i#<=pLG-gJqI zM(`(EJj>j^x$2lA9c8$bep9a4Hi#e$y$ybh*rJJ_0A`K1A z^$iW$n};jL43DQ*EpF9v$%tLowuV0$-v=$ef^!$s!-@8MsGdHYh>>Ttvu{Gl*R@Q^ zD`Bk#VGsTNk3S)yj&S-uh<~&4{ek4Xr?M+(>Q5W3D-LW<9zWX$QW7t2#5zcx!q zKlE!Hw<)t6B+J%Fd4#AQ8&EK{_TndKiofz@cwmO_55IO@U^-GZo?sDg2rFZAe*RN+ zGTBxqPhbB-NMrH*TjGeIGxa92+x`o5QKieLzL_b5Ak|K#`P{p(^Ds3%!S`;EpZy@m zz^u|&)_Tgt9x9kAIz&TLAt?=zIQ=X1tm!^|r~7Unf<$LuDW>KW*`IQHQW9x#=EJGo zz@qv@*;XD4(545gVISm08)d7ZQZCY;g5#Kmi|-@|@TuRz$EIL&4&G{+TB<#27q2eH zG~`$(HVWRJ&NpLNq4`_Jn2##WsotHN#VFUWRZNQ@+Ao2^&bB5SEtT&#PORFSzF+-g z&~yvVi|N5~VIXv_{(X>;kt!rnXCD+ZyAz{8DelkOE4-{8{H3@hO)5XlbgVQ3sd8he zQ9M12Kl?t~mT?#*=|D$N%R6%+)X;=qIrjb18uQ4HJW}`t++5L6E*;+T-KZ!Awd$1 z%$JadUDoR85p0rSzCv>Ug~rso&I$e*Ga1B0#9QO{ua+ExL@!8v=KK0Ul4ekxlv`Tj z=DKPS@X^NjU9X71v0P=2^vTTVtNcfk`E>(Zki00{%M{I+V1nSvwB**xl*93NFG5B5 z^xhtew7&k>3j{>1+;t@;ItRfpj>z>JEHl+okIhKPx82S_e+mqTqt zE~wSc?uI9Ib5=!`5l3Gn#_yF}-&RaB-aO)|c<)c~9r@o>b`pc1J)A3@(9hK)&u@R= zD=Nn7-6+#6r1V7-?iMV&=E08S<@f{JarYdpKHTs!i&DD6=L;oc8K+%8ga{J`?4xP- z&y&tw4X`!(iNnW?^knF#%@tq#08{3c@T1wtLP zA9AE?5iCUi=7K8uqa1;JZK)o$Uh1=zf49%m|1Bv0)bhre={pc|0zVlJU>I+3kRK*J{2nHP*a?_)#ChK!JY&UUA7 z%_*UzeIBIqTna3$Q3^7ZU!rW%(&$=LMBiRaM29qewx_=}r!iCh?XYOFGNR~K5}sm|ShN9c${@tyXX z?^~g=&!;*H@;Kv9{3mO>T%=H@mFpL;o@wYJAd$b) zuwO)h?vNPlu67Fj;WAT%z)rJgaAJwTr0#>1ukaCqojP%p-y`(nc$N~5#tm3}14IJ* zQsMul;s4IVe@a=xQbyQ<;tWovAGJS_b?z#SNiieI^`v(@E)khuUpryOv7nrJ%;z6&0mt4%K@ z&4n{^xGvPB?V;2PTr>h7Bp-`&Nq4n6dxPh(QnE!5oHFWa#X1C^TLN6HOp^XS=%i{o zb)e5ah9H7gAlY-EmT3O0Hl{gI&g~OS*m4j#lTA8n`%>v@MPya1l!ts?gh9j!<&XJb z@*@hPXY{85vcFmoHMgjlj0`MpdTKtkp7Hym&*u}Zh@5j(>UW#GTcQw-2JhTPYXh^{JAH%M`R>m11*U>)1wV?j*6Xxh?n@|DRe6-ThO?CVdmyg@U#=48#^?<&aWnKa@8&jX=IJTOV(g_6u6wos9qCv3>CsX9w2- z4UR86ZAkhF@Aarq&oZ|jswf=l)na>2J-h26=~g{it;XN>L7k|zqqVZ-m!z6!Tq&~z1oyO(!B}sF`~BjjXx$eF;tP-^Pj%-^D;r8D{2Tf zAL}LBu9T@W7Lqcf!WQJw%OtAH6MFX4>*00#pgw2!TY85j(-D8XF6Ic6eyMP~N#1){ zDzN>tgzG2>4T`AAecpo&B z!id{-_@@MS;)w41=SiA^3%ERh_QjcI1v0G`6>x>x03&2x_ z;}~#s+&<{2JvVJ1RAP|+=+QrKvwe+-9mvrj(CS&`zZEIhXfrt)I!NBKrl&L%b$$8) zB8YYiFZXc%cz!`OdOUFy7UoepGa6|qaO)+h?lcS`QR0aWqZKL2p4|u4El24fBNtA-n{2=K2neGd(H;Al)leFP z^o$+{%s%J9=YyiK+sq;Z&LX)kD_x=%`Xnn&A*0_15aX1&?;edFD0}7^K1S||gbm9J+R2>oyd56Fueb((X@Kpt|AM#r9N4a#&+a2M z{dihD@=$Y`!PjY@AhB(ynRfK{xK)XOpwY#{OX@Y#(T>$`bw*O;rAO<(LaoR4LEiYe zPeYd)A`B6zYcvmfqJ&iH`8O%_v|tbi{fdN1{MHhigt;CTv=0h`o}&C6H2fQJ%yaa{ zp+r>9AQ}9|$fr%hy?)34-=S$*x z+w2i}w;#xsGJzW}%cndy^G;Y|{_4?6t9_Rjv0y1wsromZZ(J+y{L>&fyil znF-ku-KsXGS5jqquzYNJdLhX^UVSq#_|_>7ooBO_NybBU6+wMc4{m5|tR%F&@Qk*W zod!(Lf4$yYa7WNC6vIFdnJp>b{N3D2X8f4Sg? z%#P%#ef+9Q>!63<&V{_C_YvVTXkKb`X>j9It7XewO{v1cREs~hfBpzZ<)_&kmay@# z&^~SduJ1nP2w)dKW`8u8exlYMQc3Sv?xME#(Gr9zwSR;soKtE{{;7zp>t9{Jlcm2a z1pf123)hE$4iWaC9y!leWxWtUblHijUyR~5SoUTFR5Gx1LeAS}6seSH=iPB3LnkDJ>>|(yPdH}I8dI9HfTMiiWo*FnL$Tvz97JX`#Wc!!3cov3SDxM7_*Hz@o1 zsw!*=P9C7Z7o+w;k?XpZssrqkw-C0qQx#S9#)60;U$qTgnQ*b&D+0boLoe=#lv@uT zoK(938%XKK&Kzkt`}-+FhD}g$aQ+eJu@8#I(kr_Mkwb}C&m|n$aAQ7<>3%n0t=%@C z{V7)?v-9fOKBzX6DaK1~KNAq(UC7PCnVjry{M;47Ghy9rtR&EA7a)|PkDXU6Jbl!8 zbpyJXkFOc7N_s>R=uov|r3WB=gNzc0IWCFke<h#n?)IguKXY1FmX2<}8{yQ31+OM|XJ=xuSiha$be;4jc>afEt z?ySc^qkd!gusyPD0>vkEHmR!ebj%{854CJNgmUFCN%kD;efu`<)fpL$8((CckkG-i@Pzz<-Sd6I+FGnSXi%_I`x2$>Qg;C*Xdf_ zkI;I(Ko8vPf_&Ll!h?+zWHUs&su5FX-p`3i%A23R>dEnC^Tw|eMh>O3vnZH7@QjE< zn6XBhw9KVFBLO5O%6)9sQTSw(Rf5)$2RFRtl0Lqu0n$Bv z5Q;f;_Ia5|e1u<4oe;_lyk4JmECDc{08iy95r`e=>0tklT%Y5EKSVYWWlv)% z*N}0NQ=)zuv-%NidAN+vhim6x0q#Q24t};da`r6nt}5*%G_?Z>4zj*IokfIzN0!#+ zoOPviMq>5`*SuywIVVOSB46Fk(7DR_RC;jiK&=Ak2{p4DIh(R{VIR~uO`g0T0&uFE z)zEGoVt)`N2pA&gr3ZQ*;Oyhnf8XF;8m3;IYrQ1b7+ds$q`%D_c4c82QMJ-&ppi{% zgV2xS$ui!=HN&2=S?4}q$`f^RRXq93F|b5{lWPCsnPh`2e6@50Z5MZietI7y6#{*_ z*Y~>>Z4T$My;HP$rTtm&1&;+UteW2EZ@ag@ks0%oNIqauI$*u=v5iyH5HMPk8s2|= zVJY~*UOWdtEa^-xSDFaWa!UUm|h!J=}!^2%CjX};}_$h4fAf2&#y?= z{T$NozT>ykBh{jnA{n5~EZ<%V<;qok^}PSt5P~KEU5q5u_qc>nZtg0BBMpsKn!>II z0Ps*J^Zb5*vE@>t$7jCPjr7+uh=*?36+i196<%&@N?u*CD?%nsX#Xj1X(s2#!eCX% zDGamz1X(P_W*;>7`>?4XFP_p2GKPK&D*ZnrKL0^+=AaVTR{=|dn+1f&_!ZI6kQq;TfywGGBB$8ohxeJmK%WcTa#$ zSh6C!Q>V6RORSiX@)Wu{tq0IvXOi50duFBv8($%m!{=qpC0y^|j3@amCXnkAUEv1{(IN_RB2dL<-f_(hpvoS92E%s(zNz2W|+Wr zc0m%*nz<-#gxw{k8C{hc+BKVoideI$Pv$87Y7v4E`nsDL&Yozys#KfvZC)%V?}GAt zOJP15PL+%zH-tllv0Wl~1QI;!Cyk|9Fdtk+AAf?Zz#xD5o=4S(rJ^^t^qiIh&*_S% z^d%uLJu6AZ7sc1T$@T|aivR7GbJ2jbW*U$e=Y9wH zLXQFaVe-+EG=q(o#OSg=L&Y&CF@OfAT*4lt6KF?SWq?;8cMO`WpXpnT8oy4tH5|EB z{TO|-VA9EAdUNAKh1^StDEB$h9~O7x3HARb&&{zbK!nG!kJ0t;6lX|muDG4DvDd)T zrwTYulSl-IVx`V$Q0-e@k*wCLo`MD1|rVD!Xy~wuQzUb%uwJ8J4 zt19DmWO#sY@bqcer|Xi|-+h*R9biY#B?*B9ZhYcRLURDdAQ0I}-v?RxGPtp%i~oE8 zln5B&U03ld!_eC)OdI<{`=D)MQwVkx4#IYS0ESeI9nHdYLFj3@3j3fR(kXNoDCK(f z-WmHZiu)juYrRlr4-ldM2lCfq7hl7Fc>%>iY^Bmz>~3@+Tf&|xuF6gvr||>Pj~7(% zKIrrW#|}%J!pBs@(N8(Cj+Cw+-Ur23sA5^Hrihxp&8=3EeKI($P zl5j~O)iKZpEK{6sz)fxx-fQq9eGwZo-Pm|n536UOZ8L82>s-B zyr7XPaTzPkJi)365x&y%ea%GTYk@B(^m!{)dtD$hki}7Nn&j!>@B` zketXkx-*1wUHeTbO`Y{P{`EW`QztX_G`N}>JHfGJU_~ya&bhF%erv{O7&!HOXm^tA zr4o5XtaJ|bm5}3JO*TAiu21|g{&kY298&1)dTa`fsD0z7EaVI7>6#i!t z006iw{$mo%H-O!>R$+?nusb1ufc?s}sm3h>?1)-(;~PUGmT(eu0?{d<0TDho=gXFc zx!AkErfYf*_6SUDrY=jwJ>CAG+ZuD%C@+lby@?JAQ1yc5D&zsi!TD8eEppx7n^aa0 zg$SP-1kVcH%9{EuJ$Gq)77QQ*Hl?3j(=xA9TEqEf5ai90vg3Zy5c$J1mR_6G06n zmsbB13&Wwa5ZWWwON<0c79G@{CT!J(m;4PEgWI5txJZl78Y&6V^(#}yqR)GZ*706& zkl|Is{XG`m@+LVx#!;N*t`T)GKm|hN2NR>8@hYPgj*P1{^+*t1A+V}a{gv#^`JO

    k+pUF9jIMcN#_3o^mGdlKI zxC8I*cN<-qD?7Y;W{kHQ;Kru)K87FDgg(T$1V}0^^e;^J9N|aFkgO5|&4nSJ$ zR}z#R=)=`%B*eIhSzv37{Z{t_SP(LfHk815hL?YJzJ*e}nY{WvA7;t!Pe6CME}gkp z{vg-?>aIr-mpj)U>o4(c12jX)K4_GP>4eae78C)b`;kI-!cwji_KY;Y07~_6TQ8Og ztl57eT@~QhzsCPU!Qb++6+>h9LAx!5Y;gipT%VmHL<6k9-ask=BP8w_wg!p6Vhv4>C) z;;*br{Q0@IPY@OEes2|C-aG&7o~hKizOdXDhBbQvau9uzI0TQt9AXu9oSR(*9|(Oo z(2I<5ni@9?{U)kjuLXm_=DUTqmq)ld6FIGZ9EjRzS7DsMcG(!oQB#)q`pIW#W2~5E zs#8)$y{)k00(NG^4PlG=HPK>y;8cg+!Q!Zi>>b01^NSh>A3gnh_Iz{Y8$iw}cH(NQ zQ_zpzpwCe=li8XUOCNNGgp6U`)AxoDg{LEhs%vH|uFYR~b-HH%zwM4_T(+WRW)l7iFbE^Q>(4 zam*xJnT}V9>d^mr=x44-4drXEdXVqp+OWy=jfsL>LcrLFQ7tz?HjP7K3!L%MSaNfZ z&h@t}_Jm=O*#}}fJ@1E0b%4henjS{V7Em{luWBMD z@h}i$KY)NB!-*Q_!&bQjYA6`2X@!0}X)gsslAZ z1jbpR6qeJ1RyBTYI|%Ouo`aE8`ZZ??2XOUJvJFE6)z(Q6W_kuK`2-_*nBtS|Q0$up zlc+a81w`F$(F@pTN!KSmo;?ZtI4lAh6wuwXGUX9-_}|wQwae5#id!i?@2nmG&zjRP ztk*zA(utU8j4}Es9g~HN_D7%0cm9Id34Msxd*BqL5|_4m(nH_2>=+le)cQ{RtaYG^ zqKh0iEHiV8EeZge8AicJ?=Wg%Z7FFZ<#xNgXO*+h$egYgG@xHaa95two2=IRb! z@K=H{?Am0*IoO3p%F;Zc3P40}TY(>vu8$g$dcN35pC9KEO#h&I;w2X?KAx#Uhb{rJ zqm+L2gLAzEaC->VV2=2`sTY>*-8T6PF0{H=OnZ^#_-ynL<(8}O_Kfbh-<{(6EEg{A zvQGooS#qqGmHQx{@Njst0hTOqhh*$WV2f0qM;>rnz2{pq{CZ`}u>Oh=?3_nDcbwq) zimlBTCsN0h)lP7&wW@AwE@GZCOwq=yZd$Y{wd*Vx{3AI?J;{&objx**D}3w1;=4jb zpn?uanS&3~Qx>PAf@R3pI_1H26Cn_~{qJ%iW5t`#`||?2R+@N}@j1GG%EWBu2k$2X&YVMmPtjkGk9W>TK%C*#5NpCG(^` zllQvmn-a*2H;vYOlub$W-FQ`6J zAMG2|iWzZf!Z89QR2IJhBnHZ6!QelL%qn}0V+FQQJnhRPEBrV9HT6x zh7rrwjdaR|VwIo1pHPade|b$QDpaRJ-=Zco7qI;tsrF>!yr0t02s~g43euZg>91a| z>KE${>#8Y_`d8eIa)W49&Kq6sIWbqLP)eweJoy;hd?ek>>-L*-O1UpMr8xPIZhyIw`sZgbp0hs8qzy4W zyEb?fj^Y64L#bGf8bQM+sSXZPN?WFtJSZ$F9?e9xbAs5R>64-p_B z!Z4CM@`~$qNdF@E*h`hjj{&`-GLPFnhDR@UG_S=on<$#y&{>=rt4J4|FbfpfJQD4k zom`Hp`29$+SHA<23uBqc99e2>ZeyH#uuZxzopS6-k=_k>+Btg&Qn2gfmGd%RR0iT{E7$m)Fc8)#pzSd!(Al%XuZ~zlk^~ByL#7^{ej|y z_-|t;z*3KN-IGZQA`=ma=F0EQO~GmBjJ~DnhxubX9~GZ}&z1+gh=i)7#eIK%ofL74sthmiyQW(P(uI>ehy z@gWZ8?Mc(`(_$N0Zv)v%D08-0MJo#zF*{%fX5lnaXfVK$B}0B~m&wfEG=khAL25bK zp=v-)yQn2AlWhPbMmd4qQ|rsb9D$fVd-1Q8nMue7Ij@D8b^qn`?;RP4Y_kry_Hl;Z zO8bYHS#uAv8VKtAvtp&#T>G{AljjqAPDFfyL&ujX6PLdHTM@5dvzh}LQ^Z&D#bGy6R>}T-b zZ9(JM{Xos8wE#N{G-tb^PZkaX@)5x_BM#kyc7^V^hSEZQe`wfMcbX zF3enLL|_1$p|v$*V`EZoBUei!+u+4^%FOgCK8es!+6|dmTFT8K26z*z8Y<{Zy(W+> z*l?&tbia+GFw}nXw=Kf3*3)J5lA~+>jEn}?TG zh>^n#?&0D>2r(uK6CFA~rSr$@N5HEKQtcC(%p3lVPEK+6wvT^2+}q?Oa5SobVXoNq zwok1#Olzm(EWoc6=Dp7tMLMI~M22qTNfK)q)3tLQx&#+q6$mT&GPl-hnIm6%{Nr|y zH#XpSW1;%4v%Gq$m2`c)?P#xW($ZCfYUo*gc7Nlu`f)(<{)X)ROnV1H8LC zc%y&p3BN8iCasFVysoItIn>(hh=?;g^>r?t;39)#u=Vmd5TXWkrrb&crV zhvU(xdkz0NbssEva}_sxXy)d-nwH$9OUWc5=Z^`>UH8@e#UvCggO;QrOtH)^jPiv0 zn2|BXmPFnt4Q4|gol?fxn3v705@va)HWPZj<=Gi{2WR@{yMPiKzk`G-Fw=mZt*`#F zX-VG>DrV}urS?t7>9Dxl`>J%_7qmT}|h&&V! zf;9HX=M`edQM5QcrEI0ZV)B>IW`)k&i%BC~qpi1*DlOyi@bUiKt76YRI{X{ypv*0D zHB)T4^I>IX3>vlwq%aOM!->X$gv;#ChMM}?=w;`$4ffcFq-v1?2C&_h3+79coR1x@fP+&j(lG^KvFxVDadxZxFkqB7R!h%$ z*ofLK>jV`~T;3A9@hxh>tdnoxAs@MrdgiaKbtjAZ3jI0+Kero`ZXJY(e080HpL(PF zb=~Q~xXo#Wsk7t)?=GdVSMJ;DkIWbrm~zz@e(66W^}G%VTHiUL{=O6NdKrTR(^dSQ96Ul zurt{rO@*ua&g6JzvgE#b3SR8(df!j3ppKB|D2T2M_$)?)g+mAF0ycI5EiFiEPI{Wd>@i|$$wEVr#d^z-Mw2!RP_K+oEiGiz>>L(q-C=7vQgprwi z?l{Jt43+qi?fnO-?fP=q#!zv5k2y#)f{s zfPQoBG{Q-Ob8GGDvmV9WjEtY*VwvSKZNP z5iRJq=Xd$p7xl(hujf~%Q{8%e0&d$1eR%cfyIv&;4r}#okJ~H`$~Nt-eSB+aud$>l z-GXY>H7zesX~>L4YP36eipR|SXpIkzA0}tP8`PezZ@_~6UiM3UyTaGU7G`w--WyKR zg=uIFRAEX?-gc|CtlWy^u@mtu(!@P8r&?e{_-^}Z`fCI&O`gC^?bX=g(qREiA(RM zfFejdzsV8b4pqwc%r0!A76j~P0FSAM3;QfUTtbBXgkoRJ%{zbTo*T{`Z}4!4cz^vs z;uSsw!vr%9l`v674fL}iRLv(kLnqYg!-p*t8^){hGB32pQ!7d1Rk=9{IRGqYc>Zg5`XimG(5;3;mOU~Fp4OTfQ z&*x>#oZMC2ghs;ae>|ZgN%o1DBNmHr2-BF#`@Gl%ag5MQi@OtA<~n~WRmX9?-(30( z|5s|%GqJO0`3CSl@I-jMeGe{C+A6ADlw==KLqE0CBRw7KD0i(}f7os{IedYQp^k{G0 zE?y1Wk=l9n)%?KAt^-dO(3(KUbEC&7(-9Tu5Q@;!2GD({>vJyfnb8sZeCixgT|eV7?v zCH%1lR~x&bUUG@{@gQlmbmS&>eHkF^*my{_fS+qe4Mjp-6#mT3&qQIAsmRSw zU41+EsOegki-uR+k^IMf?IPobe}W!5<)I4Mf&uxy?2~jvcnR|?)gq?*E39`mGxAe@ zS#epmb4R?4S0pT;WiCK|?eUR~NcY5pLCsw6+mADl>#pDK^^RpXS0#hlyE#pZ7TKO&W$ya+xyem+WXz@0IobEB-^+`#O0oVcMrL{R zY3FwoY?8WvJf?Z+O*g=hj7atw zE@4j$lTvtArAJ~-r0LZTpR4?XU#nMow$CVLBdo8k5?z0M&$IhpC=U~jI(I_8tiuF5v zn`*dJ8q+QlfFxG)qR+skacaQx;2{_ddL{%~sc53R)>8P0}T zrWvac>ZAD-U#~*u2@Ub;I7*j1SnXc=rFTz`%vyK>RPH1Wpr$2A=iHMjdsm1{XXm$M zhb086i-@dp|KGSL@>2A#h(&N56ntkQWX@R1*W@H+J)FiPDIwcm3VaeW`|DIl|_K`j5MV=P$0F z?%e1a2BDu)fIJ}=<07hYpqmEle!+^Oo;n41sT;2cXSZF*sc)(3_`KEc^-3N1vx8V&sx^!d%#M3+aSxsp@Uf z-Z65z+p2oLwAGfWr;9&yu^^WtV;@AMC6zNz(s4i}><~g;8Q0yKfm3Vs-=+tCSIRh^ zOAOe+g|vM*Y^LTJ{R8yaJPN3Z@V1}RcVHD@j`_01-_k{wz#Mdwoz4d)hILaRkKFox zpfr$GuUq9->MG{s8{)M6%?Vj&zq5;E26Cbl^)>kW=-$j_0nZ#MXXBDFL8(aV!)bH+X9BZ_wvn~(mySN;HH>UBB2t`F*& zd12g01#Qt7$0|nL^+L30zOXtaF#A0=!gH&C%6ak={?3m$&6_LgqjgX6*+p+^(? zTxm@OW+kig3K64Zl~H7%rh-9)g*d6h-1!psHJ!^`s^Tgq;n(RnS^=f>Dc%PX1F5Te zO(Y!lb5o!mI4J2@i||$Z#mE=>`1a<}7sCQ+XTN!-ghTvwq$LdEHh-~g++@UuTkqu2|MV}tR zYS=L_SL->3G1HKGLO*V1iKO2*jQnqGoq0Tz@Av;nb`pvZBWto{FWa=)nk1BMFv)H# zWgQu&$i5SbnC#iZAlr;xsAS)l8Bt`JNw_m(KEM0@_xJagf6U`CbKlo>u5-Q4^Sn;C zaBwH-LlF=_jMvpDHX%}zEqbU989)0nM$SmIw)`n6(2=)6kE+DEwHPE=XIxu(NBOxk zN4lV`l%H3D7teXh#)tQ`o6MjF)we?lu31jMllk>a3?Cy&8_ie~u8$>ny-C@d)?R@s%Tx08|`$eFBcYNMod0LvFWVEcQN0IOig{n4-R^h&*@ z_(S)R3h1qfae@$?S*Dnn8jTRfpI$^i9xD|VtNU;>ykv7`S|36mpTsnHmu2X*GN#Xy ztdrvS3~gvVvs{&{7ZKZoR7X$%o1=_81VYxJ(QQUA(ULsT-UsbrZ}Xkn2pMoi%~gAN zu|2Lup!O(7Mc`~v-(i@rtyD+~WUTEU?C>g3Md4oy>B?s!Vf#{fens$2YJ^o);7+de zFyEQL9b{#&B`S=^+gyo6{1}36O_YYi|4$JH!a|=EQ#wheUv!||0|=Cb)s{%zur(gm zUv^S6)>Xu0sLhO0&E9UEkAip2noez^$<@~6^F4+G#2&Q>;L;kL0LYXlY1DC}I7xcg z?EsSUS+7IQ<*Vbk@Zdq?+%T%=cHdbqm0oQDC+V7mz6Qq3^F{BkB{43dtxgi?yl77V z1R&1Q!ao4V$DvMRPg=)nf)EX!NAC9=5ZfvAyNUyk4)Wr!z2Q;ibYt-L`5%M24W?TK z$1P$xjZ0P|5mH&*4PAxGu z+g32RPvRAb_@H|F;=PQ!TR*HzXhtWg;y^uxY6p~(RvZDYLnZmuX&((j1S*&MDPF_UtHeq}7Wgr4H)28n@Zpj;C zz6ur_qa&2_`1EIoc}&3rv!81&mcGdgexZNJb>ZFS1?}bTYk~pQeOV8jdciYQU>nv?q7}irj**J&#FR9jM=qVBMM^M0xyiU zs3*^K`CM7(%`uTv^BQ}fCBspfsh|FmK>O_v<| zN{PlqG)q_!3WT!W4HOyuSd#m&NDmHcX=rJl>>YYC9=f5YKdCLCx$`ldBO^c+{DwMo zHn6qg(R-lB6hnI0R+CJKdm?}M+coCU)GBl%BXsQXDHCxmwd&K93t0^pXp@ZV%K}ic z8BaV`SRI?9Et?!6x*4Gcgn0Q#PTUcW$Y+h6^T}SHy_=?xS!DFQ_D%@j{O;3LL5JSO z2F1c?thC+`rp5BdEn2>Tf7|pZ8sw(bD<# z|2WY|KY-YCg>5olP-f5C6biQ)4)S&FETfsfK4TBKo4ms7kwoe^0 zJ;klRLj*v?2)yH=9Vz*PDF@0I0Pt5HFXo1qzM&6ijE*EA;iYX&?dH$?l=qy{{TaX9 zyAYcUbg>rX5dhhoSVeW6d`^Rt*RGwJy+`qdvzx)sZ+z;U;CbiT++@HcCn+>JaJ70z zTvEEqnJ5~&{fLsytxfHQaz(J(1LfxA;G8f_sb*9;zSybQeqm6$vu3{NqXFAz-J*Wf znk37o-v~g~sE6~$$MO`sLT7T{Vaxp2cKccHnZ2Fe-P)bqpAr*jzl$@OAK}3XlAq)y zFP~~Ri^bL>qV>euV8VH;M704{$ycebVF64 zA&&NOks)lOlze-2pdt6mb3zX^#;)XflG?)7skNI;X$S+g;^B;4vwVkQ!^d-6<|ad@ znmewOXNWUVJ9XPM=~ZPG;K}sXw21tNzbC39L9H!u+JpSB%v;auL|djYHEZYYe0a}; z#Sk;0uGnqc4lR%wT_6k6c{i2tOVX({31Q6K@=s61wel<+UMObUoy#y(qa6IF5h|#G zy8Uj`qFq*wg3JjQwrFS7(If>Nt=>B^+;T-I^MyWUXm7+ZCRw+P-4MI&M}?wUn^ldc^K(Dj%+Aa!G~iBO;JlhDHSj3dh4$&ffPC8&O@zpE7SUaaTs}|UXsPJ; z(R&>t7;J8;5WKH6_k?TM*vL`Hi0hj8jEMn5s8SV~XVtOYTj8KC3d+%hG@VY7Y$YF* zBnZ!G!r!U8zsX#`O=F7gs@da3D*&#fza6Rl*fSlRq`*%TYvaBw#U%@i@!J{Zs&K70 zv4Jp&s#nmlXD#QHRvhXw9u}@PqFLKC;!r`b`KIR9y~4% zekwoV@t%i+;etnC1U&3 zkVX5E3C3k_&ev_q4lzeqy_1a54YBS6?slCtxCEz5Wga1PxN(nsC&OJVrz(7$% z9D1I-quo$4>?Cup;&C!h{+!hH9IF1Sc2Uh(D7r3V=srZ3k*%n^8g(dilJ_2{FQwU~ z5?5lJ@DqQnzN;@utQb!0;C_q9#N8P?9kb+`#lc;4y(B|=ae+7F;z=m@^q2Fsb>067{)k z(AUY>R}mMH-U!PHbv=E~3%+wDwxVw(ZdBX8_LaIA88F>BkibB`40Sb|%UE9M$*!X6 zq4_=j>XjQQ5|Mm?1AOsOx6pMIBxge|1q4Q@f3O; zTjd7&C~b}2jbsnp(QRHsa!N#9)On$AM-2ET|KB@qGzEmnW`>}3Q-Co5B@^yW`)Qn> zs@>heaoTLO;Z3PZ4%1h^Uip{dG_qxUu+PvWS2^~+RHc_DATD4zKT6zu&tDVE@cv?)u$W@WupaYeC=Jf*RLmz z=nNDDM~i#fjEMLKoO?E4^T$Hno_IspgbZ8#^5h1VJwoVxX!bg#&@OpGac?2}_8et)kPmdk8Ze-gE5p* z)kQK`*?tv^*hvjCM6CeDM896B$Zz}wicu2iWeDlV%Zy3y|MfAl9Dm%_I_|PqgJ+kr z7!w^Xzib5dxzz3^Fbsaz)p?JyG;RRPMbUrtAidAI8(z0Gd~AlH#O*Vq?=U`e$?V#L z|9>DHh#ac@un(|Z#m<}EILSF!g7%LAGY=-bn=Vxtop1D&kF@7Ld_q@LR1~ z@thO?mvpV#uqEi>o0Ec|IGVIfGNq9igKIljoAoFfwQI?5vi&~Td5NuWsO4*1#XOql#Fzf0`Cg;M5_u zYwk#1HB$-JU;k$9ygM+F8k0w7*CLm8aFgnoQbFPlSDB1jsco$aue#%7WR-Gcf*aox zuoby?#SW$5W4q2FquOV0%sJGKA^C0dQq_=-D%p^@2rY^_UV<`{JkVLVnbQ`W{H$E2 zktOfL+gHNqwy-(lKQ<%5GT9_KnRM}2Z||iWb}?-c5}~-GUgf9=1+x1~^u3@HSU+9z zV{0e*B<8Jm<4ARf<$U)c%8=4#74|9a+u6$@AqEyxQ_6$?V|ckkLhdoACSuQ+CD!<6 z5B$6*{I#{Uxz)v;SDR(H^`UdrC`(5j39L{1kz3X@8_6qu$A=YbAktFTeXM(I9erSW z29AU!v)D@Gviu(RF+0EIe&_XGIm3KCnH1>LwC~=x2gtN^pCE1lZ{MSI`hCD^yD+d| z+dLIhc9`qhTE1&V&$;+Bxkp&^t%uJ zI$AGx`t&e#Vob^rxi>7=brVHc9EhN8|O`ft(nE+ z#Xn8sb_@p{KXuT*r2P(XuSP%Ca=U7 zQ;uG_kPjQav_0Bd>2sR2ee^b4mi5dHBdIfn1EaU?-yv{VP~KCw>;Hlg!i(sYVN2o2 zoFm9Q`JZO6AVCYO>0PV>;Pj%w=UETb57K#`v}6@fvnB* ztx*_3`kc zEy*mgtKqiJKFU_@Q|}Wd%+Iu^^-Es0RqddaP`>)7MNAK=at@6U$XlH|kmuHN z81qw5Qc4>=*F2N`=il`BjfLV|u0-fFyTf^VR>M{!VWgKi-pz(JPF4WzmjP{MjVz|7$HK1|JU?T7`kXjYp z)Qq*7lxQ;9(Br+A#HincS0rm~>yfp8gZ_{dL@!1d6~pO~V>}%wf1+kOu1+-!Gbz}b zcS3qPb1~ZfSlp^F+dDLoI>z>|%n5db!BOy41iuFu4GyTOpsL#w!g9>*=D>e9-%rU= zS;;!%+wOGOJ?)HZap$*^%5xe1Iv5?kJt@E?68JWca|W<6&CgR(gM3^o7WC_Q^4*_Z z@8ebVU241C^Mc*EtJVaHOpRJ2S!I!jkE|WkEb*PK6Fz?K9gUiq4t<|>lZVt)KhqyC zh+M`za#Y)&iG!}@f{#W6PKcX7Bv2C9&i!;~iB{U^aNs~}e|ntnp0qHc^v;n<$X+V1 zrH`;RuiwPl=Lg6_d(A{?Iu!a_B1R0gL3N~@?_BGy+|sVRE0M9ikke*T6lc1@{L)tH z3_O4j=JrB=Y=*9j79h7o2O=(ls(bm1zsf9L*0+WSAGJ@C5~g=CYQwV`YCf{4g_*r) z(+vk^^OlF0F_FHiN}4P1>`7qQ?BB2V@HJ-?_Uws@HoJrG%xDclm_y`HgfhLqfC(5R zV~K?BuV#Y?K3!2F&l!V@Fo9UbT$yU~vn;JC8_OO)9{-Z!I(OCIST}e7ME>wD@VMpZ zKhIYUkj`|RXj)>jZ=y8kyPw+R(Tc8vK=E-yq0uWp1t5Ht@|H?CwA^9 z%qJM+%agHOKDFvYPEik{n?C6NHgUa{qG`GiRllmFR-lsm6(sv^lAU|4j@RJ8$Zd*Y zVVV7)=LcKQw&;r7Z|?EyHjA(;-0KhM+x)VU>%UXqH9-LxY}eOY&?E_NWS^bsUh=NR zzV)QyUu7_Z^f0)V`Bl{*_}JNAP8I$8t;jGYSNZZj{CO&ke}U*&2QFyE9UMc-@jH+=5il~ujh#YkW}$2fzMJ={HchnLvIGfmNi){-H8 z0>YCM3ri-*4|BtK;S6~U?ixW}MF2=uEHGr$yI$K!%Gn7Zn-NpH^4$+(5$87PdO#@h z8Tab|RK)J`&z%s9`)LZ7cO?2Q=p-*J1-~%a=82ufbZg={|5LS{NLXRoBg*d~uS0JA z3e9&Z-e2G1;Ar?yJ0 z9p6=|vshP3*8~qPkg8@_98gy2dh!jv@tz&6NDZK!qMS}QsRFR~$(jkvkQNH>)Iqql znS-u@;^cG>udel(#zzd&(`WLv8&T@Y$6AeB7TP#P~J{+J)3NV^A~dcpJ&eTinORtn9Q#Desw*+@_{qskm# zX6S6ql19SYuRFa~&&e2!cMcT)FhFd>V}X}cMX*4gUxt!}QMji&S@H`}n1^JL#=^|i zI3tn2F2xwhUA%NY?N*G+g*%q-DlRZE-C8?rIC--ZSi=d&t{wGgj0ZtmFz$=o&u2@2 z&Q5%3o4vjNqmWznh;igeqpmE~En+A_7vDKfJ5M!1^Hbc){g46Sdd>MH7(CuN@>G#0 zM{(vWWwHC$_W7y3`VO8{Mw&d_)HEnngl0*RX$mg_VsGK6ly(*PNV_YKa53X<^4vvt z>dt9WstwFG4t7lld~5WY znDDWuRr2D?ek-7YdXIJqxVZ+2UEx7nlWHyl`+aSt_&lfL+hKl(leQM_@SOM`)@SLY zn_#i=8Y?SA6~%LCOg7r?t^^Q@HUBRI)T!312{%giFbv3pTG* zNw9TVEM>S=6Z!koacY0vY;hOlLc|p~+LXeY8gbcE(@eVhxGo0wubL&NcErz5ufyzm z_S9}Buf29wB+)m$OZ+>(%=fOQ|D9H2R&D{?RnOv;N?8h2WzA?6Qe4 z^({XnG+=0JrD>*14{A={DZos#z8>ijs8pAgY!}#JNx8`&r8II085B$kISGIXm4|o4tA+$0^<4nP3*Q zUQ$xL*wNtoXmMMWjdeT+(zLf-?}6t**3HYV_J5=>kIenTyT|PaQtuO`Z5%H*{EtDc zx;8y)OLXc!JHziS2ySnCvlJG>Qb-vgKEjcaDnPK!w{~9#4U^N+3S@|WK&To67WJlt z!POWdv4tVu*Auaz9A?(q(_DbeFyhu_S9YQ6uV0Hoci0I4DIh75O{jK-1F-BPXcXO4GXYm+!+o_H>xTZ z8xrG7f*!xRCer&N+<>@}w8Gv2U){lwWMlOu0X1s?Q_2}T8T7g70-UDja zI}dEJ!QUCL_xnT0lZ3`J`XC?82S9qwr{sjx*u)=r6lrA(^cWTpnbKP&ZCsxik^3bB ztt3Zi%2Wtinw;800E?@QJ2Ocgh-x5&B-NIGvYy}nHLG~Gn%vLz)2s#860%g}%D*i;uzfpWUHLsh-(?Ayi4LK2%#5-*i*`CxmeUPeuf75?Dz|o^E$vUn7+9RLn z;wOyrzj~(}Y+SC&ey{o*A&3?vxAd4=k{oM@?Mwc!rI|5F5+m+-OkKm_kqD{zpb=@kJsF`lO06BkYCJuMa zPf79dghpajEAbROv^cOWk`nXNavkRGyKMiKML9p>BIxc!hk~ww1<>&z$o!M#-*5^~ z?;db7rAoA8Pcu3c5*xT9r+d>&D8_d^92z(z=P&t&Z6m3g;CY<677_!tTEPjaA@s|j zAJh`k3~+WuuFeK734O~5>@@TY5;fPDtF?Mab^7DIU>3wK_4u7P-?hj3*PFi+R_lYI zTpc1(`U-f8QJ<&tK4#ByO8uC(i>t2$YUqflaaq5?vcUH>+rh|e@DFeW88i7RrKXnp0- zT>Ly{NI(m66CNJh1jz?v-%ZotfqgX>EY{}VTQi2e3 zBJi~48t!373sqOvSW>wSrEX>{lh@D}DoHhFHZeaiF&4tZt8Av!4RA;U?9uDlaD ztXr51NQocLF{GFrBy#&->2h4)`O6b`c0(NBm^H`0b#akCS+Dih}+p-!{>&{f90Ea z@NuHh+%M#!g$aMW0z9~jWoYO}WX7x6> zIQ@@Z$4u`EnBcD#F0+B7XHuB}tKW~vzD>wFduU3BY5Dx7@HSmZDZ#0ecg0I!WBL38 z2~@XI^4Tj5g+ETg>Hry@q`*`jAgmY93gnf0=nK^QaBS@LPWv3f9lUly8#AR^scai1 z-3mve6ql?go=kU&{a1NwDlpww4-{_?=wJEIaE86?g>F#T&&7GKjJygZ#X8d+9QoVm zCqIwkn1_}xTfXW9|Cq+afS3Qsc(et5IRzSrze1i)FGA(mk!d0Gy3g+@*DTD>f0eEw zQeC(>%ygT-V-@|m_%a@98X}T)!OnPQC8vSd;+xtIN}#mo+sBu0SY1EI_!rWhIy*zMeuX2<@81D(NQ9V(OXK=73quzLo;#*w zH$S%@kdH$Tv>0b-sVa=&lKj_;{^kE+`wXa1xxq%RoIb=WyqvxynI*mm6=FP&`;&Wz zujY%2oo$}`#OgmjXWYgCQCvba+B3b%># zxpK{i^d;=V(!z4*mrUk;fjb#<{Ld7mFt4o;-KHRk*_(_x%riVG(7%idGn}EhOD`vC z=6^t;MG`}u=4U>id5`{B+bZ#h&-L{`Z?zXZS3qKbX^n6ZeHZvm(z7H*AtOX6$=iaw zrBh?k)z#Ce5+4mx%%bl14~A@`{x|qk|J~p(uO_liocpv^eB_(~$@AOTQ569u> zUoAY&y7A>Uw`L4#&TBD9K2!Jl%*Iu}BYRKTb^i61AhqS)FdLnwCLJO1-LYE4ObgzY2|6!Qr#0=E_MOhG-urath)s z>RqBJxN8b0N}IEakn-&#flv3<7`7nXS0#TkJpRu5P1}gZ4g^xZCx#*4OLav;B zPfkhOm>y8Pzj6z~!#3aVV8x@tyub7oc0NRA@gnO#)O#K@Sdpe)LH#BTCP3X&d4FNg z)>Y#OY@wbXJoXWB_}O7+i7lN*qfEpN2_bf4NZ8zh4z^hrG1I z#CA9ll`kM2)my@Tl}}1`kV#1EA+bhxy_zq#Ozh^z=GlHYG2|LgzCZKD_#ER308V}A zTuYW2?H#vAVp&HMb^?vQzbC}rH?C9Hk)FTX_GwCz>;8l_r~~H(I`G7Q$Lu# znAOFv@ZdU)`j=VS>w)y@Z->^xgjmT2*auywfb%xBg3w-!Q`TX z7PG&9`F33jp7*4(Nzr+gCqZBEH~pFq5_G7K+X?!lu&3bd?HklueMvaH|D+^S-M_De zUFMFDa`rcsECtXegyuk%K_e-w+s@PrG_EO{+$#1wP}keGoUfInequ+f&Bz{dMyGD< zq~XR&8gIsZABHgxJnIYuS2MP+`MFh7z_T?dEA1}j-f`{AfJ+ob+@4>}=Crm-y^sFp zOLv=lJBOtYnL`F%z4-aXp^gu1P_D&G{x@qxgm!){kq5FBajkt)Jyci4Cp_T2Qd|A) z0C$a%(CnR){B(tYj#HLLr=Q$=m$3G?9t@Fk^9lH)C;p)f||keK~o1mqsOVe z)6am^FBuMZ2Co=MC6}ZNQ7`8y)VvVkx^X?+lFnXFNi?T$wP>dfV3<$^BnXeAXEXis z$OYLvh-}Tun*Ny+Tj`|?iL#-xpa74(0^3;FN<~x5lLwAi>3*Y5|bfa`f`ajGd1c< zS`IDb(w{N!Cw(8AYHw&78!T423M=;B+~92*KKa`F6nY1JmwdESz6!d999omjx}fa9 zI7DnS&2xv8GjSlwo?HZndAK$f=nM&5?<1eo>K*<;C%s?YHM+qLb*-DV><5S0q(*Qc zv;ApGV}Bu>C}9N4*6n*y-+YjQEH>paa*A2FBRBU9ORjYncTS1B=C0K%afpwCDR^a3+6|~-UlTupn{>_-N#igAI}*jI`P9$OCCL?+@}m& zW}b=e2wS^GyG=EmNBQsffBcnxr&(W+`otZ9;_zg%RC(XX=m?|p41b*Bv>_+=S3PUKVBO*i zOgP@d^e;7o0rrKL_Gp)nYwt|^7@}$e#+7C+C zJAXXKTVZX2N@W9B=^U4c(3m#+;Aw=QV4oYe@QQVta1Q7YWB$S9OzP7NmA3Zuopl2O z^}4fsj>-!``A!=~T@W`p-4B$yx`9_v5^JxhWZ&4PzW&(XWOvwjamrec35l)R)x>*D zu{bU~mo&ew`)Pd1QNAU}<4pr(JarC96l)}Ft=7dvh@c{OMZQ`Mhs~9E)LCtv->)rs zqu}aOlW}eG8Ykbac;Ocb93Nzf=|mnt&i$|@Rl~=cB%4@cSXhXB9-fa+_arW~>j@lD zb?#BQcS<{UxEcuHafrJ=w0r72u{rhf%rCQTs&d~Zo&&9lNtzcOjiJk7oEMbZT{TP3 z)6$k+F&>Sw@wW31$z=_R3tUX&c~bO$r*1L*={>AEsf2p)Frb|&*d9!*9JzTwo}XOw zKZYCdg(1ZQ(Psy#Lix%TEonGQaokxc^N#vEuCJDg(O;#)be#t=Et#J?xkrdF9SmlkGh!QvE!=wE}Li7ux1hdlKu{Zp8}md$=7 z{&4xmwUv3D!#@<|?IUu_YHA;+a`ds*Kps~3*Y>d&;BFn#GO6JjBTtZ6qLr_(H<%j! z{HiDLg-i4w;%$Uf#0o+H4I}YzETeTg+sVo?O{OArKAlqp*Y-q<@mjD~MSe-2XazTy zGJG_y{=1B8CwWBd9J}sHO$S&sV3q__NIU}NuZVyjW%O>naB}uLjl52QId#fmjp6>i z<~;w!x2IrX&bl02VfziYB4zkyctgR@-2v|UL|gz0J5<2i%PExLgBHZu-4_`jcHISDZmr3=GwrBP?ziIfG$DV@Sv8_UDcbw$Xq$>?pb&!M$DAxmU zZYm0EFZ`&av0Jq4H|OK}tDo?nw74ie9NcO57Psfo{JmgP{b*c>m^g606g9f4^{PX! z7#Qr;q$^?{B1G-_&*VF6*BeCY;9|mW*+j+^=y)upnOu1ABNB51&xQ6Tn+;-O1u*7AL8Mpp`wT_y$c5sa2Goh3U7yd6nkWQa|P5ng&WjY6h zOb^HO3HT8q{J+~yXFqoQ5bxyfw_m>P*#) zn8s_Iyr(&=$;$xnB9wLav2sk{k{j&x^pE!TNhRM}4Yn+wHLAAcg6gc0^Se);e5ldv z*19xZcW}j=0s5a7Faq(b+&rtx6-C_vBu$E7R58UftX&>?S$5(4cB0FkPw1Q5aknZX zgL58hX-(vS36dpruGa|Et17A1O_alAqN1LRfTMUvV{?=#g^OZzy2j5_a*;*3fh5DN z+AqZQ^qc zQ(oXga$8Y6*Rn0o4KeN-Ot;yA4mJUVJT1v9#2K)b`unL@u2N31E?p^O<}289y8JDu zJ;_dYdPAT!xuL%8(=Ri-LRatyKA76owI=d#nu7Ie_R8g+)AM&x72oQ@B=Y*Rk@C)7 zr>8$3x}s+SHy(XH&FuGmMGzfMCPXXg^KDM+3sTIIJ1p1i@rk9slART5I2Q8a-9t-0 z!JN)Li-g6$(K-dan#H;%iaBgQ$*RLEd+A?k1)M3CgHWLkwMvshxUG$&?UttpX@1Kx zgv01V)2q|He*VISDvOsbGOoCAFtnCMEP`OSt|#>@P<&YWJwk|fjVzl`tz4YsIgd@K zX{yndzx7yIeSdQtBAw;x>ha_9ud(&1o2NS5dcoB~sUr5EpJGJNYUGvvaJBobQ+^#< zp`}2I{1Y-9>>tc|>e8ITrZ}_1MG8=e_4pTi9r8Xx<)kKp2Y~cAvQSdW(6@uY?g}8N z!@NwdCeB$p?xpOAylU2ylyc9#?*34t|MZulmBy8*H3~qv=qNQV19A{?iX9r#HH*1E zP2UQG)FmEy6z1-@7&p7bwwj%uTWFw3!QG@Td43ZHqOaoI-^ zY?M*S)b!VUpY4?!2h-H_5%<&AK&azv^#17g16Zmxh(?+_kul~JJ$$`P)EYRy*hiJ5 z)fitx|9-CT4}8%~bVT_}>)C#JWy<9`!73&psx zO_1Mlao&E(H1xS-3;go(Qh=I5m~NMdOgqn6Ot%4EHSz27k(+=hdF_6ro`CI+QZ3ps zNG8$3L8b1lwr$Vj>_s0}RRez$$W_tUxs|gRBsLnX){Q?!S*c$#pHbiO>0>eve^F$k32u-?NRx63T^Qy{G!^W#77orY z?&S+FGu0MJy;QDx^Vw5bx9*^Sm%6&HT=!!5O>j+JP<{mLQ6P>WS!yavoT{yHC5p=T zf%+N}>|2bofJciV(JvBjqy)#6ImvA?DSt(%prQ`Rz#t_I?8gAl)njNh{^IhMzV6uH zd<*)By~C|F)B5aPl90DjHcYiSr1~TKDW}owt{9=ZQt&mdf~6NFELUKinCBlm;Aj7H zA&O!_7I7gj+<0MgTeNU7WZ+NxmHl`0@tkLq zo*L|BjrYS2mCxh2%tEn;nFA@6LZDLsoRvXjS>~kS?`Bvx)AY)s^)SIeWl9x(Hbm_^0pr zQg&X6F06c^^x*Xo|3dy)Yc97?=8 zjg|~w@&+1evU=ib)>G!cxz}5N>PYw1x4GxRevd_bJtA=u^is5<%W#M_fYBdRV^jLEB`^e&v4tem2XaUYhBIdjo z<^t^!2^%}VoToxn_m{4=nqTw@)u_g}Wh2uHTe`(%ocS_JDjwey7kfqo-HpLvBL}0* zp#HPM+`(Lfv>QwYXAXRu#w+5+Ebp|mv>96oS(~TdYw4EaQ)gRhs?# zEFk%yGa6-w0r8*M<;(TW&CR)YGW(N#Y8^tGLj=XV1)y%O85d~DwHuEHXa@`$qx_ie zdvq2c%%)?a1}}mRq&%v@7qq>bPsuBsZ>mNfo+h{|Dh_M@XvSR4Z`Zea4BrRKN6hTXGXpXu`8!oW3`!nV3@q~PzASt z(`4E}`iA1+=m26KOT>2VansZ(vB@=c>)$&}G4AEIqwZm4?eNYoAN%>whm|6uRn7RY z4eXz}KP3}1LdC#H&&&*H)NSRaM3n&$(%4J}(nB347dT>EO40fHEWnrGyyP!<3Z_&2 z_KT#qbjcGqyvIoM=D>q6W+nqpqZzAt6DUx!J0-O|O&L zQoe-qxWUlnd`Q@)Qg7Lyy@4?IN-qRC0rdQO^&Fj(TNkr^5Y(C7EkFu$ z{lu7k-EJA2cVP1a^<5=?K((56=xBv(&Mr=#i5G+c0s4fs5y(Y@Li>IX!EHOoROLj> z$H(~AP-zpbVLtyVj-hyoe~}fZPZTLLhy8#VRR=Y3lZ@zt`8?ODftlc^YHGbp>tA1O zZfa{onYOkHhUJRr#MtVlioc$BN zgnM{7ux0FuTz`kv9nX2Y&1`X4K%noN1^8mbLDKzFvOl2&(r7(^5u^o>k5a*Ug)+-8 zN6v9?T|_Mkc{gUoB4d}d#jQ_As_AukINLbP^DW!GJr$|%|2#kpa?(KO`wijLSD@L= z;6{aLdf(A}W8A?reFAClC3t*|czerZKKG>1?@RE8m>~I$=rqIb$bZdX!pSTG$klF% z2~PawVME(Y&sg9-f}T9n14BFNy`dkLJ)6fypz|GI`acKb!?DvTe$edZk{-%kyM_HW|#0 z>XM+U-}L{($F+c?+b3_EAc=Ml^}#ugk%5S0yX>r!x6El2m5CrzKQnoM3)x7yvE8+M zOE{;gK5w_c`GsWEdR?~}ARCh6Ms5N(C4F?hB7b)m2Qw9fzm!ZX3HH%}8~9wcH&pO; z)wk&}L7rZbFZxtRkv~+QicpA{pvwU*B&fh9=+KU=fpk6VclLB^@Cwn6d0Fk1Tjb`B z_>1D2(X?f=AFfcnnz|nF6~NHhdsmg&y$Gid9QgZZaw>>C48IB(GX@_v=OVccxmA}p zUDNw7-nMbD`ip>xoaueexn+a592@OCUlbLrz^j847QH zgXG&hni2ebSly7W179+GrZU%#Z1??JCjfY^fJ!!)tz3y!kYcyds@3~-X9iDTmWkmp-X`Wce zep5PjwuxQ2bVB$uqGqRBV0z;yP-1NDPSj||ro~l3XYaWFIJmq^d;Dr=3b-oC6^at!+#4j19S< ziHJx6`$)^MpcS03Xp+8$kE5R1{gzyIzk7@unWG-Im+V<&)TPf|$&gdTv(dZkxCrMS zv{9L7_HkX3^wbm0HtN`Y0xq#5o&-;I57Wb%)yfiYt%Yl%nJ zf4|;7yfePCT9!yoHKcs2D0B35RBfl=-&Vq0FK&wdZJRv)kOKF=Wy&{nJypnRzA#GO zDJG}eY)-jn<@7c!#Hj>T)u;VK7(DF%?yzjuPG6-Y`ZZ@>$ztH%3A@C`aEBwI7{nL% zq21xKboPEw2)T$hBL9r)FeUGi^c&Y4v{(8nWC|-6Ons_0+g|&{;rh8MogYoGr`^e$ z88O`s{(uPgfgbAQ+?-MGCXI62gU%6S6I5$QvkUh}f?Uh*ZwZ7O+wbW=%&}lbI7U%`~rJpMI%AN-pB0hr}0$M(;8lPxNCr zz&U+>$q)`*=};2f?A7u$K$!>&@F{aFs}tF@bM4@b6P78sj&2mb^2#MWfkwFcxlWIK zA4}z;d5o`995bI0p028EI2_{}@QwGGYmd(3b`PZA`AvTAG}GPNhoxK~De z7K7vvL;A6?!q8+5#2F)!8Gl{E&Mp!6sw~|EXYpObfyp`tFY=LpxYB2LZl5lNo`INx6B$fVRxl!)+aCYwt9wenU z|G&aRPVez((t%k_gyzIwJpov|Y52kS!{A^rQYSf5WM+f!NkVn_YMETLG=@<*M8Rzc zCE(6?eL-7P&gx?G@nUmJwp~PK&8(?@`)NTxR0UtVu}jhy(UCh$r$o3qW-`52$oFfU z4atVNcmbfi4plt(Tq);}=S+WS=%Fa|9_NSKpg%owgr)@ZwyUbB3Ippf9~_TZ^;WiVdm zBjlGfBm+dFH#W)`<_8y?! zTGwpxGFy(q-#eU&XE$fsg0mP?mWiwR(kZ)7C)Cx3Pi@-@Tw9bsYdgRkipGM3QEaoO zw)Zh4LB{OtY~_&OGnxF;ZW}Hgt?gzilL8E#-4=$%9Bf1H&+O{mKIA21?D5xz+ zB%|BfC_Gi=)^Eh-s=3ckXuzI|$oh9>Zcpl0ra z$Y>@^GttM@E&I8R$~=sVJu9jH$~ddqYsEB<_X9jlD9oW86|&t+j$OT9LE$jbMG{Y} zdi!SR;_Z#87U9lTFS0$|ZEvynsxZ`Z^hvyv`^EU{Ii!1ZcEv6|$dA)=g>$b~IY>lj z0oJK(%S>wHnmNm9E0J^6rSUT#U**EP)m?7@@e1Pwoz3iO!HCk>S4CR4w{hVX4$6p$ zzsU}=2|C%IQs$j}JVh5w+#=!8%-NTD-!xwGXRkp-u6#6nOLkuS8AX@c)R$?U)D>t5y|NwO3IyC2g&!NYvhnq9{>YZ9?qW+N#=A z%!Den5-L){=f1yx+;h*p_xH#9pPc06yx;HFdOn_yfu8V`EYvBl)f($~oFx?}zVo)T zqQ@T{^uuPGGDfB)#wOKinS}VS0r>*+I^_g53+GqD)dt_v56HN!V*Wo@YBKL_$EeWy z_W|Z(@z8Eo>9_u&___mhRv>CRk#YRXZ~XO%6BUxT&!bTy5+g*Tr37B&f1f-9dT zsmj|e;I~ro!ovxHCJKs;-V@vA0D;(H{@-Yz(D=pdF;G`frbj9(3ztBr=eaNt*lbNq z>n6>1R1BAVg-V)_y3ti9b9RcpdjtC0vW|TDCU=d>e#!u}!12esw}^*BIL73ib;|W3 zIlJiZ>e&ru$Vwlh=PeahiuJJ(Hw#JRVJ8L-_RPCF#nU=M=U)_SVded}1^CTw(sQdy z-Zp3jq)PQfSG)b^r?x{6rSk*%`k0KYFZ_~o>U z;;cuq94x_PnSXf}GTPkSSG)VgEnUxXN5__lkujVCj6izII6S0=(qswdp&FGeEcvvj z#;JVZ?c=|kSGl~5TQ)st6J6BZm>=L$d{>k$_!TghqA;C`8xfU<>qtz9Yd%hDvvmPg zUFvq~?G883T1P5O;OWEbjnz`m3oIodF^;H!2phWZNB_}?AbaoVsBL4JyM-Vc$Gu}$ zdtHQPmwD3yy^0zmV0&C@jmfi?Y`1f9Y8yhT)|YgaOm+K2DH>vDrz*?^ zBwv@93=HuNR1KQ=&TaI%!AS1)J|<1Fz*@4g_0V31KH)%cpCW^dI(qW8l5%oRkPT7W z@@sdvA1a94tq63MYjf+&j&_r~;QrF^OKslj@D3+&fAu`0X7d&-bwEcFtxj6;IzF~m zSNfyjoy5ua;c!fj`$MnT-H6+L;mq7$X^w0VbcpZWY7_|GIfCqTXy4rGbXR6dS3Vt6PYPc}e-vr{jyk zuM7QNOi^NqV6ro%c@4xx=^*g+FR2geC{0wKk0G_la}ddwf{?i02M$c}5q}6bNinYJ zozLC{O~LI?6}OyZiTQO=<#R728cqBk&wc=egf}i6E1^G1D(RPIyxO??=XVb0i8X-! zL<;Q3?E>3H69v8&3>%xzEAKjxub&qKU&+aAw7#jyVthinp{>jmtzU)=H)+8(D==)|8%N4yGnfY!Y&!j8(cUbS z=k>?G|7@Gad}gMa<-s;nJE~#g=OdPrwV#ah?5n)3Z2VjTS<@vd#WLU0a~O(7T%+5V zo;ue^HYN1QwhX+@g_@sZ;0h2mcfp*({EeTT4(0=bp{#+E|%AW$2 zY1Ms>LAT<~xye7>uHJ?FX_Wp9dz6k4@LO}VHV|+(xH32< z1WQ=mJb5sOLv+GbPF~o3Jn<{ajkw($aR;XvnmLu^dgqSZaKPv1!#l}5w3Mk&6wPzy zv|zj$vl9f&)h&iiP&uQs>TY|Bu4C~=;of{{o9zDT{<6{^)~MTbY97^K#GrpA!u96) zKRgm(44{YdPj69Dd)w!z(gn9Dttiew!Ji%^ozik2{WXKIz&%kf$+UTTrst&3k{#~) z$(c_tK0M=TZN;EtjPok-Etq?4>uC|6Q=gb!s5GkC9A8|tHL2Q4b9#^{*`V6T7|+z> z!T*K+uXMQ1)w4Vm_63=KyW;_SYzp9|WtOaxxGn!=-?#;f(zl%Qo3rZV4VvBfq3W*R zpf-@x`JF@k=Ccteb$lw@`SQ8OSvt}O#RR_#=;ljTQOWH|84eU?n=mz`b+L<;8#Hx^ZB9eS;i^e%_ zn(y&IjITj2uN47zXfySDygG2xPXRGLv0(qjNi_p*!hIwC*#mCQYog~73y$_q3--(N zi}!*(mE; zXHKovD$DGhZMU)ol9L27zqmpD4sVRyHqW|I@~i&N?JesHy2Fvx{|&MiHO2r8Uw1A? zK@qHCi80@_pTOjft_RRnIH#&D+V)op@&qbfJT>;1$-Gss%smKG-(@1A$UBq~ms10{ zdY0K$lT{O21+gg!<530{mMUkR`I_Pe(X&cPX6eK3xhR(&xuCfnRXr8r!lx{PS796n zi{?d8P9giZl?8jUKjnC>Vizt?J1wN)vVea9|m*Nc77mPuJIe5OOj6NLjet4x(N+j;#7qqL#`p7 zXOPsk2?dz2@;?&#=mug+Jg|F`Ei=DhULQm?nyluuFf6b8{+PLPfT$xzl6OhGfXP#Q zyXD*vmK|<8p%UWT%yeW7lmlqH=7m3;{9?&s3W*-i4s994GiVqzTft4Ub5?ULv&SvL z9FFtf6Xl)^;3uEDy!&QA&vB3b>0jD^01pEQcE_!x$Pd+dtCbfnKw9Qr=Rfn}@@{B% zaPZCTbp`tv2Zr|vjtvg&F5KI;^!*FOlAL{|sFEE+38^vgE+-GtKyFGl-q&*9e$=sS zNgRtYtY!_3*d6UFu#@g#JPI1KpJNsQ>#IOZfo7;%j}{l?B)qm zg{+EfKF23}$=;uR5PWlR!BBa^yR8ucB}uDEqvzpwB{1}$PnBLY{|DC4|NR*qfW#yrn-sE1+~jPE3@MHZ zCSGoU6nS-vqNnTj;DoGT9xj)a@zYm5i8aa!GKXiT=rgzAVPvf+!dS<6z@;L8;p zA4zny-$WrM)R>Gq3%_n8sHu&Wl!kTW~be=jb$IE9dgVGJmV1Q0h>uay$*4T^c; z#x+(rTI&2qJkqWX4YCR}=|jyS}H7VI8aB6x9{lKLS9j zR{HB|9si23#6#k*Z6_G_6Ld41ugjZ8Q#%4t=-sB+{fCW`ah7@G@>m~d9OK;?EeoRE zN{@w;kMm_MTRe)55^JC2qf~|!DgFHOEGyKyJ(u7ueoCM1>~rXzHfb|=oM-BF0fL;z zkQa%A$MGw=lo1m5poZg2R+YEh8%(lC$X<=xj7nK!!cyvpr@vIxnqPgMRmbY$hE~<0MihjjY0jEd~M~CiwC6?C66{9i-T|Z1( z{VPfN_VriFiU0F?&t>livjXcJUq6TuhI~A&(#1Fc_}pbjH3Ke+{j+l?QoEF<0^D^w zf>HEfNl>@n=7iIM*3iS~?B#PA#7`h$OK}xDFEaqNJZ(QMe_cBh70A|Io>oLn0tza9Cm-Ig!DP1RSQNqov~ zLy#|F!CW31kY1&4IdA81opZ3mmnMw!^cq$4tO_9jD4>}r!)t;yp7tT~|mDB4{ zxq(TqZ0Cn0o*nCD;iVFOZ=}qdr@uhg-v4Yt;Up%K%ZZ9Z&au|#H>aC@t?`R?h%VkX z`V+iB*!%Q9>KQmA$i=R{>ZwMf3ttp`7jGO2ReXD5_OTOu>T*8eeARkEvxq`#Ti}S&NNC+<+D1|7RR^ zX{-C;R#2o#x{1i%pX0}n54DeQQCyPomtqD6`7^I7tLGNK7XB2j10&V#*q`eA6sVA7 z;yC;6L&OKWX^@bH|ID++ciGpc}WmwR~N?zpCV=E;n5(I!S*8)Io{--+|&R z7eIYA<2qkG?-nB|yptmRsH^toJI*i))_Hf#B;*k~oBqS9?OI~n5*@rsAW4+8#7)MX z)t?K&bn%!dsw1(Zzd&W~cVz!)Fd;P2`^5`im(KcJH{o?m@T<>1oM`z!+!Wk+Eb2=0 z>~hu!+5Zfyie;i84N2BR;5e)Hvx3UcD?Uoz%QD$!80?4ncHDt>sbdj_TEJMKT5da=Z8y9*H53O z$rLc_iMeFjv@O!y22xml0MS`z;68r?yjjw+T*vhO`l(-MHbS~!@7D*PjQWJ0yu82` z@rfc@UK_X9twC$3VgM~;4+rt-$ZY*bL#v2y~hDD?I(T>Yr8#uwl zJXksJ9IfBw-xeEhQJD_R#2QU9^{jBCg($=Ymmzi+halI4jqgr32fUR>Un-7yEEXX+ zugYs3!r3U@>k;U6{;g)v0=fVqcXhN=g4LP}smhcv{F%i-5SNn^vQNHZu{OV0t@4$- zoJDrK`9|YFIuoClCRz!%dbj@^i z1`(x3i|Jm*%RPFK$7Pw%MDHK*luo^e_#7?9Q9t&;ug*{ z$J?!FaP3ZcQ{($T?5b-mH=9O(-MnDU8vMMHaYOSBst3sBJNkak-8qd=S{7C}BRy)F zYr*LhO**b@692i*5*b^Y_bvQK!uLXc4)u|)Q{D5Q%Xa{ZrTS|50G5GbVLsPs zNwC6XIV$b<@zIZ5c2#HomfiESHF;@HF~Bfs*Oo?(Mq?#PhxV-EFOvCuM#HL$Ab+2o z)8{z23C3tcx?CmoYY?4oSD*BiCz#v5m)_#aPf^3yk&2ghesAov!dZX@SX@88F1B3> z?mkl;$TAV;N!T8BGnYcY<@esLmAkOGq*Y`z|2W5;eO-F8D!({-8Vp2;2+mFsvf4^1 z$kpAk81(L=trm=LxsQh;8?w9<5a}3gM5DgzD}8<-E_!|Bismq~@O*g%%|glO%{55u zzU*xWirs^QLq0EVN1CPI4xEYi1ba^v=I;7Q_QVVB0JcC`cONy>@`o!@1cB_~rod59 zFHUCAIT^&GrODO^%Q?zhQXJsEKN6~1eceTiL9=a7;h9m?WcBMt+IQU#Yq?5tz%PW2 z&zS&G06Pz*nra3@dtg6rWhBe=AFHe33Np)ut#I7FOxNn@QO~*fC2=1V_8zX*_fo7+ z-@xIN*rLVlH|3FVb-W3SA$qQ^Iq@G{3@LZ~&TjE)^)ibJcAl>wT1vC_`Qt(=MQTd6 z2a6foHD1#+10<>|jBvRV%j6n#f#}=6#+!<&wxzQkQ`8QcTnlnPl zZJHGnusn8@6NeKORK^mQuT36NIa+Tv%S$}1_bkN0*;nOV_Xjn0W(EE-^!X}*=ru+zAfGRdhP+8qeE9G{_TSwv zZa=er%WeHZHDh*jNGz+M!7(Nndrtw2h+0#}MD4!nB`MtC>k}lPSwJnkmKc0e%+@b1=keH z`Q*vAzLt_%A*fqb4AvK``trzDXI)6V0~|~-z@OG}!=MIeqI4bJB=|`OC9iVH=(Cf< zPbd9su0V(ZkH8cI1LrV>1N@cyfsd5#ulSYk;! zKyfZDcUn@6Q#2WU5AkK6pMPNeqVeIk176EZ>qFo8-_ga)fPvogg8uBvKgHLA}Ws%_7YsVESc+?Cs}Qy8c{ehDBOD_Y~7jsHe&aHYRRznP2DVm!5j%TRIv z7Xq$r5e-UP(efXSv$C2(J%WDjynL;yqmH(;F~l1jc%j$t_~47B?yjKCI|-Imjp1GB z=`*-F>2hQd#keOI+?Qf(&{_i6%KQT@a} zoZdCU9dc!VLRNNqdv8cDUQqskht}w@QG?D3;Q?^}T1wt(!}%bF#Nw6fm-D$`}KlgllAJgsn$i@?6;{i1XR#ZmoGy z>!)R20n4**ohWQVwm9n12W`48iKbmu;-{5h7W5ndBo@dEUdlD;aA69z+S&2X)FzJJ zYGpnVIC8ViVB!Hyc2Iqe!t?{NX(--dG34I3X4&1}KM^`q%d;E=PcKPx5X-5OdVx|! zV2p*!fKmdr#RWBu^Ixx-Og?YzYWg7e0o-1o8QbE`Zwa&`uHJmDg&lUj@*j=MSWGE3 zgaD$43t&iNaa0$Eba_J41j=Ua$k^ny;De5T&!yd{ ziBG^$C%>lOEFeZf+z7U>4J69(jzr@qkob(2sWlGSTe->3_tCDqk zB{4){;zp*F7u_!`F9L_W0Cbo%@ke#3K+7Z_IhduI3s=;O2iNCgg z`KAMEBp$UlqbDb&J?k&WSw1`XC8_r=TKbEivm_mofZdgW5Fxr0y=p*fj!i;;pjy`S z7daXp*yWiq8HsCFwhB&;t=1Fk*te%YE{Yg0{kmD;? z4DhnribgGQ3yXOxoOVp<8wbQ2obDxU?^CbWPu|GU>WQDBjF7j_QV>!U?$y{qz)@X{ zWH9K=>htc8h}2}sG+fbLek@w2KScQb)F?FM#whEE%9ppIM$(!GJSjf2IV5K?EoIa< zWHkrtiMj+UemJdV=7xuIc%Pu&%84wG46q{uBpSPnXsrtFthaq!HEQJ<$Gn> z>pi``-`W8$KOJSn7gPhF-H##90U#or8;=$FbE1+J#CKwvU$T{2zBy|ogs^vTFuG83 zD<(xH-kFAjhM7qK$yNt;5%wI+R`v$7>Cgadb*FNIuz^_u?o@xWM%B@ey!`vPsqpdh z?11lA!OIH$;9G^44SKScVM+qv3yxt)t?*nx2ATX zGJ!Gn=?ZSLyus?DZ{N}85lov;Sr9)8Wao6G%!u2oCUwI|fFHlV8mm7M@P%r|kY2Uk zXn%|8#t_&2Qr*wduIvn$PUrtI*8Za*aZ?tCM6?shHnX{eGZk&5<;-rtSH0bJ-^`X= z@2Bzk%wdx@mAd08G+G8jD5ClGKvV>l`&@#`0eFEqT0bWKjsoTlr$cvZ);m3K|KQfa zXX9s;?&;1;U#fem?}eYOv8L@}R+`UKRu}@D6C?X{q=}!9e7bl?1H)O7tHD-di9BF`QNmy`<=wjVvp?iV;{pb*2sP{f!B|o6!4iSu<3YPKW>RK z$aCRJPuJs}S!}%#t$$ZMT*qP>yIQhgtjK7FN6|tA2H)GQTXoA$|7vQ%V47QkJH@ZM zsE?|D{Oa}dg;vPWZd>08G#-%UU@uYCOKl*Et0<8_v#Dw+7;~IzhJ$S{5-PNPC+>&v zq}UB6H9GXizZ(tRid3K{9s!{WQBva9_G7U$FS;dRp-(0QjJHoynSb6>j4qzWc|>@& z7pOdqPW@>AEl9M({q?X;F@7J|ZcLG!KJ9${`z&w;Aapz52gW^M7nJJ$$?L#a((i57 z!R;)qkAJ{C zIwqYyZ%f~&^W7QJ2yw~)c*~5jrX-mJVBzQ4G$v@e5KJr#fS8w9U-1xCo8qiXyWwrQ zB$vfyZ>f3-(dRc;K~NHQ0QK`pe_sodwbT0*QQe#<=_KDm{Qic2QmkvLM(du0O4HI%fVRsSDN(mSIsU-&gNRjsZ$?!W^9Q)eu7#%^FdJo{{j`YN`!La>K3Peo277`-AXjPBf z!sLb}I1kH_pVMij?B(S++T8r&8gdo!KdnqQ%*`Im24G>Ja4edSDsN(E=-aFK*(z2 zI9dJ$vx0Anj?rn7uHE+Hg;w_q(=S~CY9H$zm$fdi1Ia0;81X|yVM+@WbbJ!?>0&|t z>jI&+7>=RfmnC;cc%JP`3dh~);i2t?yPP%dA=u$&Buu#9ITH*sc;b;+;>|}siOKwI z%M>V^x839`+}Qt;6xUb(HLYYwdH4|t2Z%0>`!t+Wsd~hgk=5Zrz+>gl)RX!txtf55 z{2PXWAqq|et-+Ub5axfcao^J!-Q8u!QCVHm+iTW4oFoZ=?wneO5J!C1(U^rMdTW^0 z$#3L`XphZ*Tl^sYKFd`?uF02?C7&8tLA|+B&sv~7PIX@G(j?oUfb_IxlnyY-00@DQLvj(A7mwx85nPF-~trrg^DAbZ;>Kh@Jq z7q&T^opCrrc?tW>Q|otgv{JD30aGPiwoluX;B#hiO#l5>Al`AGXHz4k^7n%G8@Bi4 z>^XtVo@tREN(wmGr+9l8*G5qskQLmA(nSqoi6&-j$(qi27H7$SUsZg|6$TiVVmd#` z>AKr2_9BxTiVT8FEP*h1p8w=LCxZ&MCd6m+FV(rYC z#UPWb9ZE*hBX_a`9+LKM;lH>VR0FnQgG7TkKo#5p#R-_PP>PKwO}jkt3pHGB?Bs&% zP@S5~IqQqVotLCiyx)K2=GeUQRxpO04WO|nt`Grx87|nH5eMQDw#OJ8+wvx?>#U_R zux#rjtfo=I!JDMtV9EuXH$X8XHWlVYRNS)c_Eb`N&ijJj_@$mIdud>H1_M9AvN%KC zRw24&&9!Ypr?pX)9@vf#MhzSE*B8w{dU}h}eOGk`Zm!Rf(R`MnX%!l87aS}1S3B6= z(Q(&B?WD;&Te(*|IgJa{3{Yo^bF@l(ZGOcs(0-`nCIAnm6LZc+JlgNCdkcvF;ovov zynWX0Qr|UsewDEuZDn(khF2fKMyBBUI`^eL9B;JqTh^ow!7F>@6N4Sb)#TSxy7k1B zofV~B3Jctlxa~Bj!>Jh4d8fjJAbdkJL~;UpBy9G$(fpO*qTJMjAsk21G84a1kQ=7_ z-8{1QNdOzTx{pgS+2_@6miSqsW@|?CL(i&S)tTo7VPI0#&BnrfLGh*zXzd8dxYN--lcmi@9#B zD!<)rr!ByIm1&-y|52SD_@C{DbRQb}TbG>M=;pVAUNyDgll;Bn zavP}x2gk2Gw|+GKdFd_<`^ssm{~$!?2(NenF${QOVTECuJ?BhkA3LE%gTdb31QTT- zE?}X8cVm)1W?sQ9nR!3vYH^D8+O@RLwHLy*S*dgsUfj}Ly#4A}Um1yK`+3F+h}}*w zK5Np(y_xmgX>I*0)Q_}IO{Uj-X>Lg49O2CScO`k{@?UJ6hDT&A%*Hq{w0z0CJuPa{ z#>sBU81jlmMS;{Z@A8uWY8A~Tt-0!1ooM_DJH@FNUB|ll1{{m-*E-h64b6+jjmiQ1 z0pGg}ztfiH+nbcR(~W8pXs#9gzc0$#+RBa@@i#e^l+v*V;-&NhexeG2iu68+3tgCT zr=bbIzuR9F#NLfwZ-yi$qrAN(5<(;~fRTEd3wv8$6bVG-uEWQ4RnPxvLGtLRz+9J_ z^YY!S8XMZC1lW+Hb15w?hdDbX>9>nJJXnApo37xMV8o>+5{B$X86?rI?KpF4iLI?L zZ-LqC__U zK_#2ZuraJnG4d~a5|{z*yjo*6+^s+YXbeV|t6X%-<*a7sX2(tBDrPq;eFelYU!xzS z@zcbGlo>3K9#H?IsV|RdR-AY11vkYsK>&W8Q)@FRCYG?zF@`KltWe$Z4a^JO%6UT zXDB_0zya8>SIh%hzllbbM~}n9zy##ak=5)0VSk%ED-vTuvtWZKo=xNAhwOrDCEyjl zNcP!Y&$77QnLYYdFw?>_JK~WO(^(Y29OWjz0Ng^Vik7#@SeE8krpmLJ5#!0)y4}u- zXQ3fZeLpT{e2<}z|Lv|W*LFs}pV>#DJC%Z^;zd4@pq4X0WT?R)LV;NOw#*rp*MFj7 z=T7*;`v&=QUHlS(Qw}5v6{w>5a82SdABYc@an3YPstiz!=)Ppw%?F#i8#$2f2(jWa zdUZLzyc)sZRqtqLI8WW$t5uYYz)v#{6e-zchDOkm=fbf-mO~S^KQ$qZYCSHmWfE3l zZ&tD5t>IUvC6Heb83xPp)?TV?kKdDcf0iK5^gi6qBK6}e7r>L#&=J_mSotHuMrk9> z^}Jcyk5k#_^WmFS&vpNv8-s-?8=qB*JK2o#p{}_Ih zx=6KoYi7yq*@MD@R$b#BD2tieBz3#|{L-1ICc7nFdS>?^dm~Hf(0H|)e$iyww34H` za8v^{1;IjnL4>AG4J>oC6T5Rtd`?tIS4Y3sxv774%aGexDEikmS)&w?J{TAZAXt1P7TrF7|@oDfo%pg4RF0s`pKN zsO-^;b=qG7n4RF$>u_bt3K2`dCTKxCKT-9&Ns?>#V+S}lF>JLy77k$z@-@jzc_u<} zU%tmo5c2$Xnx1j2up@>GG>GcEg4O2}Cv3xgfDu=_J&4QJ`v9VLD0kwzGEzRmWNXR) zt?uWqGqt+C6 zZ0qSGE{%V#q6xms*VRU@DULs87#o68WRCk$DJZVM{dgS}N-61+f+BkBFBV98tGSJ{ zSs3vf|5C5g0@b<;7~UB-y@|xsC?P2_EB%pml#W4-;DP-~d zyeGQNP#)1wwme*I4M?|&FAr;I$aur>88E5Z@vnpPllDHXE$I#lNCJKdMUYv9`01sO zEu~>0pG%j}ze?`7w?r8+Gn{t2`Sy5>jb#B+m6gN-nU zW-Hj;`&Fpo$>X$^4nqYuhMT3}z^~8rX><}PzknCOM=HxFO5oXk52nUDU)tLZi0>4` zQv$B<6s$Q+Jwsb%{m>2de%Z>=BlIQWj*dkvf3Md6RC8MtHav*e2O&T1F&A(;g5t1& zy^9Mt6Tg$SfeZMkKj-~iQ7U>pN!dW|hmz@1EIqvJf5wB!%9N_U+~k#^R0jKzQZGz< z7$$+x(3&w|b>H_2dUJ;m!W3sWFg!}zXiBCJP@UfdD0`-kzMYadXyTDP-9T8Jfy(7+-02Z^3Fq&xE ze1<7g$Sm>Fc#45GxE=V(G)#ReFr2^glW28EUB^wLO?8Cc)6N^ek_Fg?UbN&~iTdX`9V4JW*L=-8G zcwcqP*+5~_{RhO+u3z0btm;a3n0Mjz9)i930uO80=}&spwV) z8^bS06YSYc*DP{pFMT4WWM3Q{NOpG->$WYnrLXK3^ZGMLvsn0;aULOY7T?Xcg#iX` zSQyEkkQ$YI9KWlD3=Z^E%kzX3FWc(>N7J2hb!R~0wKBg{nv(~PDMQpcimG*%47mT? z3&H!oVNY=u(lQQP^H#M5K!Dv-h_QF49`seYAY|YM?!^-gBfme(5Cjo#`&{&J6NTp6 zF#>3E$w;0~f5IRM746%pUE|$=lgn;NHwi6&kY!=t{W0>)7Tp?RW&1{G2rCQv-~~53 zi&BBcfq-Jl6B_-B{i62?IGm~;TB`d(?om+LiODVBPi+Y*J?26qzn`VFYU_K^O?8T$ za#1CTW;=&9h|xD!ew3F{L=L?6E{?|Jlx$&M8Vfai6$-bt9i&f}eGt{zpxLeS`k!hS zv6$>c2?3Zb0>20Cp?s8KJCgQBwsuxS#kMtXqVQHasrFZUnoGpBwRauQc9^a{rn&pt ztp6mN%8gkwIm%9W(!VyFjQ@{Dc*Ik(MMM@8gFiCvXt96AyyW*nMXl_*R^ip(Km52q z9o0oU*9XVJ`HAf0D9ZIUWZW#6bC+uEt&m+LtiNpkAs~@mhO=KZXQ}l2_k%lIhc9mz zvO!tThpcC+@0-mbcmvq24J0UTfR=M+^wTwO|$ z{Ur}RXUZfv${zt4ZBE)mByhy=(FiInb6BFhnP2!{sH0aQ=AQTsl^aCEA*Hv5IX@VksEG&#@z$Gw_uBr znqNq(DFSAR8iH|C1KA%2GMR&$mu~6sW81qii4P|OgvCwnc}Dfi@o{gR1Sq5ZqJT`b z$sJv>*F~KY10bSlU3_kKl9nQ&3Yi$g!s2NEa9RP)T?wCC@u+WV;Zohn@*6VDozLMH zO=|r1wVlIWwiNicn@4QwLoFzd8tSVeTi6d}&j3|Jz25eeGgaA~U+q?|1|Mdpq*`4s z^WMbzv)mQ=g@;n4Q+pIIZMyZJ04lL6L9z1fH?GZDsG_X{ErBGp@o&!|#+Q-@;<6f| zvV8Zam2da(908kMb{~p=Yqnh#HO!uT9dFEj)e(j|Rrq#nlsluUTyYP!iQjRE$mZ>5 z6o1O~Fpv0B2LTZ5J~*jE8I}O~|9>>x0b^GkA9Rc7E}tKG!uFa4JFY%!nl?c{evoyb zE9c~YVX7g6E2Ns5GSZJEnnAAuF-pD46p!C{y_A|#s_}HE|EpTTu-%HmHxpWb>Qt&~ z_?1SqS8?8`ksGa~(NmRot&H{FEA_#>+m6o0Q-4zv{-e3F3wOgW*y|RV2@mDYiHIa< zR(%CDNIOZd-FAaHHv+P+C;f^!+393EWdsId*k`KFtctupfi>Gn5TGxbes^$O&=^zA zLIt{k^5SU4iaRntUUz7HqE>MO2F5|i;xhmqSr@lYkhd7jRfY)|Os%6HbQ%@NrZ&y{ zJa(Bd9cD28`JDDGf4I=|>pZ}Bq4S_M_Cd2KV5aofu~PMkym?iyhY-z#9o^CLOV+t0S=u+&@c#naeo(7^#{qZN*er8SN3O) zH*1bqhEl@aHTPw8GJ=X5MOi<&<~#Zd%Cbkp1HMVwCK~{|SjZr8?@*sqKCCT7f%e*E zW#F~%Z`*W^H{T8j)1T}!_S&f}>}<-K9+m3)swgyG@cCP1!ihB*diD;;wPGQFQW26e z@o;d}wvOnOUxvBm;UN-^%UqgP`o3qZG%VbAU-R>DH0}Mj2V`LNrLQN|M%PNMAtnlB zNFq4|HaBCEK}ysom65!BKU`>zRA(c~G=Zr&Wb*fjw^Z5&3%yOoINfGl3=X+tzv_rI zrcB|%I8fh7O-q`3Vy#{uCDJs#2l*l_)vC+m!0oX_f}o95TqSe8QCnh+K982~srdO1 z6g&J5fjwSFosvz`0bS~@bE)jH*f|!TSsCPjSj}clEX<=1;}xkDmf%p?lDD)9x~Vdx z-&Ag}MfL#dik2QRV*tbsXD@@Sum+k%0eZ3Zsim5Le9mfKLBY2=tfK)=*yhG^(6J5?PEykWIy7P*1kO zz6fb%zx7;m^sh!Eu%9(y77IMweoA!UDj+6^CI#zH`nqY{%`k2-_q<&03E{2bn-{PZ z{b+wpBkIYZc&}KBy~>R%5k{b$tVHiu<(mLEWwAzkT8%U)k-hAgc~>TgVZW9k>>`c# zgdWfTR}e)DjJ5^jg4p>rstGcRZb%8{-Yu*|VKCA0y^@}=oqfo#nq=6(0tTJes@N-sEIym)9xBCz^3A|Cv2k zYkI}h*4)~jWQTLO1XcWWkGbzq?=pSu`e#Y^IL7ZdMj-Kva!opoR5mhX;S7J=Ia8_u zjomxS@#|ES&a3uPLnzML)|jkN<*!)EJ7*^QS%ko|fl0YMkA9yMht7;<7FDQ@S? z-Xs=4P&C|3Dv#VE_P6o<&RJdTlKcyO-epli9Wou7f&4`$GjaewU=41I)N_^L!cR~7yH77^ZY5EH1xffK~vU^wB zCIWnBy`y{n5`uMywE?~eeo;-A&i(rNDw#jNG*qU zB+odsu5@xmWdi<3Qw7eO!|p2&tsj@IL{p2A+I3mccM-2I(St7iFS`BWWg<4Yn|^ay zl!O^TQmlA2vMK>$iqTBXU&q(m7L1lbNn-)WB$F#0^O*;BJh_JF&xjEf#haCTK;`Rk z)}rDq57z|hUVxmrwNtkqx+LJML9p;U&-p zWm40y6U)p|Dlf%s7&wEfa*Bl0OmOQp`D*S8I9QCzp}bq0t*p}G`=Bl#FJ1`Og7!}# zd+(o%1G0;nu}G>XDb6Y2Sjb>Y$< zxCuU$ay>emsu&lr4}z`OCkSq<5`Am)3n_ta-AyS3)`~9FlN5KQ`_i@r#;_%0<>8Gh zhGVGz6_roM61o||f@mxUT!$Du6k5Y#KJ&au!#c61rcT96eA=ZYMcYZr;nB zQe>8r={>nFX<);<$UoH_AHfM+4O08s}_AvB> z)7@QPesB8q3OXOpmxY(Jw)?R}3*X?u74GD@B&r_JS73GS&f$IQR_eVB%2BbmQY*tx zJuX-sQ5}GF=yTb`EnabV4XXIf9tmD8>IOB;9p&ZKrHH7JGWVlPCU!5T0wOrJ7F0lP zP2VtV{LzS?pGi){qoU8PPpZHCV{nee5-m+UhXs()#my9{lI?D>*}Wq{;TzhF0}B)qX&^n%>_sp-_AZ*p(%)w&y$G~fp9DQ_#z#{i>DGonF%UAU9xtV^!RjeTnh z0GP>qm9Maeyn+gmMk>?`R_Bd>eJPT0O9`hN2efC~fK_H7;TqhWun7;TghfEWY~L&E zd<3ox<&-Vk>jnz1Pb?TH*PzE_=GWb5`WbFZCKuZ|GXtKZs%M~nb0BOiewoYIUw(AH zz?oTCe{U;*@W^zcbF~GJ``@yZ_CUf-+N<2=`QYDNS=dG~95_L)PC45}AW2(`M z!Axv@LRya=zpE{eC{J6oI-)_mMogabpVg}xU`Xt}iH;T~q?;6Y%V=P!y;tKB()aB1 z9g&}^J0zau@!JZu=KigW&uIUm;@8MI9rb59ep=vS3VHaUTn&lu5wW|pc;vILShJb@ z#i2sC1j8p`%l`MR^dCQ@`)G}O`j`>G+Yl`0%edd+RGErVLTdamz>{FBS(>fUL4EeJ zvdhd%whypUYZ{OMSfezOd}3{vf3?;g_TvK z{>gr$>7>F|5i(0^-hC%eerH@pU4kT6@iMD3RC%&kSQSy(>UGEeGN>R!U)$Y92 z|BJo1jEW=dwgw3h+=9DHaEIXT4#C}n2Z!M9?hssqySo$IEx3C(1ZZd=(|P5)cjmoo z-5>LB*05Gn)g@F>^_;4+kL>N^;56#Fv0Ykufo1bX)w%2zhpb|ZtQ#&@+#SU1#3H;- zV3`av<#L&Zo{E%Sw_5oT2BPPg52;?>E%}x^8g#%K$zQV?Elm5RW?!Rp%i!!AX+B!Y zLyW#fl*3ET*~pq_yadrKv4H-UWVY|$C$IG1kUqVA0+0ne+~>z%#}A>fR|jH9MxJ6ojUN~Y#Hu_1+-nQ0^3vp0nvi0 z{#2l<0bJ6LhH6DXHo$AMs=)|pgQI1A{Eo%`8g=UztxL`ZhXJpogzK>$>~~M^_pXQ@ zUI8~j-OdFUq&T21SBUVl1Mbie)nI2r=;iMkQrzfyRswZvZJ+T&u*Tk4;yBV6wYMl= zUw8mI!QcSU4K$=8?GTe^_?7LDfJa(7dMdD2Fo!D#_>^vD1Bz;QV@ z{2GtlE>XW!8hEG$u%Yom_tA^k?La6-^P&6&?DW2XqMdxH{Yh=V#mDUhpxW{?p7V4r z*XCAkPY72zxZ81GY9&iJkgeA`Ohj3?x#iKyz;)z9C?W@BdbzwrApQ&{liN*T0BTDg zw3qFe(bCBhi?5xlQ-z`4*@x2w!w-&~c9PUd?)K8XUkrxQ$$7A09$<^=hZyQ-yh+Y_ zKO2i&A3XIwlKEU1M`qu=V8@IZqeeiu2yM+JfYdx}a;AHIXcx~V{Jz?qvBES!sg#25 z@F@na)@MZu*n*F1#cs)%mcLQWAI?BK2LV`KB1RyET_;4P-Be)iLrnP@@FG~Z+X)}! z?Y#f7I9tMAeOmCTM!cJq!ihR1%q$e<{%$fBL!6x4gxC$+TkY!e?V`ykdw*RT#EHAv zs2VhG&KOz-#qCvGHs>fQBj=hAeUn;VQ!-Yar!qy_nYQhhgP!33Sv5qjLJyHNONjXm z@e5TZw^XugmL39fwnQX&c^d%9e_b*LeX>Ua*NE5-{DI0sBG%y8(kfQj6X2 zHC{8XR-oeYX$MBVUraoJ5kJR9T?U07lEZY1+wfh@cCL5N^=gRfYH}o>&g7v;pm0D# z5QOnw5(2l&ye>xZvIF|pURcROyE&`9%x2Y=|MJl2tM+zxw{}LAWYen9^0rFqdQ_Vp zDvnQ33fbvFa_HSE!h!i8MD`?4J@Zj(YplR9x}YMt7QC*6A?J(OWJzYhmN>Ny%nK@> zy{<1%(NM49A4Yl2xCk!B^i(6ERHO0mM+LO^`hp;JaRE4vEqzhi9;krL0|wqGZbmZn zu|iNKfiJ(Q^t@>tRbO=fGzU{gQ(IPiX=;_JuAiM8#2HQbNig~v3xy2n?*0Q9Uuyu-7OY}_uK+;>H+vV1TS|)P2RX@>eu@_ zZkDhU1O$90nUGtd7RZmHW*pUK-0FEn2S^jKfxP#Hd*@nqLI`;yodlWpwgiSVa0%_T zzg5oSC83r)d~NyKX`_!RJS9h~++zwiA}n zf;Z1OaFa>F_J&cisks0PCF09KKl~L9&QldcM(%M{nEFKxgax=8fwa9L?-m{~qpL3A zD)Eltl2RqH?D|uPm2(A3&cEHwx-^8`TJS;&eHI}D1;VZ#BJw34B=#aw1`;qRg>!1oG0u}gOB3awflKB0y-{@Q+E+KM+hlHGS_1kwBM>&$&pkfF{O+YY10vxf zD8f`7E!pSCHZL~RxQS1M2HL9_Q6^3K9L2I`o6RV-a>S6q6<^dx*$sJmy=Md^)x3y8 z+R-iyAoUA;rWLl?%f~qt=NO>p8DvJCqNLA6R(%TA-T7#>bCO|$x173p7?8mf<7*2{ zi0s8YYuiDFP@xMgK++bNI=TjiMqbJ15Z|rCE^-n5!3&qM2tZ7Khj9XxB1+bPt$nm} zr2_Wyy4!6_=w*Ik2SB_cynKw)Z@6i-HoV_P>u8od8z7k&l10NI8}PIcMZA^CVk10q zj{?-BZr(@8aT9Ef(d}Q1_w)a1fEpLP2sACd7#7)lYX_Q#F0;DDxFu85D!9j?ZZ_yF z8%a}mqTCk1+#5x|q(KxYgTuYR1`fi;$fw5dx`~dwHe~_ZKguAuvWch4uKX9H(~?E- zr^U;;@ok;9q?qcQ@UJ>hTM+RSJHLY-&;B%39u=~JJyDQ&jIVaiotD5RAd{0N)l02vB8KqOH9u%2Jr zKmhc`H^1XFHHBiLrW8Ju+>l+dc zFhUOiAI)7CmX9oqe%?8nwMRw>O1jq-RB`o#6!H={W9eoxC)H79{k$eB+-zCm1T1Il ztAH3F2ab4g5hgm_Mh6K2iH2+TYIxr0-TLq+Q0O$X{V3^Lll0b0WaZC+6U5^*C=DW7 zzbf8=Ng&*}Z_^^J=p(FgBF!Q(Z&+T7gCfX+=!Iv5?+==_vTH`BU*GC1_fb{ky6ilX zKdsnl;;K$0Fk2*CeYmFk7Ie2dM7G&=Ko`6}&(zCoV}uW!>DxEPnOf;}Y)^;3boB~7 zW4C!$M6^IXOpkm&JV|w<6Wt`cIY|zG`HFKld)Wr`guL=e{gELE;wGk>Zg1y3Pmg0~ zSa^QdtqUp6In!G)6JCvnD1@o z`L!;rmPOEpB0_t~k$arE_C=V*61P)FL7UWaXOmSWkp}~VI{p_FntwjzmQjJks`R8c ztcTDO@Uo|mMq>GKQ@LWeI)T)l>RUrNBdy>D!ja>(UsPN-h=vu15Kc!Y7f_lVn88p5 z$SDshtnlRuQCm(?X$CT!YtN53Qx0H1@|aGtS==rNWF2hAb5)(E$-lmR`Z>xjKsv~{IDKVH@5#hR0->gpOt>-Yr%)3i0X z7?pNkcuOCm6z%{I;48y(167NSgcLDUyJ=Vh4gy}(}bt8}>SAxa=d zL0j%b7l~1l2MXW53CZuhoB)`eyM}^(9NC*`$W0dc;9ZtzG0+` z?De*FI6=XN?D09Coi*jCLJAds)2P(R2MpKpPQdJQG$`bAaM*6mj+uE-f1^~qiXLl{zU*-Jp9cm^u^A?_q{t!cuTp#h=uzH zrZ0wR4n-Z_=J@KLXBrT(;S;`LoE#-!2MlQRqVAk#D;EOYMS?m z4A&?9tnHXb8j?RtPjo@kdPrKu8zf?6KRO`qYhU^R-X8bE%jqJ0IUzD67Y*llMwHQy z+HVVekdUX~0B5RPSrYU9-O;eb7YR z>l-=-)H|jPE0|tVo%q?#HV?(e)j-*c4s-3VXU(gPUSyK55tg8lqO$<>x;NyV7wZM4 zz~;r@TYB`@S9>}eQYqtd2v)M!I@PhI^DJmc)!%{s9FU|+u%k^cGo<>zj^tve$twQ< zmDMO%Dd6*xJCV+D<8paR`T}ldQ(WW3yKHK@WSF3z-yxwE$kUAtgpzI0SyLDCMsMe- zYNr<-<>OSQf7UkIzad4u_)8FA#N8xLm864zvQ79L9xiH8ngjBe&h*^$6kIwM z?yh78{hT|rF~C*Iyx*QO!H_Iz@G-UubqY>1629U^1{*^4wN$`$Z)b?A+A&b`zp zle~SYpcTw~ztu>!r4K3cps=C9L>+_X3}3r`$G!KBZp?VS^<=DdhhTMYbvV6u39ZAr zz$@<+LS=T(1@<<2ttPi?vy&RelvYW4my|)qc3kV_wBL4#2)G+#2bK5dF@bz4d}UVB z-+l^Ik%-;UbY;(c+_>h+lVKYW9*G=Z=ysyT=;DQ|f06l{R_Aulr5j}f%^HFz%*!@E z0nxh33_&%Jb|9ZnXgN~+Yu_qo@-)dJHcScfovyFcFnW(~NO*u5g~-cpJiz3>N-Vc= z8_0k1GKx0C+-3f}P>(1y{1`~f&QMP)9)pGPGyd>{wtE8TW59qREq>Shjw@j6RZB!Qi->2+TkU(*Auw*=HPbeciBd+qnMJqfCE zIs!!A^Xp3^9g$IoSMeld|-3U@mQM)w8R28FqT}v-OIyO-> zF)8|aXP6AS7t5=B^sBGf>I;Es1*nscxeQN+1*FvsK`-jI;U zz_;0n#5WN6j^lIG! zJ8!G9hhM)GsChX`+K{P}F>jv|i0wxvF;y+*;28l!4+|hepxu6M{$L!|Kp$3fuHL!8 zkVD0Jwgtu-q&}ONV;ZSGtAwdx7VE@nsPA1HbTRb7!#zmJves9At^}YxoWtiU;9RLU z=I)j45LO<^!o;G!eJdb$T}Ks*zb$#Z+h5ae&!o(FCWN({F_?e~N>BlMgzY+@D*$P# zw&WZ zdnZ_frX#_6eNHY|u$z5sW_Be|uy^PUNgMphme44Fze+%bOeBb|IsW)!Sj40aSI;-D zthb0f=kd+P**@%!P|%w~A|;W3vu~4kFpusvo2Bo|P3Sk{hu-gkVx^~MJL+HRNBzuJ z24!{}n3CF&1^77x=m+nXv^4u(uo50Dnm~nn#*q*HSiy_R^KK7|)q>ol`Q>r!=62@x zzN28?jWRe~i*@SD9DT>FLQVhd1`q{cbgbWeFA=bLPWm2C)rciOz|YkyX##c7K%%8A zDNBpbJTB=SQWrK9)CiPs>kdS{Z~b^y10uOxBeg$`c_O)VGH>nrfIQCPw6Y*j!EBbk zaIUjqmw7nMj*E#GFKH=_&koXm7=rLg&JuaWAIJkD;skkXK;?REZTuyjZVF`4j{E6C z>?H|Uc$+XX#EORIPvVUc>iCcR5tle%HiLmZun_CsX#U=`@Yry+Q#9$uB0A3fR%FIy zuo#|M&rwqI;8{sIE_7(i1H2~%7uHSd<~Nm^`!GdJ_nBs z+*a&UTC4!F?9^4dnJoPD4&c@)OwGohAE@EW2C5V$IZe;z9$yXE*F|kveTYD)IWZz{ z(V5*zKO8|A+nG@Hj{*&W6(PYlGQ6-OAWO==k~UaIz7Gf!M~mzoo>jcggz6o@$yM2_ zNV$(W$VpXCUGzzdve~$y?{eK{FH4Z1^o65oPLQ59$*Dl8{OnuURxBRBo4~NgEj4CE z67&Qm#`>@L@pX;X9(YJ$0JJ~Gi);&^CLyl-eWBe$Hf6J(g$pBjYgl&=z&8{kJX_H= zydF2DmaftA_1&-+)k7vk{uCY7R|Aw$=%qi3+lv9HEw8e{G(I;rZAj2gAG8iXs@bqh z`y8o&Rgt{gU+#E$^;C5P(H{sHknbh3HTnS1j)K%bXb%fh*j;QPTyPpAYc>OF(ykky z&4s2Kel<4Tv~?jl#GYkJaKgG-fx{1W+XgrDp}lhUzULt?VD)#7EN?v&mrE0VI_YR{ zw~MlRFKgk-NdQCt-uS&NxpvY37~TIT?&>z-k_m`zkE?5e<^6WvdS;|0Ub0aY4r>Ix zGuLeyV3W|h_3Do7E1c8i0a!hMg9>sjFq@}Y`=4(@U+~IoHdxAwew`h%J<_a!7=hD5_@=?ybWCiGlfFut9L#^MH`O>zG~RD^B#y-f;`zZ^NY&D!K7X zacIM0VUkFd(1GwR|G~lVmzM#4JRiT$TqMGnK$zC}#b8O$6euxZ^eKN}H4j6f4*gtB z-!E~JWLVYK`6lY;j^2n*g$!)g+K-A~;ql_Cy=e`|ka`NEL zXs9QrgSyMCUDkxvS?_%(Os}1xd4=5`yMAq+XT{bkl9`N(o~FrsVMGE8ndYFR)Ic@~ zcnC%JmD3yIGJwbslv3!c)tzI|4s-~wtFQ%5c$c5yT^t)4=>Os<>}pJ_c~Ed5l40(W z;`d$4$P>OV3KX^nQGl6!1wqJr37B3ggPj@C_qmLk_4SL6@jnDvj*0ZSn-%ek6tM`8 z+C|u@O`MHC=>`lys8g(K1on2qA;q?xV{u$6NT=D;5NUPyh3el9Ja$6j8|);x!wGh# z1YOCSNNTm+`%{&cciyvSIN0w1kml)n_`#}tJR4!QjL{vem4_82N11p6HxTOS(g5BE zB|)v2uYhZc6?EQ-?Z~Ov&uWl821MSl&sO~jmYjT@fGS4ITwMsiLA5NkOrK{7ZSeTj zo2s=cyTW=FzgMXBs2&id^ zu=?B2u6BJd7Md)|pzZ$TaoS{R&Zk22jlRJW{{l5pvaB~U2r{tu;((Mi(es}SI}scD z@+2c;FSs$9TLiICwr1mY=SkRF+QcZ%K=Y5Z-;uI0F8DHo-+RII3zHk;ya+?O0&)h7 zsJt96uWy*H-Oi|c8r#EHcw3$Lr-yA(zjH8-G^*~tfR#7Ed62Bs2_WGGOMfpQl=(1v zXa8gj)o9Y{2gXT3Rv8*I29qo2T?Jm86^T5C*d>ONtf4zG?5qC+3Z%-ozjP>Fub zCRV0h{C&n!?VHPBei%g)l|CmsDh3UQ*;O&=fS4yD_OSpxs##^ty59&7*cQ?qu;&y- zh9k(kPlC3i+18Lg3uY!6xvXIdwn0rwca!m(K#83Te-~@-UDxDxVRP9GCKAY%zSHiA z#*S9#3X8MsG$|k;maYuSj7^NLO8#14qWSaVzX?bG|LK29IRAhBn!x;7S5<(4g}676 zANAfJf^1D2$^*?%P=7x-kY#G_Hs*GUPEK~LsxFouHkO_&UoEU?R3)XE*;zP|Xoin& zk1opZ3g(7Kk+{g&$eqk=kpu3r#^zUuQ>)jH7ev@~8l z0&5@f9&O$>S@SAA-Im$^%*imtc&%&F4V%u^cc_l4e!fyg*SHB6lKaEQ=;dwb@2|#F zbJLPPq!Xth&~nty$XXjS+xw+*a%O6cgqn8sk8_EqA*VmN1+TQ?q^dqW8TXTF& zE1VoMuFwWv!k3=T?X$fjcp*d1ml`Jq9L|5cZ@yrI3F1<%M#{4*0L z5V0-j@K$ZF0Z}@gNQuLoguERXqnA!KXCTa=#m>ZgvLMXJf>aDb;%tKS4!*l<(0YFQR^cuL^hMvrOZ^!`y z6tQb(ZmXmgt^TzTbrpWE6bCZi6` z@9=jG6#pM#A~)*Nf<-T$Q}<{OKY#Hbi;8B0vq}_Nici1PGOQ(RN_CT&6>mFi25rLz zJPJ3CNH_bLl)3`m)|(Fwb$>QU8f>Q9bci67UNKF;6&(Gn4}qO%&ZSv5sLM~ZTVza>QVjq^E{|=Q+FO0 zPwQ?Ny8EW;$7V~JLo@U*`9-bwpyTzfD)d#SNl&L5=x$J5ou75eN%UF0Bj4uX$J@EX zrMW`eHVM0GRBo>D4Y|E6ThFR5h7@-9jbIz89+)=Nf`Jfv>-emFlrM?fW;S&$uaW!ehhU?E}3>@Nu`Op<#v++#bNo3dz` zVmp>mN=F2q7)%V^#(w#gHJdeZER+(TXw_BoJhLhlA;)CMG~&zBbk-S#^Yr^{RXIzNc7{lI1Gq4RyU5r5%lW&8 zYaF=97OT@D@(yvEQX#?oNBxVv#u-P-nl~06{~)fTW7^B{bcff}QK2>BF3UUSR^iJ5J^YaN zwmbLSGhkchk7L#l1MJuH5|pD+9AO@=Y?9ITmNU&(^(-Qwj+JQ~<9EgS?nk`j<4pzL zB}a?pq14rPc=iS44bM~F*x`DHWfNY(sW%o#UI^^4+l=NR?WXqOqHSB!fzghQ=s&u`&WgOFsu#>q(!o$rSd1rZL zMr1V%GbAAJJ3lW@sY(PjOld(GSUP?UajE71%B#!c10}=uKu^n-*Wfe7aVSJqVwi$4 zMt!(Xj6QH^Cb1Nhsq6EsT8r~Bw}PVSZ*o0I##dl*s``VL)x4b9_B-PydI(x zYNPd=Z)QVJInzu=S|8hLuSeEkDs7lBs;x!DtBiWo-n6xudruM$fky;Ub;5%mA!^oC zj=WSwBN7J0oP{-Uyv@SF&_`;-i3GBlMOPlF?aJh0Bed?-O~lU%8i$xQuBk*Jqm zlO~=1I<@x01qLj0DU1kn_}T=Ls;LyGV!X_R;{VN&Wv zQt7$K?{he*WeP-LC`)~!b2WcUp-j*X7DNfI&QxGZb!DS{=j5F!iZa6y!@@LXNJDA8 zc4i=$ZUPoosTY3-<7P6X^W7MQ`BR>j?(xsX)=W83>ghj%rR~XUO+9!`IHzD}N=pTZ7QjBxTDdW;A1)o36`Vbc7PYKc7Z9F4-6b9Lt(L&25C1%J4 z*-Hsd5BgbAz%NKgFThuJx+dzEotw4`tnhh81vDe9ay0JFr46x6$9Z`Sx+tDvepimR zYNi6NFqPf+cOtto^V!ZcGInsv0UmfMlcp&XCK2a?ni{|CTY?(~H3tgD%9$%yoq#KV4({M3mN`DpAyl69r(`c*3-6!WX% zk?QkMrjG49T$^73b5)*dN>}RX&DYe zt_$-7HHsS{!02_SC|je?+PTi=mPvTslVj1s7^{H3$0$$@BdVg|>j--I$5?h*-Owb3 z!4jG>9*_8s2qbL8^ZcAOX+($9VCHUbskQJx8{Oq#gK86mbS+_PoX6)=zc|D#(HVx{ z4u23cvf8c~zGb9U*CGh9h@h@I%%0?Hodkf>Hl5Tz?+@ky8|7hElb{9VC4QGouFj$v zPV}+}(=(GRk#vX;57v!SR7hA!6*yvGWUeD$G!@EDYLI26u#1TaQWmKRj|jxHQ;74T zeyg7TQ@9<5`tUyBfklyg7jAUW17isqnf$wpz0^i-BCRI|Y|{D%wpa>nF~Z6QTh%_N`JJrA_L^p!cQHsd^y1cAc&QRZ4h1#J7hfEx=&;|8*( z$)ln)g3|=}v^YS4YxygCzIY&DX<&fn`jjrh)7`Z%o>NT2_K2*DH+#AR1CvpOvg(dMZMo*@bonF z7ou{CM8Gf+;+kLkY5hWtm8S|VDAX7g@pV*5FfhalD)h%1WAR|NLhMhHL}+LHC(K~d z;`#5?E9VGtN{x&znJRFA4vL0z1Tnd~5-3;&dR=iAYAp=DR67i-v%zWBn>6N-q$;aC z<)QA;1bR25@~Gy~1iNv0LB`9kFFlTu^@?J!)TUTfj8IVX&(C6w1J zfY?>^^KtqNeZ(-l6lHpiuZsF;$Bd>CAHs4cQROBXV|C&xXbMvgs~G^ZwRRTnw=6Wi zM{vP}!RNa!lKG?*;Fr{eH#Ww0Zg+mFgl<=Y914|K`!JK*o*e2tK9os-lVyZB+Y*$0 zMWKXhGViGb&*9Abs4dwQY0o~lppaxdH##YjP6DDfWx){-P#6}YliaOT$xmp{M2WA^ z_~lHL)$c?}6LSLIO;n$f{geW|SM*iM{w(a8{0)D>LrGaLmYPWvZr{!H&68zJk5tEM zFH`LUlQtrzi{GH!19t!6-bD9LC3HrFi`3q^JLir+MfIH`fg^f+Txc=s+poF7CKUE7 zMd+v#W5k(A1)zyDu(Kdh1T}qR3%N7$M+`ci5$>Fz1kLmLdssET?blDa`sH)+ypj0ErD=p=%W}^6O%p`D9D0qxxZ$QPNgMUCij_F zGArw&uXsPrR7{LHjCm&`x0Jh@T-paN3-+_E1Fss@2BC!;VCGgPDBp$q{y*9XhY1rH=mD+UztNfV&eFSqQy=l0-k@dz*9)0 zqTHa#sh^N1<#_JHvwu-*$HuB`Z6XiQ)wq9-kGvoL1E^o^4^a?A*(tDYRN$ z)eWuvSl)^0%J(mBfm5mVj+7d4CpdT&e`M3i7+#0Y_c2iXrRolvn-X7gff`)>13xjz z*ob85Xz?%n3z_>1oBv|vzf;an9DIM*{+;LeZ;<-4_g71DR#j8$fBbf_baW%<{3pCt zhtzPnyO>+LlJop?frOKz8)UsJ`Cmv5!QV=j7B;5hPG02tY>)*UpEy`}c-T0}x!BkY zksvtyUkCfU^S|-Cs*97khNT<1KIDj!(&Vh4Exp{x4ar$0oa~)kG`^aeTay2U`4X<2 z5^@j;R}ON%f6ua$^ZuQM{B;QGhn&qnww7jRVI%+a&rPH`AoJ{hPv>uZfb;Jc z_J4ih`upXd{~BcX|2jYR{~^G@{;vpwGUQsYiv5i;I6^`U46 z_x!A}jvF!C!`m~5Nl8xgoA4VV`GSBf4y;;~T%am+cqpU_*g-d)cY$1m zI@s420|uB_lf#eD_!#fdbl&yvSy{o#fFz-pUV4PGvMOXpn;x!N0xh~Wnfg>|U041D zBvJB8P{TnXNiFjT`qSbaL8odojSz7Bf&KYAjh-W%oddTJv8bBzV7Siyq5PP!Ya%m) z_OOspcZQ&wD=)T7bvv78uQ(RA&>)j$*Sgpj-m>SjTsG9vIy`0Sx zi_O#BvA?Wl?ZJh5pw`PFY3fbWglcwx6Fu6iR=FXEx_uATy#4)-G~C`Od3Ya%4msxX z(%$u5z8wqWDK9ub*y(3qINxK|3FA(17K=0DK-gcmn@iJM1s+#!!A? ze5&no@Z(NGP!d!y)Q%$Gp#3iNG1!sDd%M5XB-OblaD00^c_u?Ph<4ml6r2Hy+Z?QNFwa*w16b{T)UU`Wfij{<;R6b6@3w<%5P|_1I7MQWiMS=` zS>h`gM0yC`5@M>Kuwh0dq7pHZMW5zqEASaYDj}~Rq2E+)gq;Y6AmU#TCB+kUsNa2{uuGyWf&L+torqRK)Db6}wAd>0#8Tqx%6#%2qTZt3 z5~b>>6^=_fm6@HndMZ7Ve)4{-GAW!HcNqs0H7Qj|c4-$GJ~S$+s~Mgd#w_`C#uNea zZ)Gp-&}svs0~$fa^;(6}d(DGFVRBO0MQLV{X4z&@W+D6BBWg6=$wA3&m?}R+&*~Rw zN+}4%7fZbt+MAV{`8O;#*hpk}OTddMXCuGKPJ4bMTGCkRAALL`J|aA7`l${#$YuiK zR3|8nO^NM_O}OJa?wCE8-Q=mWG5XU@pZ3Jy$l#ksnP!#tE-j%_QPXJ|dpWn#p;Bdu z&6d{I)OOt#yv$M+H(NgInKzqj%=PE+CE~okGwNIWv+e!RxzKqEVK!keApxN#VFuSI zH$k>z_C)qk_6gTm^A|nj7FfL_y^|IrwzRD4sn1q4O)-3hHMLje%=5w}TUA0e<`qpk z*2T(AGgyYnEc(1+Z3b-`p5iJBtU9d5b!s<-!fN?7`6Ycaefj}E`{&3vOk%`x1}X}p z3;Rs-OpEu-sVbAI<{66hiUmeLnYTJe1BDOtY3rkM<&x#%CX_VRmbO2ebj~)e8EJfORxK0$D#SW z{&{F)INQ$9+;1%0_Q1SprfloDrRE3ojz`~>pFp-@{T6PQa@Xtw+5;t+7R(MW6yX)o zgZ8-Q=JByB)NlF2#pz!A3^4Wpp3VnC8YEY~ldC&JKZ80`jEsT@z*}O*cbRwIYbSO| zVzFR}HTL>ixTLw6cA$6hJf*Z6cj7VZFub)UwEo%-W<*y&Pm+llOD#`1A)G}vip;3f zPSUPxOMW!FvAro5<`<3Xd3METVdAQ$z4UI^^z!%e zgk?69l9U+kl1&(27?+8O!&Sibl%9|oky@5flJb$tkW$FdVbX0nl#bPlB`*`gB_qP| zq4se3@wl+15OHw1k92U&!fBA+bh+Wua`ajtK-Zapl!%OE2jfo^*5HpL#oFJ- z9iB(9t0`R(;N+80#*;XpA;|J22{t3NF?3!sW7sgtT&`9QNA3VxMXY7lz0vqK=9^apyow!8q?@{Mng^^x*pacu3|Qc&Yw2R*P2*GqVq|LA6<2=x-#S6 z6z^qYsdH#Tbdj5E8@H`xEc1V=O?^22eVBddI7u>bHv#tNw&I4+4)uDxg})4NU)!P8 zgss#XY<4Jn8V+~ZX`wyc_JY&SC~jppyoCikQFRcL-CU+n1mX+LdEJ%nEMls=S#J$F42Jb!JS zb>8V@ndus>&(-c4)*MaU7TvaWX6pxY!^Bs6nWH1^2X*~%=JpO>5m5hi3G z_niB*UFB|!ZLaq1I0P)di-Q?~7be9f?hY7-s8Q93IgFV8+N&O#y(l{G?6VFq4v@T( z1t!hPgvpeS4JNoR3oNbjR`90s48|poHXJyQIlb=QOZZ?_ovCsbcYt6&QSbrx$3SE^ zc_@PG-qC)QXLPUm53O^0(Rzg|zgD7JOL%O26n%Ei0vB^hW=NPta{l=Kj(y0=Zu)7h zFq<@k^F4*?hQ=TU%edJsFfyX9VnU`LWppe$mXRvaaPb z6!6RS>$wVljawn@%1bxC^tU-{xpC+@``YxHeozzN``+*5*$~XS{m|)m6#ple2qp&Z z`I@EI=a11Zp)F2VXI?Ws;>=>6=ESw`t*>2sMHnc3K<*sX63zefO*u^Ed8VSV&(t&eH*Rr&s& z;2vwUiD7`<{6&_C{B}hz-}B0Y!->V|;i9L_O<8&Lc2%#)L(pp>*!=fc(%Gd7NF^;l zPQ=p}@#WTw*1_s{=P~pS{GY!2KitrNIP5&^9RK#G|KS+_T?lcaA;OrT+_CcMjr0ZGi*y;6~g}sB>e`kV3iNkllr(__0g;lj~O`a zsMuP1-aq4ix6OOw=j5>ih3c}vbsq?Oh&rZT^ufm7SKRg`5-Pq3QH+a5bMsP}8cr<4 z^~vXC?HhWKzSfWr*t5~ICVQbZ8$R?^bXO*OouU2)&lyP<6FO3Gr;vWk4PobC?e;Q( zPubgRpaq>#XjqrhSCpH1CP1lOW141BG(`4O>bngLt@5&Eoi9NZ${gs=xg$F4HkqK%=beCM zy}h(`Beelt$A(6NLWq)$Dyxbi1#6mFuPPcYNXX9B=psT@1xJ+P!{JVC+vR2^*&yb_ z^Gm2rtz=8^?IWdgszeJy@{8CTL*0bRk@wT2LH(N6V`@!KcIWM`z{pWn2O;U|;`o89 zl96xPl1GZROnSBVx9;pqLXk`D4EEz-?u232dN7dLIBg0-f`mPCpm}$$5bCbS&byOu zA0QyK&oax>vf^436~n8QlVSa8D+|7^>`pn>d{QyG2fkQO51a4Pv(tJ+pHM_)jIaQc z*A%Ld_e~te5OUMP=@%T5o~f5jOtQQ0Sq(+>y?UerQ0~=I38of4M?D4%GFWiscAz;u zCv1+r;2l0jpX7gpX0_iP&+2s0aK6DPv@CUWt%b(xoA(2>I@%Tce|mw{5v;c_YrXo@Q?>fKzcS%UP{y-UfA_1@|6qZCZRW4B@pAJ1+qV9q(f_c& ze?sK{vcJE0^zTOhEo;sGFBq##&i+4uN*NbZ@BbZDhN-K_Vyh9qP4i%eMv8@bMS3D3 zrqNed7i@UA>!vyAr!5rWTkmuhH*qfJtaFc~z#0|_x}z7FyIsx|=42H0S1YWN{~$*Y zbUy`8U!H#qb@CS20l&Wb@AybM_ZYGhCG})w?#+3t#YpWt~-90VQ0z# z=mDctc*3M2s^zRPizLhs%7bC@5(Du56xe8neiGzk`bS@FD5amyRKy~}d!>f#0k2kL z*C$kndv^xAb_BQ)O=iUWPU7)1)t6XTrQyX@KG%}S8}=oF%tvXyGGuno^hzw=Kmk)S z-AXH@219V|InA{XwCCc1zP~JGM47A^k|pJl80{IXo$8H(rFlJwJQ{Z`_hP z{H&n=h&A0LX_eZbUwpXbwfSn5^dbKffA2|8wBE#$ck0GWc0&AYT=*<-a;*btUPssa2cqLVe=$IAF^ZlNM{)GA>kLtT(Bl^_)g74tT235`Grp3i;T?e$n9`RP6>!uDp`I46g%jNK|lAmM9)7M`uwu`ogBMX zdFJ4C${xtWMmJCn80p95NTpmOa+)^^j)rxTZk2U8R}8~Tk`1p(gQEFlTvqhVAVQ#f z5z}7sE3=JVQ3q3ZU5q|y)qelw1|*#o=%a@Krm1Du+Qcw)|ZJWjpiE!Ep61S!67d;3DRi`qom(HPmxN?Z@rbU*Q1Z`wK7 zc0a6`(+6p;Bg@scf*P9C)AMtQ5KU!r(y-lCO+>b|jB_e8Qhu3G0$xF8f1+P>S~ z=Jzh5IIX1OM}3WUqMT7>66%vB#RL9N=sO{v;d zT7>yy)sXemiRpF@0cp8?)fy*mm6uoTFEUR`EEsf#Op`&Y3NZpAap(FEXbOQakI&Yj z<=kRW)(CtxsVzb#$TD!q{p-Wtixe&IfTX)P_W)VJb%EY=T;=$OaR%7)h3mcMLm!S%4a4+fvXO z$+1W^+pXK+VneeRC|4%gbFfk_)~apMi-$h5@T%)~)jaBC!K!S55Vn7)A&YWS=~&5` zoBf_83*p4h$a!?FsQ;p0zPGUY3IxZaoNz!&f4%zJU(QUgQ5bt4=i1X(E`ej#>hPO_ zL4s9z$)eQ$7s2r68It35qJVdGPn$`_L}#Q*&;P~RJ3wdhtnJ>hZQHhOdt%$RZQJI= zwrx(#i8IM0nb=?EkG=Q%z1nA;vwCHvtGc?n`{{aW_4B*0>uzakH%$h-!h#X#bH7 z$yj`L4BzX!lf!Iv*?qrlme;(Z>vhq*%xN>3$}SA#3;>&C#@0Loewy|~2Ec0gGqH4; zqedE#VB`3+o9`T1Py+P1J01FxIOuqw@f+wgx{UU{?|A^F8nn~0H{UPpVQ&J_s>1h; zL5?VCrPqJP?yuubX`$~hy5;M9@O&>yW7*=p@^;5}Sc%I_Wf^@j3uTnI=-!`c0rbW? zYSc_)g})_tk=XemP`3=`<_}{Fjz^)e{|%tekE#_osM?VG{m6u(Wjj^I`MGsDLp#btK8{p>vmi_-m}jf>p!#8h@0it@e1HvxS{KPJSFyM zFb-XhHt0DUUl9WcaxL->WG0_-m_hA^Z0T^8uopON?yG0!R}bpaUc`yBNx;W#C5Ssa zyOFcW>%td#l~6I7d4+tviz74{yCJf0_CE#Fn&$@tRb zkCi>}FORCbbmyQf1xYDD+X*1NgLxKpz8}aX3M&L!b2^Lcg0KnY4oV%7c&L8$zEAIi zNCpVxLwZ37N%Q9ug1ajpMna2qREmg-H;Wsp%!TE|tRIqqwUnYU3ZzsJ$=3%89v4Q0 zTPA5vOP6XgVhuqx*-2%QqFq_Mwfo2j=k4+_w9$FY@8-*0a{a<}*d6{sZ|ZU(Yjw!Fgi-c(vw?VR0yqdFTayAHWc`&SXx#8Odlu(AU}Xl_ zIAEmPjUX_q9TDzE4+MAvzi`xXy^xgjQFtKe1qfq1AfFAv4aMtRs5XJX_Yk%}w7wU> zF?Xyw)k_ofq2M^Nwq(c2M!E!`G3BF>8C_VqC%B(R)uS;8A}T!~KVqH-==JZ6y%m!$ zBU#1eMEL|4S_;W(My*IK2=f(orh^;tN4&A3iOLlVVevI1whSn9_@e8UDKw`hZAfCT zNwp{vp80+a(-_1mdK(J^h#sv(YW~G_(C2Ux(mR-Tz&s&RdHU%nq2a&p&G(Pskai%O zo{L@4>|)!?mP3j)PI542(vZX>-59&9U^zqk8ru=ok@Xy0Q(!hveX_&s#&}S;2 z6a!3LBmEKPFLenT1Lp%)^o@C^rn+X(YxUN!+&LHeD~;Hv?dKl%K&_D*LoQBv4j(a= z4oH6By@+bZsL5-31vGwdMiNj;dC>fU?Zwek0T*ppUJ}U8h0fE&KjQVO6U#fM^1)nm)Ye^EuaL#Jj{(0Q(IlZF!j!zn2KyP^T zZKW9jg*FnwJ28kRui>$J-p|xBzx}J!V{csfY+|y?){S@B;M)?Hx0vb30K)<#cJd9+ z4V#D%LzyBC>-1OXMcF%puZlR*NqB6Y69Xe&bXh%OwP_RVa27Di+mWS5Gm-Y?4!}%m zBG{r;7Q}|gRyrIs&80nbGRrUsgbQsDl&Iq|d63&9qBV6>%4Zg*+DqY6Ar&rLpD&hF z>M=;yUa^78ASGemZ?pK4fNkc-b5lB%HPkg`qlr5E@hZ^u9;0k%tNHDP@CdmKj>$h9 z=Sqbz$6!VKln{@^H94InE^wy3C|>@tlAL2YL}Ip$A%scTYp|;I>*u?+ymF#b8#o+d zq9!^_OrxA3QkIQAM&$0ywJFbQ@7uWeM;ANh7av-&tunzZIclGs5U__s@RCy)EIM>; z(cz`(XF7%tED%>(bu^WScuRi~qJ~FRAzHxwxCY46K8EW&s|TtckB&f$m#yKJ87qB; zh^USb$av#eFTb^EtmhO)X$~=Y^cG=at|OX`DbWwD1j^GUoQSMk{HDQLyZB|#QqaJ# zODPt1I_FMftbd*B09j^`JoKzOX!z*)d7eWql>=@&aEtqgL$D|?u|C}wlRKlba~K{O zi=BfGN|(J@CMymV)woLk?_QRUI*%w)wm!V~6PU#)l)CM6ECNn+3)xAb7arG&bFQa- zglF8`!;Drxcz#}0%*a`lwe5U&5AM5)oD&cQ8KNVaBrY~$+K$Q?wMkj-yKo@ZxFeDm z&zI0kqi9;|%^Cy^a^BA&PFM$u7%cK7fJ67ihl_jH#i7G0z6FCrC!Q813LAZW;o!Lk zgG=*f4-*b@CQe`{Y$l8tEa6888}-45OY@wsp$$B{IWB> z2l=}>a>64d}`~o%y52i(r#=%vxAAVTbcHiFm<U`N83 zR{XjJQdrrC08Z|;4~O;!zO-Rec=n4I@P5xCZ316SzThIC1#f zdPPtj-X-EF3(9v6Zcn($3O3(|nqI^(I z92-t_?~)r6DuNPJ5C@nc4ChEaeIV{507!j$*aO)HWZ-=Ri5)P^%}FYAuzH0knJOJ? z{!AU?%}O_*qBpz>FC~dihHHG}AWnTfj1knTSuh{^z z4#7yqOfm<3jmE?Um}$Cn50>?Evkkhmt_l?Dnq+1isT34Kz=yjb5Eby_b2;CQ9WzC( z{UoyD{aG5~l12|rO$P8DM%E-s+K(bDI(m0#6g3r11CfjzBpqD*9Q^7_>w?XjLP8!_ zP)w^~OKVl+MJnSsO<|Pqv?L1A45!4G;4O$+20O`-2q{bntvH*lG%! z%X(jJJtCg4chLG6&+C@r6`C$ye%KM6eY3=g5TPcuor=~mPHRY;_j2yrQ%FeJ5^QN~ zO5(_P8P{~}j;%&L0l&bC3Ba}~ODTe70zb4*KgIbfkyBLPX3S~YxY#_}ZSLz0`fRzR zr}Ch4x`<{fC@HBIp{}rR8Lfqg1`uV~-;jWuhqapW>lGWBr zU6pD&=IF|zx;IX7w#&DnSL5sNr4Y(2St{Tmgfb`>v7&*!++d|jX;;g^T4oa{@J*zh z9(JMO=K!9@Vow1u<#L>kH}>W>B$(H}XGC*nq&xi>zN*O(J=Wb|Gcmd?`@npw-RiXXnyIa= z(KyU{AJKW^*qwp-)EL{1^0t3`LF&fGW#&*r%}w2IE_j*3&Sv{aFGl$DxT5}{q+Et* zWFvA9zCh7pO{7LuJ%!Crx5Tr+Iw6xDrAZ3YX8KUs*tK~)wL1%zO)f=$Qd0m88nbjtqd958Qg2;HqrK5Dj5ecmg`!dx*(7$ zqmw6Y5UQM4MgoV)P3A}<>7TN?r1I>zKBVt$bDwS69OzzpozB94zINDoTs%L0|ES7E zx`@(^5E~TDJ-egk_qpx^7oWXM_fcQ}z=$ZGWn^{#sb!!?VU?G0&T@ecmw5siXxf@v zdY8R5K)!khMQzuN_05;I;M3w`nD!E5irtgD7uS|n{i?~1ruKk}=9}b-18CY`$uj1X zzfj?jTByrH9O^kwdnzib6bOGuM8<@CjC@!4f_+=qCW}o#S%a#m@?9x`JQ6siHk2o~ z2Ea7e;XE8nyXlNEem}=dQZsW#?j`*h`?_0ArTFt31-0 zhB;MxX`IfT?gPQ=X+@2bB>@wXfzC{aLZ(yVk5vc4lT%dF%}7y4wvyat14c`V$>xqE z(aS;q5;wo;S#2;o%TxF&<}j!-?>{TLV96k>Bw>Z0+THz_vVq7!{d# z|F&x5^h4rE&5x8>Axve2jtMRLeEo{nKaE0|+7%*BBGe|1Sd@d}Rup6N+ zJ|eQ{R^u0v&+>EZRn2+e4eo5&h&*p(V`ihKWoloCGLjqVnlcn;uDs)hP1j~_@zpPG z>lf=$dw7QJVv*Vy*2$DEu|xT$7mX->(y^Cm)4UJahwZkjqI2oO57S;_K#5QR)xrGl z;=qDiVb3D#aHzWm>R5wE!6qE!xvrYca@A_g*$Y|LveyMBHV=qlI;ICc`3&XWB!Y+a zZS|4EA`AuEq~(kWOWnm*BtA*0xvuyc>j2X8u>!oYn!^`;w4`{IRDm6#NpPBD$S(X; z-nk0A=whhm-&XjH@Yk;AnylXp-jA{?sBJT!9dB!gZppu^6wA5aZ@-noN9$ zl;?9PEztL#KiW0U%5=Zl0u+@h>N!Mxz&|DVK55Z(Gbf)PeU5EzqZYQ{!A;(}NTR5h zp4?|~Xv$Ki0-5QrIBA9aMGm#3qm(^sYGo5zby&2jx1f$)+IYzymjY9j)W@z6un<=x zJ}t|b1w_%919&|YtLxXzFTXv?)A?cy8`EJGH8=T(78~o+a1E3@7ExOd+r?&&TY0t% z@eXqgWZFW0Q4I$nvK|{Qw6An?YLH(VxC(SzG-CMW4o{yqWJW8Fb%r@7p+G|_%Hs5Zf{H{|^{nELf@Q37(Bp5o5 zRUT|QeXoU9HovKqHqBHfR20fd2CgfB{lw}Y!v^KwU!grtgASw%Sil5eagAlCID`aj z9~OZDlwiV^eO!JI_mca}kl}jM=b%WFgBBvPNw8qxZ)W_ee``Lci-@kXud#(Zk>&8A zXUQDL?YP{6cOiKA@s8rQr+gRpAYYJciLcT6L+E3&5J%}z3|<0JbF5HL8KL1i4HHE! zHg!?~OwksYCk9H(^~y@h^Cij)d|2+6z?H}^bA<&}{6YLA*E^t>fy#x=mHpL1dLdzg z;6|J)lX8H$lW7=CBy#izGbzW0;r``^+iuW{V(ggyQiR(kIbf!IU=BV0-}fp8{(D#%Gx2`|gF>S*CO!svDXRY8BC zT*eB1E$ntS_qh>MKC*$O%w1PV$>Mz>gCJ+_*xc6$SoYjJhFm9mxu{kHAtogrqso9F>d+u(H&`wY%(mq-wYEF!`ay zd`{t}dxI3%V3<_{dCue0{e}Eq)VN2*TQgsRU(%+irKw1RE~ zob%_BPh!#vOrWfZ*#+1Z(hWdX*Nl91--lL7WWqHLf!8UfB{LJu;Ukyzyr)G*N#YKM z50UkJBabmM$FSh<7WnLjBa#kvWs3T10_Upv`LKAGTdanc&I@x1HOlj%_C-09$;;R)lYZZ;yi%Pmk+WFDjl3MT&-=&dw1|6gRd=m~Wvon)* zZT`}23vHXS586rjEShlbWAy6A*Y#f7@(}y+cJHtxxXgZE z{{%g$Z1s~%r{ii1o{xJn$;kNE)Hx=zEv>xZBm-ylSx?TaUFI4SnVYw4Ini`gqnSxE z|MOM5TV8&Thz8RLUy<>-%o3ckUjjmMo>Bq?C8S42u@Lce#Q_%dSF6``A^@-y!UWd^ z6d&4g>z4cr=3O0_MxO=|(k;EsvY)*qxOj!b#6N?6@DfCO)2Lo3o0;ds}DY|92Cqe3@=JVvMsMC-TAia#-EjDPX=oUWi{~Eo>PBc^`5Sb^3 zC6S}RA}CDYS8Cv9T+P2uR7^bw&!xY)$0_C^MzikL&XVj%uX-P-AjW!_eWfhv@O-~! zv{=N;>d5s3A8Z17L(TvA)TcFNK!Hk+V+!>HnHOlDfNpA<;&#H{D+Bq83a2n5uYzaX zoogJKsFq80Gk=LbD)^c*CG~1ejbwEpD7-N&G6pvxl4@iw zqz+Y7=D(vd2I^4XQI0IqM5KyOsFTf!_waHx7%+xE@)pVXs|=Sz@aasyqvJ@46@PYt zTuuY=ml7bYEHgr%Kk@n#J4~!Y107lqUTa-lZyf`kG7e70;kVPr(K>?6TUcoqra%M1 zUwjVnx6JdyWrdR&RaV@msg=hG!XU(3eLwpX6WfF;N0`A)+FpwTYn7@>xI(i0Xw&9pdTkNKBg7WsqV1UB8LcxEk4=MQ;;cCjjhC8a zXcCH8OtmGoN_eySM(rhAg<*oK`gK+!q?!6#cMz+gyUXc9$q9}|1yALeG-$)k#mdab zQOtv9VQqHT<4e-~4w1CiiR4P`;(l#z*{JplE0ePT9RENe(63<#PTw-J&>;&DIhYu; z2t<3hIBT0dNbe+vZW5T}1}Eo7o}y^?D|sBhr|**wh1_G2@`ZXLMQLMDWYit0Rr)Wr z_r2mRB7Pzs4xNSF0lp|ctq0;7^JW%MsOt1tdscoEv2ZEM#BTdU>bQVOH+q{?X)mx3 zfq=6<3eDOAJC=PU!}-nSdLqYPi{rdI{SK#RnXlSnK89xtop~Q$)a)jo&MM7nb>p35 zIgrCy_aCfwZjpB==H2g0^I^3COcdZy7uX%XJYtqV9&Su8qAZaG_ zOiZ7Kz2AqBGqCxz#K{N=DNCtPOPE-@nmAh+8GMS8Kg0VLMuN6x)+PiDQ1r^qCN`>{ zA_ICsS2HyWS)cg^RXzt8!PJs23EexK3ry8r9t^XmQ|E&rE2{l1!i zDER)qjsJ1j63{EV7&`x9NmX%lG5K>m|CYG_;r9G52HSre>rhO;@6o5e=8uu(^yx|c zvnXca=;SPHZs7PixIc%i!C!^nH-lc(_A_ymh3%&vMb^Mb$rAd%H8pp6R_vX+-t zf+|sX_AhQqRG2`a3mMQy#Y~?N&S%hdQHdYM^=5zr4q}Ie18ZwghjDWjYY&Q4qtB0K zE~jwD(4n9rY5(AaY`$=8SGQ8n?S8+x?EZLryl?%u$xF)lLVfuo*!Tn(3rq8jH%g$Z z;RPVPs%T>enYwIJp?7({a{__7@d@B|Fv`<+0hf9b2*B}1KD8$HLB1D?fcoO>mOgrI z56#jGU3F-}n8(sd?cpo9;bBu_`6#{4Hezd8UcaMj>qo%jt3BXksLA`!8TA)y__NpC zLw+4pdb1+e`>ONkR&w*Q4 zi&h9=#wn?@5bVb#iW3&SapehcfFZR%G<|T$jS!5)4SV@V?_cQv7rlVWgtsSD`h1zG!WOc!%3Q%H(iJ2eGrB9yt}Uk!(}0I+XH2#qeMP7TY46C^|cw3()qtEx7B zr%#1~pjw|$9{($?z|_G!K`Y?58sX+AIO*`7sWOMKs^;Yw3Re_pDd}*Qo8blAg^&fb zQx1!SCV@9NaQD`I0)3htXG;H%cIHX&ErUYO2s+LIHA)wdUqY z&Hc>un=2Mk+YB8s!LWvXi>91FoQlR-0h~6ca|1ZN%cxh!*gIr>(;gIpI$e*#+gsqK zzx;U5ZQmTwZywNwHSAS%nns!WOw-F8(ryan#Ciq$U0YQ2=dkRzzFYuOFC^=8xH;x; z`QYKX!kqLc63Vw&USJ3F_RjGmV`7e=?e0m&eSU!`x}h!@Hrs)3B!Dsu5+eg2h`f}+ z4+KERdv^IjHdip)TlN{=A&N)JiE%dPb$Gs@UWL4Kz$xi;$Y@LkMBZ3B$Ud~suqwj* z*+fc-a<}uu3P4$m7lB(O5t<0N0I*a3FN|Mn!}$B#%^99CW9`D?!V{Ajl8P>ig6h?k z;j|3+yJVt=M4-mIPQ=w~1*2{1_%-n_hJ9D+zCca|vvO>N1rvJ|ho2MS&jro*oDjFQ zST{8bfS203L%0sua!|Br_8B*3aP&Z*&D}$E0jBLi!?b}GXYvSi3;T(EBxOvUN9GFQ z&~oI#-y>jYFeafj#NWmT9gzM4|Al&(|LV;72DdoK6Sb}ug0FiVpnFot^ zGSmc-+V6Aav?@_t2Mo3gAAV1yP2yNcKw2CvLpp7Y_r%_Sg-7f)uILz`O@yB$kEp{n ztVs4X_-y!TSy`(hNF|<22zEir6SX^)PtZ@mZ`hdWVhU0QT?4yQDv`D#X(pBju^ppX z6hG$YfRCa9lQ7_Fejvk`A-nZZ&SqrxMQFD@fNn4WND;dvXLcYv;!2o3tnF^Bi4jv` zy)lsmK_lXx(t@n7oGa15+W|u%k63Jf=YftRvNA}M3XnMO5jv)$m)ay_NRqLf=cu7K#6?Y?>`hhAIqU1;cE)gEY#lq=@D zj<;(XT-9RL6H^lM%2Z0qiAhL@kW3Z9Cgza>^&2Yt%pkwK0m)`nw5*UP$oN$s!@fKM zfmMCQTqDy*cz&8^DI|@fMUSx@AJzONOdBz<3sP3Sy32+MRHj2F9Nsi@um6rtm$ZU_ zfpT+a_^!y97wu7m$c?d?NtZ?_;0@-MU)ox^bK$fWIgW^@2j-~7hy%DwsEEMF&K*^s6bQjVzH`>(#b>Q)jc-{Ct-mO;@Y+W50$rdSEcvEWqbaSz(e6N)Wm? zMDPB&n@drrgkAe69S&Ty$f)|lxo}Pqg&^c_5V-~??6>k-t9Ip#&IQtpXo^JzjXPR_ z!K@zfXy*4Az+LQC6dR{^asv({5c~S**4vlEh^a&30<@|r!S#xg4!AW`wf)Q+$R|!F zpzdktlH%li^ShAQ0v~st^&aq2b6TaL75Z3Pqr6s~4ia{~y$z(x;w3TK z7xwa3>baj8n!F0(>GY3M3lXp&V!$QzfSpH87*8QlvS&s^Oj6r(!Qno(0PmT-!87w1 zK(OC85wAlM4aRm3T@7&?w7te+jhuITL;LWQ> z!l-SLhE`q%?@_u9)mcHDLWFLGuaPon-|RE;d>W*i`KPX$2sa@dxe9a0n6%x}FmfHp zVODk!2bsg-WA)w9409>+f}W4AmE+(gTxxH`S-$Ki9hPJ1?Sy-oQXyO?@9)^*M}=cnrRNC z_!EVRbdf`fV#J+-GC}rsJt#4kZ!MuWCHbcFjhcU2&gqN~@m_JIjV__K$E0o_bEYKj zSjIHOLR)dl`x;96jwQCFO)YGo*U-E zZ*fiMzP1nzv21%Q3J3^l-Rii)dcpcxIn+hcr&Ie0JqJhUihDHjJc~}f8B@P(L=)UJ zyv9vcMjV&lU|w3$nfg>KmJ5?4u%8+;=_sL!FIs%*0FgUXn8eYJRWXxsIWUJEDT@I= z4sSaHNBq*=;C*$j52*sD$_2hSA&nAbQc{e&zKG*mBv`G4eN*L8s!hc#%xYUT_>)1Z z&%Ex4QgTC2mx!w2%!TjK0Z*Zw-YLgvaZ)d%C z=x3Do``rqAYI5x>&#z_dGD-4kVKC`J^mNH7?@dVJg8@Ka%@}K^5x8GX*$MF=s)hP4 zF4i%InR^BelrGHVWnY@ObW&<+Ql=HA733G2Kj)0<4(J1(n~HbjLD(Qet_nqK19l1w zDv{iS(P^oSyat{SImz+Fbf*|47flD%!pKrbi%w5j+gSPv`Ut6o3`)Agah&S$u0?fj z<`Jn!q=kjpE^RCt#nKdF<}#zDrm#sV15s{`c{rpB2N=e+FB0)mmP1 zWCt>CP@H5s%yyIQ72cjisH&2$Nm)FxIIN7MEw;TbWR8jU_$G6ezAeVXK5A&RHxJ+5 z4DDR{SF4mh26#y#q6szJizCqc^&0y&CwqV_%MWO2OiFktbtq**xnL8gEGvYl($+L` z*mMlOShB9~kAS&(;|4B%gD)K2+oqZvw)s-wBtiDP>29D*Dl}7GMX-ND*_*3B8Qzs= zbfN2@cYrirrD;tN=5!~PwF$@CHW#>tUEi!*nIE6SsSPp_V17YF|$%kxuC08Fk)?5SD1(P zD!j#eNpz!57_4Nt%d3O|#fI>tOTu7Bjtr`&{gIl`M)|?%_FZ5+@{!0MQVu?Kubvz| zSY?e33mW|87%E?lid%wCz%!h))hZ;aBphh0rkc3V_eTmFAu`_{E)I=Q9H|@$iR~0? zWpSSv2D={ngPx%E>q&UBxU>`ajrt!g_FgKg-@j`# zT5sFK4_;a@-o!e<@@2uU&_9Ob=aHh;)Wx>elO(B!2e~74B9W5F5=t@@6_qkLvRbK? zDREUZ4QFyKXvqx+0fqv>Ccc*rtMkPFT7of(Un7c;WwHU^%iqa@MNJg+W?mh zwlBqU8fC7ixP%5l)UPSg4(VWhEjmu$k|F<@ise?8JFB)RlG;RS5f!LPZl#1yx>%)= zL@3KrMqYictraa(HbYLf4I?TEnnG9*w?*X_SmPh%AFa zzsU^ga<~_^QnS_NLCZ?!$odPiw2~yqoY5qB=Uv(UayuZP+bbxF2wY;4zcz7s3|fs6 z@6DbUY&S~=yLj-HSRkn)tbEB>GF;}c`Ef}!hY6JMigJsaBK{BhUm`M&{Ngm*CiP?F+P&0i-s`%qL{=1knEj` zEyRxg*EqLhdBJA{Wz$>cGr$(;2e)SfQ3t9tl`0-7RRH02_Y{9zx|4tU<|K>Fbm8$a4oi%2S2gumBQ}Hk*xKAOEe7(H4Vzhl@r%_g zR6u<}{^C@eqZA=W1s*{~sxgzvjN~%zSSgoOK1t>p!$$qmi9mX8rs?7HSk+c=czEI= zluk4etuB&708$P~DUumD1vy7^ppoKoxbMo-5v6Sa^Oe#ntJj>4P%Y!^>eqc@sje?0 z>&wx#&iSgd*>zNp(UOtp0bH*vi5QQQT%hBg9ti}X77`F9RQ?>ruf(D@JT!p)H^peI zRwv18Z3|3B(>;p4g1O6!qhTqU=%ZaN^dVp}Ed{sYeZj8h9&7T00$q_eZj4@fSOE{< zV0pty;Nbap&Crxiv1UE4uQczYoK_6V&1i-d!{d7_NJ6x8Lfsx~HFw4yL31(S2{{xs6ml4yD_!lrs zD`aPF{2%D-C(ipX=`1_ve@|zB^RwT~?Jqj}r=|U$=B>}of7;3ZH#+-o6Ujd~=)c|i z^XUKe@<~^Jn=_8vADr`)8Yf5X(Q(%Kb;H+@DnP ze?*r5&29MqamznO%Rjm0A0Ojq@xS4gjGw6l{ylE_$(8?U@B1qc2=ia*`~K#ZOiYZQ z>?quDh3G-euYR7UE4=O z`py*&hK}<(vZa{mkF*&sg^}JbKjI*0He#dFmw;&|2Abf!-D~RxWc+wE{o=FZ)$7(N zpQHJ+TFd2ZsY3Zv=~9S|HmeV(ItoJF>-`*x{7$86s>}OpHiUIN)U}xAmk{ zgzFaf>ix%&UJ}iv6X}A}v1~8plkYuvanOo<9y_taN3!AWCg=@1om4S>B;Zc{ZYbvX zOGO~#i>cTc?u4zeoGamCf@8gROE%-3vN++hv}wCoX5+8*X*JHM-ZFrW5UY(dcD;l) zS)V(3@iZKG4xy1J39O^H27tS>+M&t$S$mA0Qv%o_EDe`*Z~XCvF(3t)BPy~GO-pJ%c7~0yfs_<;&2lwr zVsw2-UyMi(rXND&(d`v~3h1>gb;Z3n<-_2I73&H0$J;VtQ6z~i9Kb-cA~D>9g-||wgL2dM*wh(N`y+GsgYSDz`kaHfFIE20|^qM=&;Z7iF0Wip+$)D6*rc6-nXgf4R(pg=gnZK&86a|4% zUHUFbk7cERe`dutA!>_EJwHz_&|`Es>V8`}HB|_XkCYIGuS5jTszv-z*ppYiNc2Nt z(Z?Tln#a+vG2Gg+G>_fRc6*IR7tzG9@H{(xRcV$uk#xk)r9G-a4>=v=Tp2Ucz>uOG z4d(}92n{|0XfiO0B87JYVj)4qO)$LM(S-<7Bf=&!i2b|GWF^+)@l9$D+7r~fhO}nQ zI8a`*Kf2~q87-`8vR{KsSzygC_?FNYsrTpn;W)v;x67t{Nh4fucDuCO1G_6Whu?HkR9n?;OfS@$i%YT+ENFee(i0}9k4K&# zQICR{oPS9|C_jlT!}n$MgEzteh~E&=+V(ze`d7obA-2)dv|oJ59Sm}8^9TpmN*>c) zLJx*HjZu)RITxXP6R^tyexmH|3#1Xrn}eo(6Y5ueE-rB^H@7BxOL*6JoW4wYEFyVl zh=891HwDF?dpYvCTT>Xd4(@}y zde}(pCFVXwKWFS?^FMh_&NaV&W`uDj-!kaAx+k#(O11W}n8XT)>?kgg1CX`RC<2nT zu!uh?Ocjfm(84lV1Esw~${`2;nj+T1BEeE@QZ_XuX`RY% z1O9vqo+xP)#a~JBOscM1I(@_Cr%r$HsGEvyJ5r$>YiA z*5w$?yguP5B#sA~o9_D#-Mzw&FFXyNDH>KwJq^x=^HL-*Wy8}3dBTNkTbt~e5{G5#xg_y%e0N?<0Rh`P)pS_u$n!!K(==TGBh4|py z0qFvMG&$PUQhW!phdHF}k z`+0=7^kbKD6H+GIss9|MOfI$=KXG#!56Iqw-ncvJ4W|zI3Q{2Cz_pIi{~JJm17)m7 zQh*t-c@nzIa{%)k9DN_CJpc|NIgP zo!E7>?-j#+&{s>aebS1HLVZ9Q#QJ3VT>4o02(YGdXX0;&M+*R{Lco-P3^`2k`-JpD z_)iH*QWg+j77!efAQ(eTFnpk@kS?9U#2y#JUfu>@a|?XU3U+jN*+Uo3s4_mNRvvd||T?@)x71n1&ma;HS2ePUos?M#C zF^(zv4kcEYaedV?2a6tY9ze_$uKT`ke)k>riFAq#8t#q)9zGp@ zH6(ehn+)Y*&Gf|GNK0m72W$98L2uEKYSN}+ptqV{_W09)ppNXI~vPqFN&HZQdb2LnysZ{bt#fI zd*q9%>WG*Oop^RkW%O<2H<=L|mu@N}P$ozDS}8V8wzEQY0v@`B8%+vkG&-jF@dn!V zHz^}v>9SF<6xFossaO#VBy)QU8eOjL6!*ubff{TtmE{!JE(A5EwF$N4mR~02p|xC2 ztXs3qknN+)qVpY+mN|24lzCek1dphJP<_6?*OLZ@?9`J$qex^)iaM~wNm|pw z1gKgn{~M%AngBcb94(}?70_0yz#U{0SpXJNIb*=GRPs4yQ{ucS9lczBBl1Z$2v(B7 zJP1~nz&?_>H&6fH^tZDBYBW`wQU#AgEyHo99K7-4;Zfk6>Sl%H%v96r%5!X7iwLFBKm{q59 zK3CRU-)I}-yS~{v@ik9Z;9sPvoEIP3?1#CjZppYN(=4g7KtDaI#c6+dnZBo;)^ah= zD%?Am!D;oYUJbuOw_xOfEAG>;l`8+t4tm_TCcN*xa1Pdyt*y&AkteP+aJ%_6Q~Rx( z=F4U<*{&DSjM53+}aGM$gJo+o#=mFA!=LDd#0 zXY9b7ryy%>wf<8K7M@5x?Vz5`KzU{{BWgbCtL?lT&oWgaWu1$bI*?Y^>jWwr*fuVB zAX}!Qi^Ofo*tclfe(cCKee2`woVDGoh8fHpX>NAc9w^)Rq7RWoYFDDqcNJr@7J;fY zQm>VGl3N^Cxqj1M*(l3!+wlm-3Y{hW%rSkMm(;7(2p@1Ql7Wi+W&H3uAo9+8l%xEeLN#-#9L_Q9M?UTkJyd zN>gpEWUHiex<2J0ZxG6unoHcJ2I%4=NoVBfQ=6%7JM0uguJ;Z1h19W=gEPlR=GZwc z8}I4z>J4hDz}5A`0(H5Oifz9s8!saF_J^W(Xt5{A*L~0E^(glpM{m{ay;9uWOV>FI z%+Bx_6IhJVI7WU`Be)g+@0SV)`!`w=^~^epikH686W)xlolkXJs|Ft!G2%z%^d(PHE> z)Kz&rRp!SK2g6`#X)qG=+?IC*69Rx8UmkvJkY^)9nLhkXK^^;yNJ-zlbm6s5W_+*P zs0tNbQ4tl$!xuTNzr98+aWcZ(udGK)4c6&Te+#smosD9YCqQuphrXc#C%=rWNWi#} zi+#$!sXrN;NJl&Mm;_~{lJlYsI&l^qG+rGb$2$fZ*LI3KYhi+(ULqjjt(>pqbP5kk6vT#}( zALqmM3xe!Zb;oue|IKc_-x^Q=Q_u%gCoWJkSm=yWX5M}yOlYvb^p64xw39AAaznt! zlmTHhP0N|{^>e1OJ%~7C!zvmTY&RU0 z?%pFuSkp1mMI2iGj0T>~mzSIH{*f)k zUa}`4wEwQ~YQ|WwK=rpTo>j%gN&q`|cnvy=02|0SC+1JlooUcQ(8pN#VKOWqQq3bk zRxs0b6q1J}R(z2mHj%W2T1M) zK_Uo$lss3nASLJ?txwy`!jiPbom@?jmE`3NV8e)(4sLv-sPglTj-$Qepb!GC`9J9u zPPF5Og-Nob(gYhDMdqi)dJbqy#PR};4*1~0c7*Ajr)%=cOM862QWqam?t8a#e5;m7 zT(pcOd!fn*<~rTkC3TrYp^oXoDerdqEL~#sXANi7EY5WyeRul*h&)oTwSw|qG2bCF-+pB&!Gb3e>T!n-_~>Og_O27f>cp3 zi#5w1eNa)Ew19fD+$b?4&)^`(q)u1##~`x%NnrT?J)!eH!C8UfbEs{;mN@AQGX%-y zY?dae$d7LYgIs5ay=YO1^wWW#CkFX|f8J%ZO+Mw2Tm}`nc2giRMbD}ZR+St->z7Ii zDIJ`5psE9+uc- zX&&RGqxAtYm6Zh|WjxaGBYH*#)`X!5RcvI|W`4PnYJ`bjSAn% zS_+8O&5nYzx)bNkpQYmqW~4!-uOZ}I&#T#I)34}3mHXEFW~deXsg=>F8Ft37{jME8 z{F{p6AEl$1-63N}xx({mUs%LVTv$=dQv!BI0OB0CK40=?UpR@|PF9KbTTc=)L?#a> zcHCm}F2|%MU$LcH)|g>uUo2o($Cs^{?FltJ0>syaUjZ0=o*sArpHO`VY``n;%~sAhdJklwNytgmHAl0G0$Yq?LdWClRlX5D{h;n z)v2QxhD4{&Sx@I%38pn*Zd)nJ+1WIhEXnR?Qejl96i`ovWyy2q2{H5Tg;k}(k-$5y z{?a~CC`{3+h3`FO!H0d~04rc9cUZNrH9 z87-BH42VN0!H|I^iG^el=`RrKnvDI!05Am6$X3)QA7-Y4R7g=K_r6X&6{c0lkeF5+ zU(_^GyJU(+Ecc|@9-f_$>!T|w_=MQ^2|q!p!>(icA>?oj2@|@z$`Gl`pVfR4h9LFm zu=NA<{5Zn4({+Bl&K5tN%)swQI0?h5Wn;nTbfnHTIwDACAMUyF`1K3bH|ZXs=u$An zvmEeU>mR!xQ#{j3@(z9K%NJQ%Fr=z#GmvfUIKpY{qm?(G@(ss2}#fFf;;eV?c^;~4Bpoej@X?sk-djAu#S@5JC z8k*Xci1I*Ys`!WuUHRaQxXH-v=^b5&*RLN7xrb;WWAYnG2XcMq6lqjTA)RMfBz;PP zScTI3BPp_*eeVM(X)I_E)owI+>Wa`4m(>CN{KXO85t6tqxTl=nwx%DkM`t>}T=6I# zTO8NTNOaW1_7%tf+60(yd9ckg+lSsSjMa~%=!v9P|3tD5=a%9h>W&e%x|U!TPX75R zF8PGxb)W(iurrYKN$_I7#7tQk85twCT1**@#6ii2_r%#v%Ct@1lVnbR&(*W z&O@|&x7z(?TX^cmZO`k+S4k~Al;pq{lho%-LRsoIaLW0)e73f{roNJ+PsHtzXC>7j zMWz7y2KX`y17Y8~jV8^t9+ECmF=eT|_(nDy^Fd_be)>VX@vQ#+*fu@}R@9 z2)y7$TerZ=^#*B0xh~zKZ>kYKIGhmf9O##kHQxAi&-!~QLY3VDj4l4S01TvMuA6!>GI+PLIiWYL4jf251S0axhX_VRDY?{WdrxS^lHGJeLon9+3 zK9~8S-@~!I@yiJgz1RuP)*e1lCRo_Ci}K3+kXyyN9#dsKkKqd76eD1&rK7JeqNA*# z@2-P3BuRy%sP-kgfmaz^(m;L`j~O@hr4x>^UfdD?vbi)-y1^r5yX}N&hr^<{LfkX!h}LQva!u2 z3g+-sy1MiGCHDikWnni3;+j?lf>cyHGTkKC^clYr);?`SC#4~mw%|3qRw}|vaFjg= z!<76>-~Cak%M7lQK))s}oA8Fs+u-Ba-9U1<{nIL4s6l}SVl7|&(f{p4c z^t;UV+KjhK^A5gCVwS2iYZf*-eSQFa#`B7;$b)Wa_g(znTv!6&e~GSGRz4xBh1+qg z6>^sENwa1oO(s(qLS~``ZJ=;e%;qPwKegOVe6H9;d$D{9iO4_5h|u!R+Dt>~zt`p9 z64&{X^-2jRyiDA~5o`Xv#@C|oeGm{P+ZK39KRtuuZ@Z;${kmK4?Ybhn+V&whgAOg9eBB+E=av3?;o&Kn(ZOnrVz7A+xe-@0~&Em z(4ofRuNmB#wwZ1fZ3gN-8i8||Y_;aKQY)3r5#wG85<#{1sh|b&j-`5R2_%6dH5#?k z)kTtU=JMrQEuvv&Rw<)V!>lN)<|)g?(W-f-vzILsU=6`M9Dh} z1%{`4Mt|Qegf!Yej~;Bu1W+mZq>%3tv+f!u{|wQ&xio#D4?0mC*cf$+bPN-;);KD1 zXR5MxHkM->SOQi6 zJPfGadO8jl3r|1MY_jPJ~aBaeTykWy|+1;saJOR`FB!RCn=G+^-?V6vn0wjOp8)A@^T=kK4{XgFHWZ zX4{8X3{~+5NM)U6039>@ApNFLWAL#gQr}a*X@yg2I}QjZ?u<9HI}+^5A@GigxXiUz z+qJL^xo0T9uhB8l)<>&Vw3c8`Ad;vtHB}Lxu~LsTvKo4zm1OeA;m_*U zhHtlAJjACQUkJS*zw!Z4u=zaj0bE&hX^ln>D^x&WC)F^4$}c%~K$@7yT;^537$#+R zLY>)?Tq%T^Hx7E_RB|M^=J=N4YThd(g7*DYak$NLersa(SnK%+KWw2wONn7W7%ptLZQ|}>*8Urpn#bmvV7fLEZbA0g++YbU zOUX>}IO@Tahf=n=Oxj&r)anQtnSZ9kRh&gBxjDmLUYQqMPRq=-igLLjv8!aY93MY? zvzv22Z=4OsOSTBWn(HNlLXgdS^Bnj!Bf=?)S=TLqu2~VDX_f2Nz-i?f{Byz;E_{ky zIX8I$*(#iLlBPK!8eJdc@=OqRdjpzVcJqqL8%b?Eb|Y);`^4&+ELM3R$YxTuj1v8{e)!$|LQY>Favs_r;vWJ^us+r=AkHDqN6w>-L=;3JfH>9`JTgl+ z3V_JK67*WvlJz1RBI-&HLusqYJB7gutXQ)I76^p=M0flAEs0JA4vi@6Ytfp&e=v(I zcJ9cmiiL~CL6Dlum8T>w^PFi!iQ(dfrGe$8RjHNBVqK}F&7y02xr}*{=ljuRhrv?D z>#CB4t}bn5i2W=(etzTNJQkuTB ze{)!?>4JGwVI;eP5@v|v)Z_y&QH^prO2S5VbR}W=AYr~^4Wkb{VGvtJCDI0vNt2+v z<#3OcHo5Kh=mJYIk(@!@M9QgdKi$+>>p8qad+2`Pc@C^NXr75<3==4b z@>B@8U-xwv<1Pp#n?9bK{<6G5bq*%eagd@@Sj2-A5=f-taADUqMWQ{yM-VAcB`sXf zK^+mQY|2ppYh0zz?F5O@v_(vNhFZ;_M{c>%sUQ(kI2Aw6zNI`=MK9*XVn~S-v{f+L zw5wT?E4p%YN0muc;52m>BFi;FIHqR{D80;50N2!|Q_G7GeMGQDWhdo8;6#<;urSs8 zf{!>s$RA>Lm3>@wESh@vE{(wH5+s|)NbVq zJPM|UY9<&VTpKY$LDmA&Nql}J+f?J<<|-{eE)8;cfX7}SQ)06;y+s&54)D$lMqWR$ zfxYSieR)pk|9F;5BQfN$xsKs|{S-Df;%u1rt)s~7sp)l2QL&+QA47I^spfJ)l$osN`vYJq^WCp0-9JXY3g7*R#biXF%ZTa@T_l-FTFN7 z{UE~fVP3sThW3Y@1@P@SmPQ(scG^ZZkVf6nN2-R`_bo+Or^7-q{;iQTUpt zESXD8Zp6gA$Dt6Xagb`ur-Y|uP?3yW?d@iNcCsZN=+{+SCmlLu86$kL<~J%)8+M`;^?nJtiqV@)u~De zIEtX|cZvwX?QlGDFb!j$cIYy9{A9D%nl$=pMKn8muw@Vf){!8oT~g`9A!g%>-6HKU zJ(Aspy~E+4q43MWK976a!oG(B3r4wJbUqe(VZ$ClNab)9iv{iK^UloFy3^{I0=&&& zInlw@jvHN|1l44#gKG&rJ8BPu6MIrfXR@A@;(AmbsCJ%!iMo#{Jhk$1avzE z#r+|rhY3l1j(u%R`zxt*)_wf_C!<9h%O{3K0QPJBVN+Pgnz_q)B8&;F=9e+4PZkSB z%+@U`&Dw6z*u`1h%EoG0$T$s8rXPx{RuCFnP32MX=eh294&CN2O(H*cfIWJ=fDiL4 z%w&S`w9SHsiBgX2mb#H)$gpURD5{ob7L^FrGZl_3TQ;eEygx-LmccHSJzpshW>PS| zbUfZVa`vmIuZ4rn12tg85`*P5MI***&?J4{D$o|$GTP8cYN8Lgr%!}R!BUgCf(sbq zlde7}0M9I6X?ygmKc|!BYOq^PoO;A$H=A_l+3mmW*$ci4I@kPR4CCHq?}c^5S$uT3 zXY@%+sDjT~9p9*^%f;_&Q|M|Mc_U|h_zUgi zq_oe`@fT&N%7gLd!fa>~^4uc`P`9iGMpnURTg7-GjVo3Te0&9qK+;S_@H`9F8>EjFx%aez|yO^RuDj z8SJ2mO<;Sd(OKyiS>`K+mXyJUXfc~NIUX-(yFHFK3FIuKxNbP+x6AK5&eo2$?J_)) z_(O7GRV<5&De!5?s3YT_S?3L$t;cSo$8xLP^R} zmFokKalX}bYiY;kVf+ZiT6|s$aJ*(IJs2v?gj;-IBBMncE9(fdhU4bSGFN7jO0lQj zRF;4-6oGW+F|V_2%?#~Q+R9l6Ij|{2Jia3G$$ahXg;PH@Iu`?Qr#{=?*wplDJnC>( z&YlFoWz}EGITALxH}Qjo%l8p)Vjb<^U?F(V@jv5q;A@(>IW)H;54J*D&5oK<-)yNZubo3XUWxVAndQ;~v>WLi6zPTLrqvOlR^yJ|9( z8G|)FagQ`>PEg2%u&EV(bT_`_X^6cv!1y+6AMDFhb%CD7A#RS6n?`Sv+#Pmo@Hsf? zVxCh#VzoyoIv^cThIB)t4YwWDlDRaj*L5WAaRbBQvl40JExho8VYAF_xiNBXqr3S) zgUHNt_pA1N$Am6tf6SX--G@`;(`+T%xzp|~Mj0B2d+mDe>;|IWNuXkx6gxKK&ykZa zqjT$dAfrUP3cDJ+U@}f$+eLOy4|VsSZXa8V!SE(nJlpgv{}IEEBv^?0(cd| zPhY9UZ)WEZg2^|!E*_8FnXC6okIjlYb9us57nW8$=eVnljx*5_)iFNfi01wgMx zxjNe)l%ryQ3m1018oHiUBV21qIbu#X57QnJj?~WCa%^0Thxe7`9ro1oEt8}PtI*Q)v2M!nq zyT0PY`(}&xB~h&f$#67A9Mq-GUbIr0q}hB4*4;C8fku)a)$>wdP#7&WV1lqe6l+QT zOeIKPitj9$T#$<;+VQKL#Jw z6VzNwCS}!gmMU9|ibSm(l|J2i1yeo0I4&QL^Z*uxWqe>Ii$fMc!d*D{RMc9ZhIf+~ zG;XIKbJg3Pu6wmB^85p@5AyB{s%(cbT{QH&{1WG~`KBcB;_!=!#ecxboJ(R0unG!% zit(h7_ju&a^l@Cr&0M)0IPt)DHNSnn<<3p3Jq@F4_jK?6T6sT_$P-9$zBh#LwB>w0 zJs)Lf?W}n9dAPn%n|aQy;nH(gVmEy>T;^CLoo5^|gJ*eCk(|B77I}!I3C=G~HA?2S zC7C1$9@A8eCqdUigJdWENTTFb>QbWNsm?5ZNEnpP5l6Sw4K$N%{=Xr(Wa-j@; zC2gk*2#zKQ5AIow$o%q(wAz7JFetjIY?s!aSVPvX=7z)^^BA%!%<2kXVk?JV+@w%Z z+SR)rXCb{9+lLKdlsAcE){bpPS9^(3&OFXB$N?ilZxif zJyPQvdvuMUnCX-zo_S0vEr1a#MJyd|j|`HR#v&XbBJ4|;tW2}mU zpM7+qU@YBC>FyVTa};WN(+o&4t*pB&Jmx&oQXQy*D@Ca9KB&3i@>^*T{#AurpJML| z6N2ic#;hQttxed{?m!bBP&sX8#$XZHK>f+4rH@>s2|nXW!Y z0Me3QA0$&iD}9(y0xMOTlk`lRdda~RXR|zvO;dV?%zWyFekkHF&z7dgkCfO=j;t*2l}>Se_Hz!{Pp28;i;;5#{3)Lx1A51joe@s+sq$o(~qcTGx9iV@!Rma$ZLJ-|Pf; zUJfdB4b||lCzafst;HRVm&Vy-#B%k)+srhf5A{gT>XvtUZcax1oKE90vXy8v5%EmHoTr~>vpUgYaXQ6 zMaHkZ-P1;C-jz@4>k{L(aqtdeydz#!#^TFQ;zUpg9c4Y4Qslw52UU#o)vGO|;S9==2~VzAD0moTy>>=7w#D_u#KxXxNVUD57^W7sTC zwVw`^V+Xbw?A54?x-QDv40wxcdUHJFSBvIeQm5d6-l<1W=9N0iahF^eq@~=)KB+!| z&5=%XFMu3Jk&aH&{xUYpdEe35r5@QAxGMdz2|Ld(6;lS`UVl4SDOk`Ky=b)rr$Fdw zSYjOXoCY9wSFX5`#aHg8kkt_N!=0^wz)mb{k=RRtTymM%%!bsR$XlBe?bm(zd04!I zj3wIcP(>CU8%Ci~Hl6eLPaI~+ z&FBky?1t;&7MJp21IpD^zo1HpedY4a&Of<&ic+EtG6;WLo5Ba97SG93|5;_M7FHI2 zChQH(yA`PtW-MEe*r;o+`rdLUbWZ=~lVT>L9&|t=;RvRFnuV6JC9Ag)4zuGvlPMSQ z6l0^wAIl;WZ}Ea0%^T~qFfjmc&f<~a%xsl}j!VrD3Z?tEcwu&_j9u zOM@NMT0TC@3SrP@<)nsJksn;`|&~aw;JmX6TOPE2bI6q2W_;GP{d6<<1=ezk=; z*@+gHy%48%se6s}Gti~Nwyg64bzHkEV#^ku3}5o9%}3~lHvz& zudgqGFYOiTuW+*bn?BnVSrk=H%*0+0GIuc4OOOaW`&4K2WgWK2!|Q8YR4-QG9xErc zJmMyPkC|x@b5&0kCVmtl`P6+uG$X|){;&&}nXfGA((_j5BJOL4cM*5AJb|1WzxH@J zc#hbl#`+6=9h0DF_dc>`rORX#nFI2u3n{2d=OHE(|o6X99{<(*4p{kJg?%}Wul0EAVE z6dnN|g~ZSn=T9#L-w?BiN_r?h4Eh#|s7ZHxyP}tp4@@z1cC{$r<6urgfRZMRIM(Cq z7bhMGV*0!)qDaCU>9-13NXlRUZ$;Zbp5fS=M4NQQVA~}5V{ zfJe$Mk5?2UzaQlL4V+FQ5sjkyF8*`^n&UJ-V&uFR+?ew`GiUAY3po5ar)ioX8j8fsA>LKfzZ{!O8GA{=BT*WJFiyA^Atm!+f0hX0-mqYGM zfciA%wB(pk5_XCpY%-*sGimVB72^$raq^#BUt>Slxa_;RL964*d=t+Vx9^7Y3H?b> zR$_}Ou1ZlRIdoqTG5uLyl*PxoO-#GvrDt{rtC^iklTS?F<}j>UMW-1)0yl|lt*eVY zyobM&eSjLFTHQ&X~=EOO&6#>9i~^i?!s3_3_D*O=)s$5>IH zV*=N>DZ5+Ngg;m~J!%@4auVA(Tfru0-R2Zn_^xl`_E(Y21~T4`hdtQyWnxh)co`NR zyjZKPNu5G1cn@40x+5Jo%DQ+6TTdo9BwPOSIYd4!kWqn!%aBC9F1=we{~U1kVxP`p z**JHkTcV7FZze|}k*YigKRsY>P+Hu^m0a2aQGS^jbK%-SC`sV(#ZPZA*Q=MqDI=9! z9*^NG+{rfZ9x<+82xPYNn1O839NW?1n(wk z6@ihyF!VmpYA*zr;4u;MX#XH=WJ?R?V%m^!0Ur^)X5t0V*I&NMr6$0N@#2&a!YF zzsB11AFf%L4AnOfusGk{fu7+q+I`FP^yPV1mm={_5roB8*mv>WAxI=h1Bf%o3WO)f z9l~(&J759dY+vL~B|dn)4{Lx<_f7*}D_?D27ZBZd?O@TUzQC9WdQ>^kB-kPlYS1M} zHAL6-$wd1D9B0NKUNi*|p! zZ3B{_+&c`AG(w1jD=5L>XDAZk&Z@@S66uZ zr5<^e`_UEq(E#;}xR;S5d7e^Hcjw5fG`xKa;G@InjM-0KUXfQ%c>56Cll4G;yel!f zqiCI}Pi;-2?&6VG;Xu)~7~OHSjb-5*?ez=*;B&XvbN53EN6%oR+) z0045_F)#rBN|vXlCaj_+Kqcj5U}mTf%=l|<2xQRv6JOpR5t$71z!v|+n8yx`)%=4o z55WBI&`mKQtKXlv^L``o`#m}n_P5}_KXc~+!~A}a&SYYEyZ-YJ6Od}}?R=BHJ!Jug zf&N|!01QZcJO5n%x9dMK!U2FLvatTEJ_|FDyY9EvfN_>=ZzOC079hhN&>%LzTU_iL zN8VdxBJi6D7=ihw8K{Ykg@phBO!Wu&E$`u<+hSn>wh!cBV*w^V|LvQFm4FGr2E1kl zo(#Y?-r_b{*qGn0VA+8Yj&D)`Ff007>9>=a{;f8!XRQBv#sUnUWo3OU1GMXnX^-_^ z_WZfkH=F(+{ro@H^k-hc-{LV^F4w+0H3#}aGRQ?Af z5d0S4U(LjXl2j--VYKX-yGWFmerAw*;5HEnt$VI~J$kYqV0F`tt`oc{c2sx2&` zLZ;p&4%z5Lv(;8dRY9h^NR$rplX$7flwU8mOPRzZdr%Q};z|Pni3@#8oykgx((4tE z>1p|;J9>?85j9F@%TWWG3$%ECzg#12ifE+V}g2UUNd~yrQ9GS7OV50 zOmwT=Mw|_|9SeBT&B2|alb%}imA7|wm=Ir37??xqY z6#;otSD3RUffeA?=&fQJsFbYcVSMGVdsT9@(c9v$&5jfcZ1Dl30%H`&ef5JMxD@q* zJ00UijK)6~#izdr?PndSk(FCs$yOQib7RFRU(tY$#AQ&#p__gYMGN<*J%4Ws{$yFk zF#z-+2=UJh)cSP_ z{#xF73;x2?+{w#-8Y(IS9cd?h`elo?%PsXrp0lijW-6+4^CwI0W({bocghO#PX<35 z1qQZPaurAjUdkudJ$%>lefVt5*{jJh1{w)j3^K+ zcEiuXEDG321p9pK=J#l%Gp7F>`{E|kG!~8%0I_4bAn9TF-b1)XwXZ59Bc7p3NO07^ zHQ~YNJ@rJE>-*)#Lr5JUWC@m-gg zH*TO^_Vb3$cPOqfK4~9{i)+lVVBP3*Q<`T8=^#&?xcN)C+po5j`WCmjKpvW|jOGY0!)2$oakJ{FoB2H$07)f;106IKoK zHs^`!oMn>ID&L)M19YHA5^HvVoprseA*%ues1TdMdWGyg%W_nWM?l>M4|7gP|5S9% z3SmaN^mYuQNtk8A6)#ghgQ6=Jdbsj#!|bG~dZk6scMfl~+STiDyzxt;Ke zjYC$YzNU89gml~=O4gs}u9tzXrIF(z=nUjz+jzfaOZo^-TUjm`>Rfc4~ z^K0H`E+PlSAsv`%6%TyI51E1>SA&NrQEJy=L9l*2e)h;i=B9VusM?W2!dR_E#mv1_ zXvfHRoS+sY0t^+!wyM&S6pB!Jx+5#rSkqthU^W&U?J%e)#R({+siCj@ZnFd+N z0~B?aNLnEs^JCiXYY52}a9A#cvuPYb+JM=gXFasJ`L75b-AsD8!0^ z^5hw;15m7V2fI8GW-qG@e6b#OmW$WJ38<5$DWtVhJ+#x6DyrqsY?((N?Rq@7%o*w9 zHZNH)jaw(}#oL1*zNB)nxB!ty=tvjIY};azy)^h)5HA8F0*Y(vQBHeEeVrnoi%>MX zwolt_byb45WcDRg38Qgxe|*tYYSyAywZjN?U1Tzhy-d|&xV+Cf7h8$c_~Ofg<*;xm zPc8(zNBFc+LATF^@8S>>OMyBh%_fael0&Aq0C!GiR$_d7s9a@s=04A%*V(+bIYz4s z>tkj@w}tkOBIdr8YeH_BUpYdWUFNz_i}a!$DEx&fgE=c#{Ah-M`1V;`0t!=yd#wJ6PS=gvzt+9HA}UGUKkPfr zwLUZF@>Gss0_IAX4-o_dqwtELuvag5_0SN-EeR#Zk~cw8uIcx z!j&(tac&DmtOY%zL)3TqhNJ*QSbhS#n8wweZ84SSOmNM5%9mxb=?fiYu3T# zbTZAUhZmFro64Jekh*1AQq5&As@m3#l94VL9v4co@;_o){ry`>n$tR~c13w+r{_Lk zu^qQsMbr_dGsXF6y%y8YYQ%^KMP$#ozIWc#?ZGWPoHH#Wz$c{=7=^Q~uQ$0#%jd?$ zmFy@F|FN%qX)>I|%lkowFsa+{fW;)4mB^>QUii2|AW?p2a7awmHcYyix2o;5xBv@? zbJr4|w&u;f-*LqeJWpzAu5Cv_`R4LDINfY8H)a#MRE}+Gip}^6lZBr|)I?E#Gr)8} zH$z*V=PBCkc}1(GSW-e@e)~mkN-cdZU03}GCU3V0O+vNz^M_w@-0E0pMkMiR%4pJE zY?ZvcqgP#yxzxNYGL1D}Yu>9US*NiKEGGJWZsIakb3JUDn;4M}apuUf&B}h*6}9SC zgMBYxF5h&#Y1t^ZZnwZCoxYl7E=F5_ZVeB@^D&L9){yRJ8=_>aK~8i_2l|X78dedRcU@Pkfm7UW>p&C<>ueI z1bOxeF+AjRw}6}1t|sQweAIA{6|KL=7`VroTGeHGLpY&0dh?yq=NRCt`OHJAM8&xz z=0v6`_pl)%l$(z{e7-9*4{}A9$ZOlIdQ#eHD0{kVp-s__vNw*q(wpq2Jl>R+NAkG2 z6xEfDYOfYqlJUqjf2#Ab+{@@g%Fp?tXxJW&XPt@!)6f`%SaM@KuFLu4M695k ze-NppB1%N5reNLPyn?NV0jt5PEGYj?X?Zc0^8`jI_2{6EKm{dJ}62Zx* zAXl`rmKJ!@MSRkj(j-_FpqZolFwExfcBUzU-XwpNWj;I&TH!w{ry!Jq0?rj8E|3K-BrgHjvmno%wl>z7Bcjs1%96Dii12?cu# zLW!(r$T6@^iw57vbfBJ9UZOv3mL)5owWKhD-B=)%-mSW*dY;xT5Fpd(s8; zWhh1${6_WdD3-oV#lPnO*J5N5mt&`cW$W&+(D5=-(~j5=7tAQ(-i1*m8n_r)*+ONz zCl-YBB41#-DBhoWERtRxe?jpa3M0%&8^~ceawI$PqG>us%Z-}3s>(qA0v4bx^dz6kmD0kfjZ-+)UtNSg*l$JdT= z(H3Pc&9;O*!s7D&sG5X#tB zvN^(pHEZAl5x*Gs_%ULfdu%?r4+!v-2gJXhtM4W~k@(Dte>x3!jyn=!!_V`87Aj)q zANN`qEqjb}E-2=fhHgtJfbh*FBNQI7XEWWi>fm=$$zJDh!%uqvY z%Gb=R94M$S9(hkYBel%Q78HDQ=N}&wQC4ycmr$nZc$<{l;dYo`n!(6K>PqRU(?O9o(Wq&j)_?UvPu@TAs{KL#6C%YPLbmF+SKxg6M?D_Vki32CB+_J>#YFV1o+ zP{T9vOO6$Q%N0Gw{tUq%B_KaVICc$OR+n>T#PmX{N~3!>Ni2sc;j??B3;|LDyaTf7wf1;iWb#XWnD$(_$H&B z9Bkx_Nn5X>C$hq_C%3unC1o`8v}P{M2$j7G*sX=0jN`CGp+V2<6Wg`Wp=Va7_|kpb zc9W{&?~H(5_!~us&&~5;E>#-t&D_CzR~w$!M2xL$^|)UMN1L6JbOAf=NDgJfN5D<1 z{aNU2#$w}RNp>`t)eYzh$3z!YrbzS$|GNXs_%E~{hJ~F5zy~TKB1u}hNSdB|aS9F2 z#BfcNG1#M++Q=>Gpz0BWdKevsiz8vpqGe2Cn)4`PMMU88cU?iXuJiFp zGp}5UJiT1l_IR8Gez+0o+V=Jt7^+m9E4+}j?9$%#c{K9d=QE7!=1kmJ+=_R7Z2p*R z2Jk{e4r+r<+%PfH9ZPe2p8i!)(Z1CX-?-X(b`uwvUrzo@?#xGoNAUbO=FBMNj8^h) zP{#4ywo{yoJP-f9JMlG`%&uLK=ashmk}&hFL-tEx*j`1L=_m=t!K&v6uOZ>9$*?^% zjERfaX}G?Yl6rT#|@nU#h4&loZbJIkLj zWac-@wf~lH1pq`rY2M&URsb^r13lYcS^boim1M!^?gLh=E|}Z(!XY^nE~B`FDUA@IRy;`PZ{QfBqSK{LA(KPvG-S z=N~Qq(E{-5FPi>Z?q7BOAnN-w>D?bw^uK_ee;?Vur={!T^My|L<^4pqt=dLr(5!qUisA224#tAgf5A23!!d09qyj zc_)2)M|T1_ArV2~Y#O;b{!!=m&XZd?oBSdD-5)~!w-$c)h>#0$Fw!&7)3eaCGXfLx zFab2^>B)fq{?PEd?|}Rd(eDtg=AVG(e_Q-pS{E}%U<#n$ItUyoekVs$U^XJ=H~WCM za`rZcP6ogw{97l10zrEt{kL>Q%>Sk*{O#V%##%_<(Fn+_`L9~iHimEa1i#-)8o9aH z*c&0>e~|(CH|c?M_ZJ!P2Ix!pn~aJ5&8za)a?JF=&Ean{U{a^Q>tkU1FLhbj zfXMz|>oPLG0qcL2y)_G5vv0>A{Q$7D0AJ6)$e5V{Z@!?v$mp3FfxeT!$pGv?KgvI3 zY=5_fo|Ww_3(sHmv9huK-9CCY0Q=wjOV9qs$oaRrY^?w2FR&cY`}5awz{UHI`2fz) zKjxBwp7Cw5ydD1;7X~&)`hVzSWBf;d8Q7Qsf47f;ou2*gGDarWH}BV9+5pOcUaP;! znCXFjzQ4Z;QPk#_r3vT{}?Nv?9Vnk+Uo-ogxSNsZ8VBz z?!Ya9jet(U#>SBV=o9>HD-gFfwjp@iU4fg6xDbIR0T%;1lfm1o!v5LVkeShtg@J*U z$&lTc-hj;rz`|g_&I9|u7a`DoOGf742;6^v+f0~Q7+6?fNl1icMPTJ5!N5`9y&Vee z#azLm2!e7T@7}$gSU}llCo@Y!6(f5G;HD;K<^bI5+-Pkrffro7u2z-=&i{JCO~gP; zPefq!@2phZM9NAcG;Bn?K-&m~WCfMf<%9`-t3@EEEFdK=NI*nGN5^k#YiUGBC!{1q zASWfRs6+tNK}RPnLqtGi>gZ_8K}YA};zFzcR*KfZ#_Fx0!`lvQYh>@}CIvKs23UdC z(9w_x*v!AQ|HT}jRNvlS-;D!|27SI)lnV(`tRP! z7eTy(d58MW4Cr)sr3H>S$UCIp=IYNM$8BZ~n`{Z>56>^K!BhnXq&kBe?S`^9;Y(Wa zmFYv0E#JTI_JzZ0kK8B9=(7axI_|MrNBs7!3|^czN`BoxAHw z$L`g^Y-Nuf?X7)%ZhXC7t$tmA!PxJEM+SifKRtNR;`+0NidD7%kfp5LMx3psI zb?f=TddIKLZnM12c9R{Io5~!%GH&+%gNJbboxeF!b%8htc@}B=Rc4AAHoB{jlf7G27U~yy(<=59xp4zgswEE#Bqq z;B?+-XK%4@^e=;!TXrWA+p&=MqB%72c|1kFvpnv7w+GSzH#riH4MWd4KgMbzr;Mho z-bd5@xTOzg+gAI3i24TbN}8ta*tTukwl~?>8#~$9+1R#iV`JO4ofF&q^E~hSe*ZPs zxu>Rk=1kAjoYOUTRd>0Yf$OqKs>R;Np-Z?*Hw~#W4K@9W!Vf44HKH+}={trg7|NA4 zKM7W2$_-`!)>^W9pVjY2DY9q(WjegTBnFN2+ZcWoiyZ;EB8sS7Vc)eV*DvCV%r<2?T%igke>JunHXfH=E|CIHQ^TanNp?p^pi=Z{TBf>oK}ap6$r{S2e@7uwch;^wsr;esL_D^FlS> z`Z422GbrAvO(F)!MbJ}A{H;Hrna|u0c$@^S_ZMyE8pjVMJJE8c9i(c`{k`)~cRZ1( zf91fByy2{=PS2MPpi)Z@)`5}{u12`}Gd3mHK+nz=1)(|(=)1_W&j2zAb$ag{Ib$^z zAS7|P7tFkJOwU%2hkbXboNZ$MI{M1>g*fG9$K}0SWc-R&xF@i|yvd2PoaFcz47XNr zY=$P37Jq5w&mffXU7b21r2H;k>=Fj_D&sT=uj&E=dY32yM7UfsRB^!lr>~ZK(l=aP z*qj)pCps74%MS`tcKd9-3R$MNyLFnc zYkO3)>eMRgS?yVg+d*Mu)mlb8xibEy=<&nALL&C9?dbMN*&>qpVsCNq?sIg-kPhzY zUvZFUivKkUwz7g72Q*S{+n%07<2{ufFy;bg$r2JorFrye?%u)?lHh0hm2dJIA1^X~eJMkOLL6`CwkUdY-tPUG-P5H=3qQ=>$?l z^*!xhj1L&iDF$wD^bS{iPd@4a(u|eO%j(^avcweeB1a^j$}r(GmD(vpt|%4lKocj$ zjfZDZ87oAmYtO_tDBFYG=1p<*=3LdWZp|DP4O%|H8j#u+-#YsO>~-I2J=Ak>AP-b} zEYD=d4%sFyS0`^v>~;&2kt1@`7Wz|L0s>o5~v3VwJiS|QqbGY(Z$)`S+yf8Y|-Jl#)&zlSMwmmNQvkk?g|3L#iPU~Mgoi83-L)D@F9q)VR z;9$9dY7>rj;M4KUl=WsEQNv<7shd+XPB3?_+cpeK@iD&KVBfE5kRFtkdn_v$Xe|D~ zaq=Ohx_7X8r}}Ji-)ZdADQ+i(4-;WCo8h4?+ATAev7ul*Q-`C&N19=+^3(X-lu+Rx zU2@?9wM60=1(=Cqj^@(g^jS)CcsvA}I6Op}xI84QY$0@7Hr_|{)FygKK_mHVhz4uP zQTb69%Y~#mmBfUzgSk}~Tts#8ade{Fh*!HeD4;DgYw$m}r-OUw98;(PcF6Q1E|u;@ z<6`&G6*XO}aZy<%rISqKbRvSA3&fp}7n87aOi_Rd3p&bZsf;v$=C<5X-*s)7PHT>^ zfqj7p;_D85O70KS;%+~WK_IETeJ|QEv?#l4()dBgGaQtlhJurtXi)`J!Mr%)fZu_G zj~jDpj4}weG~x}}jku5gjkxa6O+o!I?+(2iaSM5K6=w6%=_E%V6$8=n!m*ur4v>*1 z0u@FSNQ{Fs)B{192~t*DosI!@{I5{X!7flqCJ!OA9PVP_tQ;LkMNayGmmtep%m>pf zrNocujM< z9jQ9Ay&2thZgD_e**swHGP`w2BgK-h&8?r1=)~o9nRVY=T+=r77BL;xHm8=^4)MWi zOisGRo)&g_^;X3pplQS+4|o5_e!a9i1}a_+8Iu*{6#1K~eIYvN-q8y}dv!sk!00=z z+cHqaY_=l&GuqP62U^gOF+-FsV+73`bv5j4B7bob4zrhsJMA3BTYa!5V=pDj+^YAf zLf&K^^c2$B_wV%Jg05TFcIC`xsIb@dUKe{l8nsxWEpkBx7R4-7M8!x>pWT5$VxD(X zoXlfiI+Ru#S5OuPBZ*xGaQ9Q@HGwpX(y`ewXkq=@d*7A^G#%iZpIshkhm*nL?&f^e zx3H&8=x{ZiAm!drwpF zrk+<9w6|F%o6>?1eyi3xQxW>9s5u5|{zS20Qo37kgazXKqKBU3m8)V;R4j4@p0Ue) z^ZC=Aw%U`n5`CTB-k#*-3^Xh%-~iWKu8-aje4R;xKz**2p)5WhyPPf!LZD>A20ZmI zX2>JfXU7`K(8fCfNbi<=D|>Z+RdZ<~_X}Dr zA|M=tf#$332CY2L2z2*7XUlb&A0AYv^+CBMI6*S9?zD{Y_jc_F-TAheg`(Jr zy~WW8enwbpcYNrCfFyY_5}927I_NT1I5K#$U4De4YQe4Xv+$j8u14Qt`>PZj%1VwH zASEL_S)P=P)>`3)nNk?>KA`)UOUSw%@aGOceiMs)i-ZWNhph-YZS7Bpxm(NBDw6xX z54^ZG%89q^>%be6Fsr}j0uGz+25MT5Gw1rjZlzZ3O4oDQ4CpmfM)&$#Uhvn&85wr= zww|ueJ}+EUQii=N9?HG@d;_J!N=!X8mW)0|wUL z+k$$3nM2C;o7m3me?oV9h(3>iTP%Itk2~#hKTgNNljx}y(4F3A<4S!1AcDfj4xAni zFAa-9xyYjgFXRgiDj=o+GovO?nF8y*FltyH$GPKfBAT=%gJ2XWI2WNzSStIIt8zTc zaI>{egGlU3De}70cH@V8HVDqi6~N;oQ*K1W4SKcrj@6+n3j#~QjU%T3fE#-Cm0Z)_HR z0S-x~wXqg{eaYhW#^v@?$j7R%(bjm-2|Z05aR01@yPy3$Xc?eDmJk7c=4X-d>HT@I zI9Q@D*fT^jW}n36^XDrJEdE_lh$w^(YSnKRn-x#6@7S7tdJgD>hQkr7hu=p}{4yOHe`QheX78|_RY)8ekCi+(-_Lx(oEgr0y z?DCU8iQs}mO3^jlU4)b@ixu$oS7s1|bkaogn=M)LRh{yWI8&H3Yl85rBgn4 zeU&OA$J~>tR}Q~sOs33sd0@Vl!j2qD?mvyEyIt17?oMVkd5E0|e0`sdz=7z>W8cPBJS|8c0?`}Dkj zVV|yJU<=BuNg;#|tY5G%C(cT#?h%vUABdUD=OU|>4d&FAsbPK)`{ z%nimTxIMeIJia)i%~~Tpe>6E?HH*T@t;RCmSf-o|+tNoUU|I)V_B)($R8efP9xYt_ zf%}=4;M2NgogBAe_Jgn9g3MpP@`A6TBUDFS2pX;(fvz3XKS!412L>R4kUo1Q51CWn z;f|bBUx5DZPzp1{Z_kN&_Jpv)@KxOV<&{D2`E(%C*!?{RHa zR*6rTQM@gD;ac$E7EACv{k&H8C`9+Y)+*F#jnu-nkJrw3H(4!GHce;DOW`L4yQzch zoJvdKDC0aFlClXyu>3mtH%0qvVisHWoDOnE3secsZa=1Q$Job zgeSIr-Q#Fj?mx64{nZtS8oBV+<95Q-SJI?2%vOAaW%3-6iNf zfMs!z98^MyIaj))hlqg}YBnpxH2B6v{VZ&V_FD15GPWb!rZs+%4|ED9K%T>+2y2u7 z?@wQo61}Q^uHM2OyObU;PXFep`KX;=#*v+3zSZQ$S>y4kFPJwDuj_MKHh%X(5=S6) zm_o~|9?U4TV2TWnwsN-YkI2u#J02Dt(D=l_jR?U<;IPxupl&%(UEZLglHJ#}x&}ry zAlkm2Pm-2)Q$hN9Dmym01${%Jf|on9TOBB$Qk@!5xmi@sU{4{q^ix4Ukgsd3KRbj% zH*604RgPjYq%$mjWE7->MaSGgjU;wz27P8DQf(-F+jHcg(32ci2;-@ zGZlmgRhUIly~F$(EfQOR>6({go5zjW21-lbVh@Lp2t19t?id6fJB)U8w$_jr;fJ)9 zi{ZCb$Uz`rizD%?zeU}a$j#S$3xaAas#Z>&fX#^kDCG%5aogUAQ^XN`M@g9@9SkuT z&&(0hS&#yRY2RsI=wu}5MFZpoq6iuj#Q*AZr43=hSt!Q;$e@&H97afK7dH}Ez87Rz zTt-9%eK1=Z=$s?4>*ezM8IQ*NjLVPPbT2b1IFvz>@Uov>uWFY9J`XR2@e}wN9C?<|d$it6QG1shsz$5Vv(uc@4`%PCF&W zyS)%~&Dz@mETf=u{X9{?Dk!#w20?$!fYc12f%8U&-u!r!|8$P%+xZ>pMLU#= zxy5KnrSyHZ%tAaVgkI(?Gsn&WT~Fx2NO1;l9s@ImBIryAlE>jU5b$FyeOHuLoYQcO>~WkOR5~Oh>frlIEM1Y$9>}n zgog!7hlA4N&PxcqIXKA2s5>;f#~C%ktW`k>Z+?D4dPJ?Ac6y7QrcZw%`g;`w$=L?U zCKZ6haf)N7-2*MOsa{&z?v$l|m~UsjLt3jU+A&pob;-XH1AT5BsI8RBAw2gUoeAv3qR~z<#Bv(6SrWmzg7XO57XT+ zU%~vr(3);Ps7&RWV7DB?LR`_eqvry2wuV{4hu1`&v z9U{iiE)dqlMB}v$K;?uaIG2O#Y@z(-z*wW+O?JMP!9*EAjlkIZTkXB9_6+ZP9!&Vn zLbe>nRBnoH_vRb8yITk&sk89|Tfz4jse*Y7p-e_?KjEl-e7A@^BBao~M8r-ITF|L; zhinC=Qt*Lut?lfEn=#7$ecUH$7ayv|K_18O(t@-?*cE zB;tTX4ZtfwOc9{OBg3}6(>F-{zq5T}uBO+3Avrg>09liSz^-|P0qX%im&*7#`$-1Y;GyjmxpOw>visrVPY5Linp}CLhC7tp{LMlsJ;+?@*VL$% z%8-b+N0LFqR}2bT0+wvM>MKwnPKgU5c$RrGKNJ$f;{s54MXm^!&}67n5Ob>;G_{t7 zI#U{#z6_E420Wbf2!r zpSD(L#l=wg-ycVE^Q=Y*Mxe?>YMt!$^fuQ619z^uBOBS2t9N*!QGk*2`>id#f|5TY ztS5Xo$>S3PxT2$whWV%T_^tM-60n>Nr}201SZNb>NMSl_K5`Sx_?5p+A}(WcfeY+95ySP~+g70eCzipY^#7O-4pn z=vQH=l8&MXZ4lGr7wXZCDvRX8jB(YhFD%rrJ^sObwYg;JqO#?uGx0|=HzRX1z@uw3 z$fL#F#Hbuww4Ge;ouf{`Qdqr40K`l&HC>E2hM`@<_$D|9` zYs4zai-Z#Ed_V5(@5BgML(*87LNr}7kS~V#d;pqZUoRiFnuf} z*?-fjyiyi*6<6t`W+&&HS_zkVwk&8(*PRF#OScSOkD8A5*VEl`AhU=3O6=XjE%4kF zAhHwu0LDAq*bL7nl+&Zv2M#PFe3YPw)kBUf*O_o36Qe>p@X4^-(fcrs!`o9d!oFQy zH4fmV{AW+AxnSNZF{50mw9`}bdP^C>+dJIPQh|3Bw@Ye87zG1V5~^lMs!J*WUQw)- zu(}j$kN9Ih9-%cr93FJQ^Uk2Rl^C@HnQ5|3;aud%HzR{R1s!7FcS0J>LOf>us$BMB zev-2;$l{t~14SpoUgU`HWF*e2qR9e(QRz?ylW;Q!<&3KS=TO4~+|W6!OozdZuayWB z#BGS&2WSehr!1Hz8{hv5s*CgyrXVBabH~2sCHT<3hLHEtxs`Y;FLYKnO~=^waGRAf z!f7Xy?skA%b0J}5v~L>tnHy;NDdsoA?^|k^LIY};+aAlvd$1tcbFdd^neaC>kfYIp z@6SBs#JtHk9ugxW|B;niNA!);!_Yv2HuKG;4%&|<=XTuEAE)a|4AawUae~UGi+!xq zWqRCQpOBx)l%RRAqKc8$_jao(DI~#jkO+3NQz~yPE7WQ5dq~j%iw}eW{DE7$Q36;Y zg`@VR(~pWLqpouUa&ug!S!|}nZDfc;)M@kN(fsh|aS%o^eRUy5KR`VPdRGb@&UaTt z6T)JVe=%WLP!gkS;x#MaG)q9Qy;A3@*#6j|a?Ym`KUX&zO{xR6RqA_5bTXU5EI7cg zi;%M|?VEXE`e9fj>s6D}@v8!1h&lqc3D48AZ$I4t27O7nm(f2?-+4oW)RMVPFM z$Qjx)5?Lo(&37*QO{@0ZRgudPZD0P%VQ-df)I-yjZ=b+N>Xd(gs8i{(!;%|`{DqQMk-gc=I^{G^1!D(_1BjqAD^PHBApWe zuXP{(kj(gd#MyQ$)$!LqZ<>~dN2Jh{UW9~(gblBS&KB<`lQh6sIcyzjcW%PH)X|dN zE3;Yn+;H2ZMh*ayK!BOa1fovmd%A62mL{6*xo;qY=*e}i?3cZ}LN@ZH8k(Yv^1Kr; zq>yLjcfi0^T{`4eU#j1i#{~BCG-=axy8r%@UlZmtOXMl&Q~&-9x6@_FvVqR}JgS^> zi3@gv5DNhh-Y6g8xqk6+|M=-%s~6rfpC-0|*4->j^zmUTFg5_=6&FD>E&q9I%j0I0 zY|6WC)#wSKjZf)ywFpo!6PtKFIe3OTuIRbn?sv!S#Ns(D3%|invdA##^!uc`Ug0l1 za)hD zt_P?V>Jq4+H&$w77~hj{7o8QbF^aXIh=+Q0vq}TlOh0`h(#nne<<2N5%Oi*2w2fum zJ&5+YJq#kqT#~Rl7*;^la#3{iHh!iG{}|Ine^xm&sfsIX6C;2Zwrt_Nns z&c>N|#-+;~wVGiz{q2SjkhWTRcVxA`U%G6fa_Wk0Q{ zfsaxAGBaK@>ky$?Hn9}}*E=*4^ff3a!!K6Xln~N8cc~bnqi2R}HeO6gNjAlMsW~Is zj^ne~K>4qMWqIXDu*H$GHm=t8uc8x3E14FnVg{VH{IXN7(ln6dM1I2zm zO?G$NIj}Tfvm&VmC{kAlUfaVdsb0;z+a^J7hMSpyQS;#;kFOJulvhOz6xJgGkd!cA zxi|&4UJiU8T;ozr&#mA{LO~$-hJeA^Q0Z};LM_$zBP&jLq-k`Hzm~f1d%Se&I580i zDivHcKQ20qxB(Y~cIo|u#GurUcG&6GVb>-9WtJW>{%sqoOk~S%??1nibL-7n|89Cd z&t(yt+nFXc#;+f~`L9S<61m6`EgBpUGPH}IbdnjXErs>jzr=gv4bUZwjxKIvrS;hy z{SC`XzT4(Bh!4zoht{{I_TrUA#uTMyG*HXHA`WwvT@Ih4*BJLsXnb>y00z-hrNNUV zI9tR4Ig2|OA`X?O73Ireto3VQImKqiP8r%ipOLRzNib`jyP2=Z@j;pX^LkR!Jr$m_ z=&*V)-PH9RAM(Dz{njEy+LJa28fy%R^Zb1Fbh_N5yBg*SID(9NcwFB7pwkSofIC&t zfsn1j<1{6AjI%an%;opI@f}#P^kZ|GZDxYMW;R;VihHLYUe&LdXGg_FEAce62pR!Q z&4lA@!g~U-X~iM-0M~RzX`iHk)|8KT`enVjUrrDb1U{TiO4M4R`U1gW3ltG+)&0n* zx&nU5aUsD{Ec*a+5=vFdial3#{JC`2SnCHJw%;N^&W^-CKWAHwl;(=A9XnxgLQ=|_ zOgx&dv-k(@eH6`PmdRvsgb`~a)}y#oqdxz*xSTrKK&hG?^C^nj=;ROzl%-hKVc8nK zAgE}dFOoXke#9BfIOt=lNg7Sk{T=gxZyJ34f1t+J7NEs zqetp|#!i74Ke$|v=B0y$c&Ia9B(>R3I=2X6ym189uGTla-oxR@j492sAU zsG#el%BXWpM!c%#4c^UAJ|q6X(BW59=xAyN?c1LTpM|{76iu%MP3McY`mhiK+PofIE2StkosKqK^=3Ljk$%wkD!(`!D7Rm*#(w+^ zX?1PA8D{9}zysnb+MhMbVfpv_*7lzZGI>tt)mpZ{is5RVwnhD(stIJM3#OxrcW5eZ z7ij;-0VYN_;`&7_nqM~mEpi0$=4U@Knk#LQr|F^GDNS97g7y zpP;E%B$lHn<>LoaoBCrePpdInKP(e95a_cJWE{IOy(7Q1&)a!K&BViQwSvYlNH~n)_MY$M8n69&MG22J zCRG%Ca{t>RrG(B7aXi??yHi5Qg4eVC)!JFWrC<)v`lAnLX#bwCor{0t(kZ{!-sRd` zHTbn_Y~$%+Wm$Dm*tUko)$`D{X5%7BU){S*uhug#FyT$4Z*-G*myQ4>IwWwPcvnJl zB)P~@vqA3RL}KI;4>%_FgT!#IEQTOCIi#9*bH>?QA?BwU33^mJpF_$pu!LU3w@`3B zZtuRqDL@B^2E`*EG&`qSH6oEr^+f$V?q-u^~Dm$^U< z&5+sTyJwn1>vXcd-&8FvOV!k;%zARqNTkN-*9JH~DS`PWgH zKxGh zt6nOwucOA40e8qDUb5qgGz%uTO2byOhGRZ@_*jz(3Ja_ zWJleMnBFFyb~2B?&+aqebu5(Pk1t;kN~_GW#wG;GK6 zi^cE=Q?Fn<8$MU_Hcf-ujNHx;%MRn;Mh~s<>4`IgHe#}}5wbE883R#CXSi^H>Y*ty z9PfwK8LPd72@j@jCmk>fW_&7uwcc9#6umAEKCq`pyM}&n9mDXGB1o zm*r)V2l<#ZlhhH+3B$dJ=Kc4cnrufo^F#l#r)c$U)vxNMnxy3jcWr|3J`-$>dtW?> zF~>oQGtauAp%M%RN;2;%$>|xRCA>)O!J5PcA4^=_)V;g9o~=oDu-cpjfQ!?3Th>)r z&IDb?A7V)-sJlm>{qfC5mbmcS+I^A}P%q<1DCDTXU6E-3aq@;SPUS+t&srX95iof^4##N59|HLkx?d8F(h(6CwN z7sW=KOAS9ybYcWTVwLbH%G+mhd{WH|SsfCcOQu{fR>J($+V! zDqqg`#cmlZ^unf(XJhp7NKU10rl<2oC>DT@RWmRvI~>6MN5b2b*w6Vm5R<&#M@MLtwYp}BeNEIy39q9c9@jL71Y`W>n=`?#Dx1XV`qU5z8+R6Kl1KrC9nL%O)_`zDV3zU1NXIQ^my&K|kOre99v)N^NT|eZh!5F`oSuw>810eAcKFCYk zDt0+Jf@b)yO{QoJItf!)7tuv*>9@pZWn=MHHy&j$J%bgT;sXk1(WwWUF)0zc|Gcuq zm>`CUR!$NgT&ruekBp?DhxllwhZq78&!Mn~*ujhf zAxPiyC?)-+$keAS?O)NgYk`h`Uv{#f%oy@V*CWdI#lbM&$cVwq&AXFE`KTt1Ss7nz z(nNyx&4F6GI3}^&!?IUryJ2O0pHVag++c-$y<*YY&NaRbbv{dr@ld#-0--r5aR$gG zk*OA9^@hd^v2lz1&iak7roULk*z9cBkUEw>k}k)JK}xg)35~0<_Wqd&_<(@JD1V=go-G^*j&wSGhHajsFCv+j?&nP4BV$^vjvpH*AI>yk%JIK5zQr#Q~ zsIm9FDLwm$d?udM2Oi&Q_XLrmnnt0TDx`8790jJK9wGR>WB>@#7|tYf$M+F7&lb7Z z1qs-f-{`wD(;Y3R%G?Fnmyy`N&Od%BRcY_LP_UVw4$xo(^Ck@8JL32r^uL$Ur*Ssu<3FGQ6G*&an> z*wP2}c2(5x*8hkDsdJ6)fkTU94Jcp@;HF&5G|%}&uf-Ji36rZ9h;9!hu{VO|O#b~J zkPgAdwHg1uj_>9J6jECOklm-Znb1@-@~(#vl64dv-@WEWdQTPfzo1L$CE1NuTAB%U zcgL@=@$y0-xXsI*Oe7HRm$of=0AXr7yz>o&C-I9hK7>lUPsrTtfPZ1Z>!)S4K)hXJ z-xa9?cg8>0%x!WR{&s_v_|K95xeMPVEb@X7CPz=mFYe@mH3xvC_l{dJU5)4~*Q#(G zEr8GRzwl%A3P*XoS+E0(QjZs6HL-ueKLvi-X(1HS+PAZe zX{9*b#(ibn|3b;p<^5w+0siI5#z=1z-hZGiQ+eAcRyQC)SHXpQ_0affFk$n_7q)`Q zgR5LNV|0}9f9JLfdM60Dz4dg@ymc5Gr)G35rvg#O`=ZDkvg^Ca`2CI*y&cH=vd{8< zOm7PQ0p~l@!A+}Ib5W&D>E&;)me+s}ZpIT-?2>2}J7Lmw%RgK+ds^(Q0h6J|r%wJK z6JI=mUYvEb8U3~RUn^hvC;6yi(!|X!gtsys_VQmcQDoKepH#!u?%Q5(z%!9;5|ckT zyKuR_1G!CZTi4PRS2pMzQm~(1K5zWTe97*t1^9eMW*hL$-^WN}2>Kcc3HSpg5xHlr zAtj$&7mP2jM$qZM;RC<5dmjs&!37iQ1ow44g$cw9X`;M5IcdVYo@{CK2K_}QlzH3& zd3Vu_7s9whsQm+%&*wT>mh&{X%FvmSaSF!k%nyxcsySb7_8CI($g@Os0M-}DW3+js z^RAEW^Uz<~pfoFA9nuNmwM~RNL7vk?mTO)Wq{Ch@1O{ON*eLCXwtBNh(8xZy;=om? zTt3Rp7VIi`HG6EgXOPDFCXy!u4AN?fY!kv16>`DqMjoXJ`zuG)<>)TSGHGfVx^3~1 zsU-uyJ?n#V*WwMqY`2knlLg-o+NYWw42etq8y4#zr;FP!FTb02{U+P6;B|_GJNzJ{ z3;XC&w_8;L%z*KsjrtLjONKVDcSn+JuOwwhABBN3?vkT<d*&!cp(VL;gV7^DIh_=5;?plkY1w9qw<#AT7Ck}iDswT<44Lx)R)rEKm*85LT ztChNB3h(5`-V&c}Ea?_@;Tw=upsziY$6M?Qil(qB|1_B`*HJa}YCm~aTMUGJz@792 z2^@D;B)Y|?P&1Z4|6SXP1C!bHv%bWj%}z^@ii-G8&q;#)fpdqX6}aI$%iQ#>v)9oXT|Xkh$14r=d##MCg2Rjv?jj954MOE(DHNmUrue%E;})JGd|fA_8^Xe=GF+7PUYbH5J5QAc+`Uf3zo#M0?+Fo9vQkC!7Su+lou zRT&gN?oUf{2JUivUQcQo_RJb}C@vX zpqN;dt|_{9Hjlv9ZpoGXY)zn$T*VQ^Y|y|TDNTg!i}|vOW7lPEoxTtkgZ&u|X+P;> zhZtTVL`*^U$1+HN(-gWM%O$t?mmWd#)^!8FQc8tj^=}TXn1yhJjpvbc4>1TEOmzH0 zI_EK>nlRr1%WKjNWUF)=CjbWuuw-~*kzv=f{ zVdg3A8J|}XT&h{mfSqh%iEdFtvOdbv)>JJMosP^Be*LbU<$5$CBwrqDIl5&$&7J z-*`l|{am$CQmw#llLFLzWh3xATk0~(bX%^H`3Yq1c&M1N6!qXpn_f-Oq$%lu{f2N3 z?=O*)e~4hA$qn9lNrlG5*pz36?Zt7 zHCqY)SbO)J#d(@&+LeQ%WJgzkZyk`9we=RCf&w8t$gIus6|vb2ie4y->1^RnVhd?V zB0E&6R6{CJ=+;cO&ASc|vvN8jwZP^QySeZ2W4SSM>A>v2Wh={s2pb&<;e8W(ymCU% ze6B}`h`RN_xOjul$p@_ref@!p0cG||^i!i;EePL|nn zl0~{J9*DKVRGG1po(kW`-IPn^d64eCF#opPj>DaW!)lKWs&~%fG?Pw55p{)2I6Znp z<-Kx~MSMV9q;VO*c|!jHf|f?w{bB&8f2(n-?(HV|eEI|S&c5zAb7URS7HRbbWa^sD z%q#XRLn%Ag*@W-Mk*c#P7+ajHA>ZC7rE0)Pnzus-=5~P+{c*)824n(gS@3 z)ec+kkPjBR#{Pa7h#fK4t3f4cU=(QLSXw&~yw1T!xba}%yDVB_v_=)*n~xUsn?Gtc zx{4zHM!`(^I6y6v;gXkuC-QVvJ?FvKMUSJW#|0~70C3jG^b17$_T{^4g`{=W=ZjUz zXWt|XfhA8kf3BX4z8@M^h9O)(i5GZvn0pimHxVk^J8(pK{Tf7it>q=QayEtphqAy{_%ftPE zjYnYJ#=wx-xD>3O5wbv(ijhIVGnNO-+}4{*DSem=ocQR*LLuz+7qju;zK+(ZZ9MtV zB`q*K$ZB#Df&h7EgOZMro+mI2NI=PGB+y5$EC+x{%b5)>ei4RJL)f*zhJ3} z{;aK}z!XJ%yucJlE+{s?5-1+w7itSee7NGWy2hn;zT|94WFV+wW)F*!|;w_A@e1XDan)@=M0C9|D=@21)M#Ed9sYgDl8QZv$Yn!(lkC z3IUIv)&990OdP+*TC!6&VhcoG1cHJ`DP~Jmf`a7XQhuqzV#-6>>pB0WF+TH)acHfI z+uX(k$SPI$nKk9_t!%}aHPUC3YpN!`@Qbh!w|Z-haD&uf1F8B-EmQDKcKKyGO%8Tl zC9(ODD=kAWPN=SaTVIdhRLX!q3idXLvQW_k$4wXoS&(h~BDw3|i7RhLn)z-oNPmhG zz@&uqdGF=@mj2s1#D2#(f^ebEn3iA|&}V03GfXW)iDnPPunK&YYH|rw4M}_3gzW=^ z{ubHaF}SF{NerE8g0XV19}t+I>L!d+%*@K5VX|kAx(cNioUgwTC}_IiZXJ^0UOvyS zHIXpUpsYa<3*B_Ni&6558zDd(`dh=bcUJrMe_k(04`IY0A(g(pGrnJ07MtZ0gL3h# z(Cnm9@EfHa7n&wydk1*0`YbFp`krA|mlJoMet9hI!DGKX6|&C`rWe@7+2y4RAUluY zT=KkHu0<6CHf-uI`EgC|I3SRo)zVU?^^hc%xAq?A;MG1RnS(CKH#^DtV>OjR~ipdxh?5uSG@YMiOJ$9C*p1Cg7w=?_$~b$R-hOtdQ&;_Lb>t2yj z_4+^WiVVR@#&F>-vqiS09FF|a_gYOZ9Td4n5=O>xWfqT?CaU|fYJMcizD06z;ojbAu}UIRUB z>Gvak76|fwdZxzdqNrsBLB-tnLT=XSq=UtINH_!;!NbL?Uk-<;jyP@>y%uZ{kR|TS zy%sotDn71BuwDxdvz2dQHelX-ZWOG9JNyqsj6#3Gd}e5333Xd#YJ8D-yzxoBX8Wq> zH9lmXAE71thHqsb1QWVw61yl{)c(K>m!(s>sIq@SNJ*T#Tl#)*iF-N@TFj6B{dqnL z-2-4tH{bLUyj;_*jxYrvqn}S(TP<-PHJ=PJ>BbiLy$~xqJ9m&MFS~QPel7P5a-byw z)UHrxvy3aVm-&ig@MZZ)vgultQRQY5PtM}n)4mfS>}m3$9QT%860PYqjWtJB6BREo z-RaXcM}id%_oUb=hgcb?HS&x>9@>^b_A0QLBs-S5(^X$fOuZ%r1b`FJ$V zX3JSU`1wS1d1PnNI=R`{@<(;)MzPU3^1%fYC1$6re-69LdHm0pRm?4C;4{%3bMp*Z-~f^Hco}@mquce?P&0G^jTz_^m0Y=eDCA8&DR?jRy!cM4J6VE+^KbjNzYItJ(V&|89ktJBHzK&C z^n{dGUIfKj7@lKDLNZ)YLw$@%Qq4t11=)t_C^YID#arWjK36M3^OpZJ+JPol09ckn z0%d!eDYHh=V|aY_-&+Ek25nAFN$N5uLW!v>i=7r^6#fQRf?KVS;`|v(J6m1Xw~Um^ z?jD~^Oow~)`lLf>k$tow%zp-%B|0N*hSl_=v!x;u%c1IEP9hl6-MyabnwV=6f?PPx zT2!L3St)kyQH%47*q*!;u1aU>^(Zpx;3}oJWSTiiCrVqMJf&JLE@gvG8oLu+RHgJd ze1yetkMd>)HLRg0);7FGrW9+`FzHqS9{iRm*tEfCI0eC%^|uqgJ%p~e4_c~_sbY_N zO&aZRV9q4k+lqaf^_mS7z_1(5<7E0fDE&U7VlH}(WuQ%fvrdQYUegY507HLi7+7dG zt5WW>#)N1Q{ze+jgf@}eq`hALMChSNifD$Sc^Rn7ucz+j=GBc5SWB(NHP1)}*q;U; zwYu%VPQ=$bR=Hk+ioA59@nj-=|6jD(Qp08jn(#c}y@H6d zwmgA$F3OOt|5w*n2F1~|?E)Lz0xTYaySqaO?gZE1E{iR$!54QYxD(u6f?KfQ?k>Se zIGg7^RnIwZecw#=^gYuxSO1#oo?EW#UOLt#-&$yLKtCrZC6BB` zOY8%Qik56L)J18_y+J`!W*iyD>k-lCgAbB@Yc+ffuXZ~y>%62wKx+8IiK+N7o%=sP zS3do|S|e1$uoYX76Y&MPH(T$n5?^~u8w9MSReN+&ZcXu&NF|TQywvbZyuFLzwBX|- z0PWR)fZw6vmB{T2#c-n|z`fktSGXs4+`B=xx^v*GX2v%u^mlV_lAfufIz;=)YJFW{ z+zG#(4$ChPgK9bNFoLvmJ$-|G@4m2&x-WGJTCrJ`29D)pTpYEDyisb@=GRN~V8jfI z{B+xg*?-8+DthahD?s6l@C1JBHCR+a<%RS^@-&cGF|jcRO^0gIpgyOygKIE8ozTLD z-Nsju@+5`?(TN#J4K0m&iWiQ*bx*ccD(=;JGW;X*i`q)m4O$m}=5?%ufHWbJKbs!c zAzOTlEu-$P#qtjm>l5z%HU$EhEu37z;@=H-ejcw*P@P4|%6eHncJG#@Nz)eEh*i=L z8X9}Ulquh}7V^ENQvKz!0FlGNGSw1U1E7qsvtdF#c68o64-J8m`UqbJYO$L~+M2Un z;6wK6wQ*JuRI}~50AJ7i!>|%=MOS{jx9L|&+dUWOK}K6{$~@yxGs6UPvHY{p1#YGl z7wQbvipS(sa!k^s^oL2`%~f%HE0K&np|^FmG*i}$Tym>05#6d-dcSCii3yjn*1EJx zc2E4=8+{jlvE+EIM(C>kdKC*ytnmd8i@`)xu_+`ta$-GSz%DT6T^gKXB<+>z5okT6 z>R(3~zvL$Ny9HIGGFUxaeKaGQN$kkxbDSw1_ea^X=9_Cv;;ovT3f9Ufsxgk=RuP>% zUO`47aopORNwf&B`n+)l|8<4XTJw{}Ny}?>lU+VyW1!}f&g6-pw^KXh?Q0I(uZBgg zGff|c!wm<4u13?vUtl5W+4A?`WR}$RSP|+YXbHEX*+ro08cw2~jz&b&ErPbLp7863 zxkHe4WjvqQGauq@} zMQ6wSe|Xj&^Rx=03MMfl8gJRpuGY@PQY2c{=FY{QK=XRr$k+=9 z<$1Ml#~!?Bxz7=GEJ@7V4MS1AS=$%b!adS&9*2mAPVT40JpKCQMeZkx`|zlqa=AmEeB#5{y}-lds`%q3 z->*KM1O1HAJ}e1Uh~K!CC3P6s2!c{a$5)E3I}C(CKkG+V%uAs?{r(vTS59+9Sf{NJ z2!Cqj`;FU`kK`09C^hfG^8jn@3LE`FniDvZvg?O(bF}Btgxzkr~nN@1ci057L?Mcr=-JGtt64u=#RAoxvt-vaBAzyN3!{J%{ z>t7+t{mYN=*%Q32oWaM4cPK1==9#L#wY#z9URCn&t3tlEt=0F08U+VGZHC!mGYn4% z9BhQT@usT5&X0|*BPvX?pJ1!GkyQ+rIed7z#AkuBLJ$S_)fZjF_50cVpC%92 zXn{OcjIpBzo*r(t=cE1gdfjID7&`Y{mX0 z#aR{Z+3_mUG;G^)IN-9nItdb#21Zw_j&nJuqGWu`P-Rl8l^0&m|K zYh?brxW&U2mLg0e7n*5N*hZ|${ReB<^;6x3?rpD4kwFF<$6i`Xl0tDKsG&D7U}=wb z;*cM~tGfPnRxWwW^^t~C?HnRpk_H=|qUi$XS)jE{H{ya-X7y_DJz!p;wMDe*70ZEV)#`gmY2Y{l;36a$AS8-#v>AINNxc z?G9i>aS&(FFM1kdb<>qX%rW|hMbjXcfn%|jt8Q|^=Bne2bZ6VYepHPTd{Iq;9>Pxs z*!>e_BJ?6e=AC}vTm2@+m2F8YPMA&5Q}7F_duaRxd>Z3xz>S`}B}q7JerxpuH$Ouw zVTLG3d6yKSR^hn`^t6G&256$tthhM!Whp(M{-9Wcs^zvzeG=WgXozhAJC^qPxg!!K zH$F8CB&^#+;nr2FyWcLPu|4{_%M(9i; zS5YXj={2l@=)bY}1H~Af-^JVhxh4%7e{Q`J_>~RO;{YRs^9`uJy;COPL2(sZR8sX7 z44o2od5E4{Oe6pi#_4*d(ac030GP-5sFF8weJE9%#lqnuQXsZOHnXT)BL!NO2za-4=O?H}{kxnb~yqyv<8_%}T;S}cbZ5-AzLI_cxw||;C2M}`*@OSvU;vf|;nQ+(6 zZxQFsl~JUVUzm}iinSqu3v`}M($Hu`>z|e^@*0t9n&u`J|00k3OCa5o*#au z*Lr$vJkZsV1KdC*gGh=z^~JT2Nd~CfUr%<$B=bC!?Q63RU}Z*6D8J^CVWmM&puXag zQ7aKMAu+$>4nSVJyj0kk$Rm0cCZ$@L9^LNT@*u~ZHi6VrJ-WH zq8isYQQrxKC8Ld@98zf&>XfwUDyZr?R?@&nolX@UDKGpJq&+aaUwp{Y+7m1MX zP=TscPIu73g$K=}U{VH7k6=>aX+gd7J+jJJsG4%I*q0;)YuA}09l4FOq4x56!=QGdCkQ7PGC^@P6PGAOL0-gM#J(b$oavC#xA58H~EfO zh8Vi(v3a6KTm81!1f=ubo_W)oGxV{9>NTBi?sw0J`|#inz-na%RwFpS=3 zq?a=Q;21XOqOu1)Q2ONkcLyy+&1u7~8Ml${5+X%jiH<6U5dlZ+L)hR{MK{4fc8TQQ z-(sSIw8NQ&#|G)tY4$QjjMDzyADiI)6H$|zqFy9qt;}mWZBIlDU{nv0g>y!50I4Ve z(?J~KIfY{h)I`N}19 zlC2(cVY?+9D_-F-ky3eyt1azl*c97EJfe?-yWJt&C-kuA-^a0YF{_I?@l zDe%&#!=_3~!JPl4?NP+p9GkxH_x9)*09QlKHCqtt#|CZhvPoEeapO*Hl0SXpPtL- zY&Bh+BZ>-qwp+FJ`@)Z$HI=tBc|(7rcbL&VmVP%T>?m3&ldWE>V@k^jhECo>Y?1;x*7OFSt*xBtFMu@KFYrLybYL{ zx#g^4eXl?$ubQZmV5!#4nNe~dQ-f34_4^u>So8_j>qvrw5wWT>Zzp)QA2@2Y9pjOc zbJsM+jc(X1Hkht5scgHUG39%gHg8X%G(8{Wx*hFZ+XW}*qse+v=Rf>FMFBhbGXb$6 z32eHJDJb)8sHZcL+re4GsYqY@E|99ELO;^m#d%EjNf$IIoektVvg1$Whnef1C=Jg+6N1 zVv_M_*g(BZuUd68dZvA=yk*;sWK;OU<^&HcEFVn*VJ(ez=f@JPmr#&F_ny-?YdU8y5iVo>B?+ax z+~kJEBv(!%?V?|^>vi^Ddfqu|2d9|hHbQK&Ks+rQ9pC@#OYfWPP^RuG0i5UdCA#bvS>Gpz z-%A~`I9Aa=wd3>cIK0$|nHdVaco0yVG$;xFDy)<$vDCoklA_@)<#>F%ol2U7oI zAd}#Bp0BDzDuk0_j>r!ilZjzC2b*-lo0yb6^udYs#Gd)9(OcZAn=#gTCaU65^mn>&n^Yv$!MV{krZw=}FwQ1qHHh(=P9N(&gSf!OVZb;kR=bWSjWBwms9o18 zWj&RazS040Q?@mj^G6hc9ih!)l)?gzY#}nM!ltWlU9szQCseB^RMRycQ#3JwP`sp+ z(j|q{@_Dm#J*@Z!q3E_L!Rr0ZFRQ|lmQPoN*#a6T3nK4U48MFGd-wBoi^ISp2AvD$ ziEs6J>qm@p5{~F-x@flx!S10YYh0#i_9*@aCqpcPF74ID4K}~6#j5I&*!;aQkp}b1 z{Pu6@nHcW_=2dBA1@)tM&uw=6-@|sQ@b1Sqg%WOLCMo%2vr?pE_@x=pni~tMJx?u% zOrQjKbR2$p8t)k+eFh~e+4#L_h0Gc2lg+0p$ zy2LCO3lLt7HTLdB_F33=WrK6EEb$)h<2sx2!R?P9;N1ig*)}CFrBFAu(}lrietZ30 zus59J{gtT*fwR#?cg!$=klIv^gNodKe7ms|j7+G_;v*Q7hu{@LA_H^l@TrV~Kj)R` zVrIL8z-4ocQGH9T2F;T2S2D4~Dit&u_RENKFqK?xDn5|h8z!$PI#FUz%fOsHsLju% zc#rqpP8Zw?(}Z-6igMjJMP8124$QuR4U?PV3aHv`%w}5KKG*is7XPdaaw+7xmBzZQ z5;-9P?wve@=>%fNkWqw^dTAi4rKs2>p~wT&V9(7m-qZ@EIVmnKYYQ3EX{j?wO9{^- zL_uKE4Mic9fK$YVk4Td4ri#$T;VolGp|hp&8#zQgZ?lm>c){lW!RFLzB9sEgm{_pGBD>NOrSTv3G&Mej>~R|A2Q|HSWbEnzUA3zi0y?O9(loeGEyx>T?k*Xf!O;0t8&cf;XG&D;zC?G;FT*+JX3qO-x1jR z>zq;syt0f-58W&gCuURKnM5bewp{7|2W$ z%~ETqa)~EX&rYN0$YM)ExVWJUU<>vNayQ|!>=|E@^B>)tYP&KT&ojlt7QcwKZiGWa z>7L|zZ#dAT2vM<3_(f&xsI-P}1>?nT^Pr$?yvXq&;!(DY$vgNX3n?}qypVgR_fJTj zG%d-Uc$r1@mww;z4|~QVE!{t#8s9owKeHc=G2F1EwZa~MRr{@J?bQ_3C%P7J&23x$ z$*AOLVC~A$an>==4n83Z6HPawekJ)Z9xYSjvbYVP!tC%Kw{mKmqABp%bW_WqC! zv0Gg6XiCmmjwvrF8z05F^U<@Frcou0bB^9Dd8|^GZvq2?$1R>jzA}H<0-MrSmu6$m zPMMHpd6tiZoMczFP(NrpmxnfsNB&3H|2%Gg=@*u9J(8`k>pqEK?G9Ts9wGUFQ0|^K z1nYPr-B`nkl$wzC*&}cw#LUuS6@_HdSiN$WNG_Di7#qmTpg0Xnd%s1{Q!Eyzb&S{% zvr6up;xe7##X{j7Czp<=W6KM8i=ZUG6zubjxaMLaW}UL8pQZ?=0>CbQs(lVodNEYmx-eua8|6?EZ6jgF`wCb3toi1m6v+h2+UHeg={4m+%a0 zIAb}~pmo(YuWYNl)pjKK?FZUP9NHLDzA2W`_18c;S9H?hlv69zVcS;~v=SNOGB4+Y z!@j1S=3XVG9@8d60F1qxSor*bJ|w>F4)`?^D&dIHT$BMDOgB0U&yCI6fXHZ` zo^gY%!5ApnQ)=G|^X8VJ-xGOc!tGD(OFz^Cy&l9Yq2pvD=>UeUPGV-Y*W^nWs)fcK z#9$8VQ7+*82c@%bHJ&(}2VGA7Ld1u`tOJvw{$n@CypDkp@~FVKr7s(=;2U zqS(oLP7*X&ejsgvF7*XGhu~nB-V}HNN?Cxn6i~8T7{Al106OpK)gZ^4fc+Ke9qC_q z5c5e6F;nA}u3-PG#BMloKa%I1oyj=M1w+jdLl^(EP0*K<_Qh`qRMRvPZx_AcOqrhm zLj8Izg4?)#2Rd@T4a59MBWV-AbH_K~?xU z<)zK2uO!ISjk+er7h19&^7}9Zi$3>-`_-K=gg7sp2gXIBiEdgT4GZxuIdR$V4qgSd z5m8WptWFB*x#(3oH{c~JsuR|a!QHKT>(wE#9?&9!1dNbB5qe9*+XI5NqcS?6?0^l# zqdw@<>*3tBdvgnThl>57Qp|FYQ&y9GaEw4*7g6T~Vah4Zw95d5lVXD9x%e><#XLN^ zrz0P8ot4Ul4L0n!-kUcR(QHpWgD~m% zbZ8VGgPpZ__kIfi&wprfIVhOK-*T_;eSS-si5AYeGA;f`d8t?&J3NjAgA&eIFvc-V zwZn|@LXx_*Txs%=J$=QsF|DtPOQt2+rmxCXwXdp>K@utK&xj{#8+FutFOnmMzqB6V z$tj4`ALf9(64e|@xw+4L-;42uwr2eGQj{%^D~kPdmn%2$@0Rez&;BH*){qWCHpFz?})V@{y__mDmCq3u*WDU%UwcwH1Br|Rvr z%QA^-q^>a7Mfzm8F_>6As5 zL~B}IKTS*YPBZm#D4bdXh}9%=)Qba9DZ;LFlAb~)@?2x6bh}++qH|TSRSpNGzVSZx zG%pfmY`_JeSB`}uKX++%bWp_OYRNeL-<-{AT5dW>_ x@h?hg3GDx_?eQgn+7v4C@Cbhg-#||UsD8*P3LNzA{{Vn2Y83zg diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testRFC822 b/solr/contrib/morphlines-core/src/test-files/test-documents/testRFC822 deleted file mode 100644 index 9ce423a182a..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-documents/testRFC822 +++ /dev/null @@ -1,41 +0,0 @@ -From: "Julien Nioche (JIRA)" -To: dev@tika.apache.org -Subject: [jira] Commented: (TIKA-461) RFC822 messages not parsed -Reply-To: dev@tika.apache.org -Delivered-To: mailing list dev@tika.apache.org -Date: Mon, 6 Sep 2010 05:25:34 -0400 (EDT) -In-Reply-To: <6089099.260231278600349994.JavaMail.jira@thor> -MIME-Version: 1.0 -Content-Type: text/plain; charset=utf-8 -Content-Transfer-Encoding: 7bit -X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 -X-Virus-Checked: Checked by ClamAV on apache.org - - - [ https://issues.apache.org/jira/browse/TIKA-461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12906468#action_12906468 ] - -Julien Nioche commented on TIKA-461: ------------------------------------- - -I'll have a look at mime4j and try to use it in Tika - -> RFC822 messages not parsed -> -------------------------- -> -> Key: TIKA-461 -> URL: https://issues.apache.org/jira/browse/TIKA-461 -> Project: Tika -> Issue Type: Bug -> Components: parser -> Affects Versions: 0.7 -> Reporter: Joshua Turner -> Assignee: Julien Nioche -> -> Presented with an RFC822 message exported from Thunderbird, AutodetectParser produces an empty body, and a Metadata containing only one key-value pair: "Content-Type=message/rfc822". Directly calling MboxParser likewise gives an empty body, but with two metadata pairs: "Content-Encoding=us-ascii Content-Type=application/mbox". -> A quick peek at the source of MboxParser shows that the implementation is pretty naive. If the wiring can be sorted out, something like Apache James' mime4j might be a better bet. - --- -This message is automatically generated by JIRA. -- -You can reply to this email to add a comment to the issue online. - diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testRTFVarious.rtf b/solr/contrib/morphlines-core/src/test-files/test-documents/testRTFVarious.rtf deleted file mode 100644 index 57fadb99988..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-documents/testRTFVarious.rtf +++ /dev/null @@ -1,329 +0,0 @@ -{\rtf1\adeflang1025\ansi\ansicpg1252\uc1\adeff31507\deff0\stshfdbch31506\stshfloch31506\stshfhich31506\stshfbi31507\deflang1033\deflangfe1033\themelang1033\themelangfe0\themelangcs0{\fonttbl{\f0\fbidi \froman\fcharset0\fprq2{\*\panose 02020603050405020304}Times New Roman;}{\f1\fbidi \fswiss\fcharset0\fprq2{\*\panose 020b0604020202020204}Arial;} -{\f2\fbidi \fmodern\fcharset0\fprq1{\*\panose 02070309020205020404}Courier New;}{\f3\fbidi \froman\fcharset2\fprq2{\*\panose 05050102010706020507}Symbol;}{\f10\fbidi \fnil\fcharset2\fprq2{\*\panose 05000000000000000000}Wingdings;} -{\f11\fbidi \fmodern\fcharset128\fprq1{\*\panose 02020609040205080304}MS Mincho{\*\falt \'82\'6c\'82\'72 \'96\'be\'92\'a9};}{\f15\fbidi \fmodern\fcharset128\fprq1{\*\panose 020b0609070205080204}MS Gothic{\*\falt MS Mincho};} -{\f34\fbidi \froman\fcharset0\fprq2{\*\panose 02040503050406030204}Cambria Math;}{\f37\fbidi \fswiss\fcharset0\fprq2{\*\panose 020f0502020204030204}Calibri;}{\f38\fbidi \fswiss\fcharset0\fprq2{\*\panose 020b0604030504040204}Tahoma;} -{\f175\fbidi \fmodern\fcharset128\fprq1{\*\panose 02020609040205080304}@MS Mincho;}{\f209\fbidi \fmodern\fcharset128\fprq1{\*\panose 00000000000000000000}@MS Gothic;} -{\flomajor\f31500\fbidi \froman\fcharset0\fprq2{\*\panose 02020603050405020304}Times New Roman;}{\fdbmajor\f31501\fbidi \froman\fcharset0\fprq2{\*\panose 02020603050405020304}Times New Roman;} -{\fhimajor\f31502\fbidi \froman\fcharset0\fprq2{\*\panose 02040503050406030204}Cambria;}{\fbimajor\f31503\fbidi \froman\fcharset0\fprq2{\*\panose 02020603050405020304}Times New Roman;} -{\flominor\f31504\fbidi \froman\fcharset0\fprq2{\*\panose 02020603050405020304}Times New Roman;}{\fdbminor\f31505\fbidi \froman\fcharset0\fprq2{\*\panose 02020603050405020304}Times New Roman;} -{\fhiminor\f31506\fbidi \fswiss\fcharset0\fprq2{\*\panose 020f0502020204030204}Calibri;}{\fbiminor\f31507\fbidi \froman\fcharset0\fprq2{\*\panose 02020603050405020304}Times New Roman;}{\f210\fbidi \froman\fcharset238\fprq2 Times New Roman CE;} -{\f211\fbidi \froman\fcharset204\fprq2 Times New Roman Cyr;}{\f213\fbidi \froman\fcharset161\fprq2 Times New Roman Greek;}{\f214\fbidi \froman\fcharset162\fprq2 Times New Roman Tur;}{\f215\fbidi \froman\fcharset177\fprq2 Times New Roman (Hebrew);} -{\f216\fbidi \froman\fcharset178\fprq2 Times New Roman (Arabic);}{\f217\fbidi \froman\fcharset186\fprq2 Times New Roman Baltic;}{\f218\fbidi \froman\fcharset163\fprq2 Times New Roman (Vietnamese);}{\f220\fbidi \fswiss\fcharset238\fprq2 Arial CE;} -{\f221\fbidi \fswiss\fcharset204\fprq2 Arial Cyr;}{\f223\fbidi \fswiss\fcharset161\fprq2 Arial Greek;}{\f224\fbidi \fswiss\fcharset162\fprq2 Arial Tur;}{\f225\fbidi \fswiss\fcharset177\fprq2 Arial (Hebrew);} -{\f226\fbidi \fswiss\fcharset178\fprq2 Arial (Arabic);}{\f227\fbidi \fswiss\fcharset186\fprq2 Arial Baltic;}{\f228\fbidi \fswiss\fcharset163\fprq2 Arial (Vietnamese);}{\f230\fbidi \fmodern\fcharset238\fprq1 Courier New CE;} -{\f231\fbidi \fmodern\fcharset204\fprq1 Courier New Cyr;}{\f233\fbidi \fmodern\fcharset161\fprq1 Courier New Greek;}{\f234\fbidi \fmodern\fcharset162\fprq1 Courier New Tur;}{\f235\fbidi \fmodern\fcharset177\fprq1 Courier New (Hebrew);} -{\f236\fbidi \fmodern\fcharset178\fprq1 Courier New (Arabic);}{\f237\fbidi \fmodern\fcharset186\fprq1 Courier New Baltic;}{\f238\fbidi \fmodern\fcharset163\fprq1 Courier New (Vietnamese);} -{\f322\fbidi \fmodern\fcharset0\fprq1 MS Mincho Western{\*\falt \'82\'6c\'82\'72 \'96\'be\'92\'a9};}{\f320\fbidi \fmodern\fcharset238\fprq1 MS Mincho CE{\*\falt \'82\'6c\'82\'72 \'96\'be\'92\'a9};} -{\f321\fbidi \fmodern\fcharset204\fprq1 MS Mincho Cyr{\*\falt \'82\'6c\'82\'72 \'96\'be\'92\'a9};}{\f323\fbidi \fmodern\fcharset161\fprq1 MS Mincho Greek{\*\falt \'82\'6c\'82\'72 \'96\'be\'92\'a9};} -{\f324\fbidi \fmodern\fcharset162\fprq1 MS Mincho Tur{\*\falt \'82\'6c\'82\'72 \'96\'be\'92\'a9};}{\f327\fbidi \fmodern\fcharset186\fprq1 MS Mincho Baltic{\*\falt \'82\'6c\'82\'72 \'96\'be\'92\'a9};}{\f550\fbidi \froman\fcharset238\fprq2 Cambria Math CE;} -{\f551\fbidi \froman\fcharset204\fprq2 Cambria Math Cyr;}{\f553\fbidi \froman\fcharset161\fprq2 Cambria Math Greek;}{\f554\fbidi \froman\fcharset162\fprq2 Cambria Math Tur;}{\f557\fbidi \froman\fcharset186\fprq2 Cambria Math Baltic;} -{\f580\fbidi \fswiss\fcharset238\fprq2 Calibri CE;}{\f581\fbidi \fswiss\fcharset204\fprq2 Calibri Cyr;}{\f583\fbidi \fswiss\fcharset161\fprq2 Calibri Greek;}{\f584\fbidi \fswiss\fcharset162\fprq2 Calibri Tur;} -{\f587\fbidi \fswiss\fcharset186\fprq2 Calibri Baltic;}{\f590\fbidi \fswiss\fcharset238\fprq2 Tahoma CE;}{\f591\fbidi \fswiss\fcharset204\fprq2 Tahoma Cyr;}{\f593\fbidi \fswiss\fcharset161\fprq2 Tahoma Greek;} -{\f594\fbidi \fswiss\fcharset162\fprq2 Tahoma Tur;}{\f595\fbidi \fswiss\fcharset177\fprq2 Tahoma (Hebrew);}{\f596\fbidi \fswiss\fcharset178\fprq2 Tahoma (Arabic);}{\f597\fbidi \fswiss\fcharset186\fprq2 Tahoma Baltic;} -{\f598\fbidi \fswiss\fcharset163\fprq2 Tahoma (Vietnamese);}{\f599\fbidi \fswiss\fcharset222\fprq2 Tahoma (Thai);}{\f1962\fbidi \fmodern\fcharset0\fprq1 @MS Mincho Western;}{\f1960\fbidi \fmodern\fcharset238\fprq1 @MS Mincho CE;} -{\f1961\fbidi \fmodern\fcharset204\fprq1 @MS Mincho Cyr;}{\f1963\fbidi \fmodern\fcharset161\fprq1 @MS Mincho Greek;}{\f1964\fbidi \fmodern\fcharset162\fprq1 @MS Mincho Tur;}{\f1967\fbidi \fmodern\fcharset186\fprq1 @MS Mincho Baltic;} -{\flomajor\f31508\fbidi \froman\fcharset238\fprq2 Times New Roman CE;}{\flomajor\f31509\fbidi \froman\fcharset204\fprq2 Times New Roman Cyr;}{\flomajor\f31511\fbidi \froman\fcharset161\fprq2 Times New Roman Greek;} -{\flomajor\f31512\fbidi \froman\fcharset162\fprq2 Times New Roman Tur;}{\flomajor\f31513\fbidi \froman\fcharset177\fprq2 Times New Roman (Hebrew);}{\flomajor\f31514\fbidi \froman\fcharset178\fprq2 Times New Roman (Arabic);} -{\flomajor\f31515\fbidi \froman\fcharset186\fprq2 Times New Roman Baltic;}{\flomajor\f31516\fbidi \froman\fcharset163\fprq2 Times New Roman (Vietnamese);}{\fdbmajor\f31518\fbidi \froman\fcharset238\fprq2 Times New Roman CE;} -{\fdbmajor\f31519\fbidi \froman\fcharset204\fprq2 Times New Roman Cyr;}{\fdbmajor\f31521\fbidi \froman\fcharset161\fprq2 Times New Roman Greek;}{\fdbmajor\f31522\fbidi \froman\fcharset162\fprq2 Times New Roman Tur;} -{\fdbmajor\f31523\fbidi \froman\fcharset177\fprq2 Times New Roman (Hebrew);}{\fdbmajor\f31524\fbidi \froman\fcharset178\fprq2 Times New Roman (Arabic);}{\fdbmajor\f31525\fbidi \froman\fcharset186\fprq2 Times New Roman Baltic;} -{\fdbmajor\f31526\fbidi \froman\fcharset163\fprq2 Times New Roman (Vietnamese);}{\fhimajor\f31528\fbidi \froman\fcharset238\fprq2 Cambria CE;}{\fhimajor\f31529\fbidi \froman\fcharset204\fprq2 Cambria Cyr;} -{\fhimajor\f31531\fbidi \froman\fcharset161\fprq2 Cambria Greek;}{\fhimajor\f31532\fbidi \froman\fcharset162\fprq2 Cambria Tur;}{\fhimajor\f31535\fbidi \froman\fcharset186\fprq2 Cambria Baltic;} -{\fbimajor\f31538\fbidi \froman\fcharset238\fprq2 Times New Roman CE;}{\fbimajor\f31539\fbidi \froman\fcharset204\fprq2 Times New Roman Cyr;}{\fbimajor\f31541\fbidi \froman\fcharset161\fprq2 Times New Roman Greek;} -{\fbimajor\f31542\fbidi \froman\fcharset162\fprq2 Times New Roman Tur;}{\fbimajor\f31543\fbidi \froman\fcharset177\fprq2 Times New Roman (Hebrew);}{\fbimajor\f31544\fbidi \froman\fcharset178\fprq2 Times New Roman (Arabic);} -{\fbimajor\f31545\fbidi \froman\fcharset186\fprq2 Times New Roman Baltic;}{\fbimajor\f31546\fbidi \froman\fcharset163\fprq2 Times New Roman (Vietnamese);}{\flominor\f31548\fbidi \froman\fcharset238\fprq2 Times New Roman CE;} -{\flominor\f31549\fbidi \froman\fcharset204\fprq2 Times New Roman Cyr;}{\flominor\f31551\fbidi \froman\fcharset161\fprq2 Times New Roman Greek;}{\flominor\f31552\fbidi \froman\fcharset162\fprq2 Times New Roman Tur;} -{\flominor\f31553\fbidi \froman\fcharset177\fprq2 Times New Roman (Hebrew);}{\flominor\f31554\fbidi \froman\fcharset178\fprq2 Times New Roman (Arabic);}{\flominor\f31555\fbidi \froman\fcharset186\fprq2 Times New Roman Baltic;} -{\flominor\f31556\fbidi \froman\fcharset163\fprq2 Times New Roman (Vietnamese);}{\fdbminor\f31558\fbidi \froman\fcharset238\fprq2 Times New Roman CE;}{\fdbminor\f31559\fbidi \froman\fcharset204\fprq2 Times New Roman Cyr;} -{\fdbminor\f31561\fbidi \froman\fcharset161\fprq2 Times New Roman Greek;}{\fdbminor\f31562\fbidi \froman\fcharset162\fprq2 Times New Roman Tur;}{\fdbminor\f31563\fbidi \froman\fcharset177\fprq2 Times New Roman (Hebrew);} -{\fdbminor\f31564\fbidi \froman\fcharset178\fprq2 Times New Roman (Arabic);}{\fdbminor\f31565\fbidi \froman\fcharset186\fprq2 Times New Roman Baltic;}{\fdbminor\f31566\fbidi \froman\fcharset163\fprq2 Times New Roman (Vietnamese);} -{\fhiminor\f31568\fbidi \fswiss\fcharset238\fprq2 Calibri CE;}{\fhiminor\f31569\fbidi \fswiss\fcharset204\fprq2 Calibri Cyr;}{\fhiminor\f31571\fbidi \fswiss\fcharset161\fprq2 Calibri Greek;}{\fhiminor\f31572\fbidi \fswiss\fcharset162\fprq2 Calibri Tur;} -{\fhiminor\f31575\fbidi \fswiss\fcharset186\fprq2 Calibri Baltic;}{\fbiminor\f31578\fbidi \froman\fcharset238\fprq2 Times New Roman CE;}{\fbiminor\f31579\fbidi \froman\fcharset204\fprq2 Times New Roman Cyr;} -{\fbiminor\f31581\fbidi \froman\fcharset161\fprq2 Times New Roman Greek;}{\fbiminor\f31582\fbidi \froman\fcharset162\fprq2 Times New Roman Tur;}{\fbiminor\f31583\fbidi \froman\fcharset177\fprq2 Times New Roman (Hebrew);} -{\fbiminor\f31584\fbidi \froman\fcharset178\fprq2 Times New Roman (Arabic);}{\fbiminor\f31585\fbidi \froman\fcharset186\fprq2 Times New Roman Baltic;}{\fbiminor\f31586\fbidi \froman\fcharset163\fprq2 Times New Roman (Vietnamese);}} -{\colortbl;\red0\green0\blue0;\red0\green0\blue255;\red0\green255\blue255;\red0\green255\blue0;\red255\green0\blue255;\red255\green0\blue0;\red255\green255\blue0;\red255\green255\blue255;\red0\green0\blue128;\red0\green128\blue128;\red0\green128\blue0; -\red128\green0\blue128;\red128\green0\blue0;\red128\green128\blue0;\red128\green128\blue128;\red192\green192\blue192;\chyperlink\ctint255\cshade255\red0\green0\blue255;\caccentone\ctint255\cshade255\red79\green129\blue189;}{\*\defchp \f31506\fs22 } -{\*\defpap \ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 }\noqfpromote {\stylesheet{\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 -\rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 \snext0 \sqformat \spriority0 \styrsid16456967 Normal;}{\*\cs10 \additive \ssemihidden \sunhideused \spriority1 Default Paragraph Font;}{\* -\ts11\tsrowd\trftsWidthB3\trpaddl108\trpaddr108\trpaddfl3\trpaddft3\trpaddfb3\trpaddfr3\tblind0\tblindtype3\tscellwidthfts0\tsvertalt\tsbrdrt\tsbrdrl\tsbrdrb\tsbrdrr\tsbrdrdgl\tsbrdrdgr\tsbrdrh\tsbrdrv \ql \li0\ri0\sa200\sl276\slmult1 -\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 \snext11 \ssemihidden \sunhideused \sqformat Normal Table;}{ -\s15\ql \li0\ri0\widctlpar\tqc\tx4680\tqr\tx9360\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 -\sbasedon0 \snext15 \slink16 \sunhideused \styrsid4535536 header;}{\*\cs16 \additive \rtlch\fcs1 \af0 \ltrch\fcs0 \sbasedon10 \slink15 \slocked \styrsid4535536 Header Char;}{\s17\ql \li0\ri0\widctlpar -\tqc\tx4680\tqr\tx9360\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 -\sbasedon0 \snext17 \slink18 \sunhideused \styrsid4535536 footer;}{\*\cs18 \additive \rtlch\fcs1 \af0 \ltrch\fcs0 \sbasedon10 \slink17 \slocked \styrsid4535536 Footer Char;}{ -\s19\ql \li0\ri0\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af38\afs16\alang1025 \ltrch\fcs0 \f38\fs16\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 -\sbasedon0 \snext19 \slink20 \ssemihidden \sunhideused \styrsid4535536 Balloon Text;}{\*\cs20 \additive \rtlch\fcs1 \af38\afs16 \ltrch\fcs0 \f38\fs16 \sbasedon10 \slink19 \slocked \ssemihidden \styrsid4535536 Balloon Text Char;}{\*\cs21 \additive -\rtlch\fcs1 \af0 \ltrch\fcs0 \ul\cf17 \sbasedon10 \sunhideused \styrsid4535536 Hyperlink;}{\*\cs22 \additive \rtlch\fcs1 \af0 \ltrch\fcs0 \cf15 \sbasedon10 \ssemihidden \styrsid4535536 Placeholder Text;}{ -\s23\ql \li0\ri0\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs20\alang1025 \ltrch\fcs0 \f31506\fs20\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 -\sbasedon0 \snext23 \slink24 \ssemihidden \sunhideused \styrsid10829135 footnote text;}{\*\cs24 \additive \rtlch\fcs1 \af0\afs20 \ltrch\fcs0 \fs20 \sbasedon10 \slink23 \slocked \ssemihidden \styrsid10829135 Footnote Text Char;}{\*\cs25 \additive -\rtlch\fcs1 \af0 \ltrch\fcs0 \super \sbasedon10 \ssemihidden \sunhideused \styrsid10829135 footnote reference;}{\*\ts26\tsrowd\trbrdrt\brdrs\brdrw10 \trbrdrl\brdrs\brdrw10 \trbrdrb\brdrs\brdrw10 \trbrdrr\brdrs\brdrw10 \trbrdrh\brdrs\brdrw10 \trbrdrv -\brdrs\brdrw10 \trftsWidthB3\trpaddl108\trpaddr108\trpaddfl3\trpaddft3\trpaddfb3\trpaddfr3\tblind0\tblindtype3\tscellwidthfts0\tsvertalt\tsbrdrt\tsbrdrl\tsbrdrb\tsbrdrr\tsbrdrdgl\tsbrdrdgr\tsbrdrh\tsbrdrv -\ql \li0\ri0\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 \sbasedon11 \snext26 \spriority59 \styrsid8288896 -Table Grid;}{\s27\ql \li720\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin720\itap0\contextualspace \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 -\sbasedon0 \snext27 \sqformat \spriority34 \styrsid10055055 List Paragraph;}{\s28\ql \li0\ri0\sa200\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \ab\af31507\afs18\alang1025 \ltrch\fcs0 -\b\f31506\fs18\cf18\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 \sbasedon0 \snext0 \sunhideused \sqformat \spriority35 \styrsid11105546 caption;}}{\*\listtable{\list\listtemplateid1249008552\listhybrid{\listlevel\levelnfc23\levelnfcn23\leveljc0 -\leveljcn0\levelfollow0\levelstartat1\levelspace360\levelindent0{\leveltext\leveltemplateid67698689\'01\u-3913 ?;}{\levelnumbers;}\f3\fbias0\hres0\chhres0 \fi-360\li720\lin720 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0 -\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext\leveltemplateid67698691\'01o;}{\levelnumbers;}\f2\fbias0\hres0\chhres0 \fi-360\li1440\lin1440 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative -\levelspace360\levelindent0{\leveltext\leveltemplateid67698693\'01\u-3929 ?;}{\levelnumbers;}\f10\fbias0\hres0\chhres0 \fi-360\li2160\lin2160 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698689\'01\u-3913 ?;}{\levelnumbers;}\f3\fbias0\hres0\chhres0 \fi-360\li2880\lin2880 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0 -{\leveltext\leveltemplateid67698691\'01o;}{\levelnumbers;}\f2\fbias0\hres0\chhres0 \fi-360\li3600\lin3600 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext -\leveltemplateid67698693\'01\u-3929 ?;}{\levelnumbers;}\f10\fbias0\hres0\chhres0 \fi-360\li4320\lin4320 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext -\leveltemplateid67698689\'01\u-3913 ?;}{\levelnumbers;}\f3\fbias0\hres0\chhres0 \fi-360\li5040\lin5040 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext -\leveltemplateid67698691\'01o;}{\levelnumbers;}\f2\fbias0\hres0\chhres0 \fi-360\li5760\lin5760 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext\leveltemplateid67698693 -\'01\u-3929 ?;}{\levelnumbers;}\f10\fbias0\hres0\chhres0 \fi-360\li6480\lin6480 }{\listname ;}\listid73432867}{\list\listtemplateid1071396652\listhybrid{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698689\'01\u-3913 ?;}{\levelnumbers;}\f3\fbias0\hres0\chhres0 \fi-360\li720\lin720 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0 -{\leveltext\leveltemplateid67698691\'01o;}{\levelnumbers;}\f2\fbias0\hres0\chhres0 \fi-360\li1440\lin1440 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext -\leveltemplateid67698693\'01\u-3929 ?;}{\levelnumbers;}\f10\fbias0\hres0\chhres0 \fi-360\li2160\lin2160 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext -\leveltemplateid67698689\'01\u-3913 ?;}{\levelnumbers;}\f3\fbias0\hres0\chhres0 \fi-360\li2880\lin2880 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext -\leveltemplateid67698691\'01o;}{\levelnumbers;}\f2\fbias0\hres0\chhres0 \fi-360\li3600\lin3600 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext\leveltemplateid67698693 -\'01\u-3929 ?;}{\levelnumbers;}\f10\fbias0\hres0\chhres0 \fi-360\li4320\lin4320 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext\leveltemplateid67698689 -\'01\u-3913 ?;}{\levelnumbers;}\f3\fbias0\hres0\chhres0 \fi-360\li5040\lin5040 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext\leveltemplateid67698691 -\'01o;}{\levelnumbers;}\f2\fbias0\hres0\chhres0 \fi-360\li5760\lin5760 }{\listlevel\levelnfc23\levelnfcn23\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360\levelindent0{\leveltext\leveltemplateid67698693 -\'01\u-3929 ?;}{\levelnumbers;}\f10\fbias0\hres0\chhres0 \fi-360\li6480\lin6480 }{\listname ;}\listid169494399}{\list\listtemplateid-487930464\listhybrid{\listlevel\levelnfc0\levelnfcn0\leveljc0\leveljcn0\levelfollow0\levelstartat1\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698705\'02\'00);}{\levelnumbers\'01;}\rtlch\fcs1 \af0 \ltrch\fcs0 \hres0\chhres0 \fi-360\li720\lin720 }{\listlevel\levelnfc4\levelnfcn4\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698713\'02\'01.;}{\levelnumbers\'01;}\rtlch\fcs1 \af0 \ltrch\fcs0 \hres0\chhres0 \fi-360\li1440\lin1440 }{\listlevel\levelnfc2\levelnfcn2\leveljc2\leveljcn2\levelfollow0\levelstartat1\lvltentative\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698715\'02\'02.;}{\levelnumbers\'01;}\rtlch\fcs1 \af0 \ltrch\fcs0 \hres0\chhres0 \fi-180\li2160\lin2160 }{\listlevel\levelnfc0\levelnfcn0\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698703\'02\'03.;}{\levelnumbers\'01;}\rtlch\fcs1 \af0 \ltrch\fcs0 \hres0\chhres0 \fi-360\li2880\lin2880 }{\listlevel\levelnfc4\levelnfcn4\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698713\'02\'04.;}{\levelnumbers\'01;}\rtlch\fcs1 \af0 \ltrch\fcs0 \hres0\chhres0 \fi-360\li3600\lin3600 }{\listlevel\levelnfc2\levelnfcn2\leveljc2\leveljcn2\levelfollow0\levelstartat1\lvltentative\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698715\'02\'05.;}{\levelnumbers\'01;}\rtlch\fcs1 \af0 \ltrch\fcs0 \hres0\chhres0 \fi-180\li4320\lin4320 }{\listlevel\levelnfc0\levelnfcn0\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698703\'02\'06.;}{\levelnumbers\'01;}\rtlch\fcs1 \af0 \ltrch\fcs0 \hres0\chhres0 \fi-360\li5040\lin5040 }{\listlevel\levelnfc4\levelnfcn4\leveljc0\leveljcn0\levelfollow0\levelstartat1\lvltentative\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698713\'02\'07.;}{\levelnumbers\'01;}\rtlch\fcs1 \af0 \ltrch\fcs0 \hres0\chhres0 \fi-360\li5760\lin5760 }{\listlevel\levelnfc2\levelnfcn2\leveljc2\leveljcn2\levelfollow0\levelstartat1\lvltentative\levelspace360 -\levelindent0{\leveltext\leveltemplateid67698715\'02\'08.;}{\levelnumbers\'01;}\rtlch\fcs1 \af0 \ltrch\fcs0 \hres0\chhres0 \fi-180\li6480\lin6480 }{\listname ;}\listid1132862691}}{\*\listoverridetable{\listoverride\listid169494399\listoverridecount0\ls1} -{\listoverride\listid73432867\listoverridecount0\ls2}{\listoverride\listid1132862691\listoverridecount0\ls3}}{\*\rsidtbl \rsid724479\rsid2255182\rsid2767955\rsid4260063\rsid4535536\rsid5051464\rsid5706211\rsid5843828\rsid7218132\rsid8152053\rsid8288896 -\rsid9897893\rsid9969477\rsid10055055\rsid10249050\rsid10829135\rsid11105546\rsid12662658\rsid12941695\rsid13331334\rsid14163426\rsid14225018\rsid14292078\rsid14556934\rsid16456967\rsid16539678}{\mmathPr\mmathFont34\mbrkBin0\mbrkBinSub0\msmallFrac0 -\mdispDef1\mlMargin0\mrMargin0\mdefJc1\mwrapIndent1440\mintLim0\mnaryLim1}{\info{\subject Subject is here}{\author Michael McCandless}{\keywords Keyword1 Keyword2}{\operator Michael McCandless}{\creatim\yr2011\mo8\dy29\hr5\min20} -{\revtim\yr2011\mo8\dy30\hr6\min13}{\version30}{\edmins445}{\nofpages2}{\nofwords95}{\nofchars546}{\nofcharsws640}{\vern32771}}{\*\xmlnstbl {\xmlns1 http://schemas.microsoft.com/office/word/2003/wordml}} -\paperw12240\paperh15840\margl1440\margr1440\margt1440\margb1440\gutter0\ltrsect -\widowctrl\ftnbj\aenddoc\trackmoves1\trackformatting1\donotembedsysfont1\relyonvml0\donotembedlingdata0\grfdocevents0\validatexml1\showplaceholdtext0\ignoremixedcontent0\saveinvalidxml0\showxmlerrors1\noxlattoyen -\expshrtn\noultrlspc\dntblnsbdb\nospaceforul\formshade\horzdoc\dgmargin\dghspace180\dgvspace180\dghorigin1440\dgvorigin1440\dghshow1\dgvshow1 -\jexpand\viewkind1\viewscale150\pgbrdrhead\pgbrdrfoot\splytwnine\ftnlytwnine\htmautsp\nolnhtadjtbl\useltbaln\alntblind\lytcalctblwd\lyttblrtgr\lnbrkrule\nobrkwrptbl\snaptogridincell\allowfieldendsel\wrppunct -\asianbrkrule\rsidroot4535536\newtblstyruls\nogrowautofit\usenormstyforlist\noindnmbrts\felnbrelev\nocxsptable\indrlsweleven\noafcnsttbl\afelev\utinl\hwelev\spltpgpar\notcvasp\notbrkcnstfrctbl\notvatxbx\krnprsnet\cachedcolbal \nouicompat \fet0 -{\*\wgrffmtfilter 2450}\nofeaturethrottle1\ilfomacatclnup0{\*\ftnsep \ltrpar \pard\plain \ltrpar\ql \li0\ri0\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid4535536 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 -\f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 \chftnsep -\par }}{\*\ftnsepc \ltrpar \pard\plain \ltrpar\ql \li0\ri0\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid4535536 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 -{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 \chftnsepc -\par }}{\*\aftnsep \ltrpar \pard\plain \ltrpar\ql \li0\ri0\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid4535536 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 -{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 \chftnsep -\par }}{\*\aftnsepc \ltrpar \pard\plain \ltrpar\ql \li0\ri0\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid4535536 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 -\f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 \chftnsepc -\par }}\ltrpar \sectd \ltrsect\linex0\endnhere\sectlinegrid360\sectdefaultcl\sectrsid16456967\sftnbj {\headerr \ltrpar \pard\plain \ltrpar\s15\ql \li0\ri0\widctlpar\tqc\tx4680\tqr\tx9360\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 -\rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 This is the header text}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid12662658 .}{\rtlch\fcs1 -\af31507 \ltrch\fcs0 \insrsid4535536 -\par -\par }}{\footerr \ltrpar \pard\plain \ltrpar\s17\ql \li0\ri0\widctlpar\tqc\tx4680\tqr\tx9360\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 -\f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 This is the footer text. -\par -\par }}{\*\pnseclvl1\pnucrm\pnstart1\pnindent720\pnhang {\pntxta .}}{\*\pnseclvl2\pnucltr\pnstart1\pnindent720\pnhang {\pntxta .}}{\*\pnseclvl3\pndec\pnstart1\pnindent720\pnhang {\pntxta .}}{\*\pnseclvl4\pnlcltr\pnstart1\pnindent720\pnhang {\pntxta )}} -{\*\pnseclvl5\pndec\pnstart1\pnindent720\pnhang {\pntxtb (}{\pntxta )}}{\*\pnseclvl6\pnlcltr\pnstart1\pnindent720\pnhang {\pntxtb (}{\pntxta )}}{\*\pnseclvl7\pnlcrm\pnstart1\pnindent720\pnhang {\pntxtb (}{\pntxta )}}{\*\pnseclvl8 -\pnlcltr\pnstart1\pnindent720\pnhang {\pntxtb (}{\pntxta )}}{\*\pnseclvl9\pnlcrm\pnstart1\pnindent720\pnhang {\pntxtb (}{\pntxta )}}\pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1 -\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 -\lang1024\langfe1024\noproof\langfenp1028\insrsid4535536 {\shp{\*\shpinst\shpleft4866\shptop1990\shpright8593\shpbottom2658\shpfhdr0\shpbxcolumn\shpbxignore\shpbypara\shpbyignore\shpwr3\shpwrk0\shpfblwtxt0\shpz0\shplid1026 -{\sp{\sn shapeType}{\sv 202}}{\sp{\sn fFlipH}{\sv 0}}{\sp{\sn fFlipV}{\sv 0}}{\sp{\sn lTxid}{\sv 65536}}{\sp{\sn hspNext}{\sv 1026}}{\sp{\sn fFitShapeToText}{\sv 1}}{\sp{\sn dhgt}{\sv 251660288}}{\sp{\sn pctHoriz}{\sv 400}}{\sp{\sn pctVert}{\sv 200}} -{\sp{\sn sizerelh}{\sv 0}}{\sp{\sn sizerelv}{\sv 0}}{\sp{\sn fLayoutInCell}{\sv 1}}{\shptxt \ltrpar \pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 -\af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 Here is a text box -\par }}}{\shprslt{\*\do\dobxcolumn\dobypara\dodhgt8192\dptxbx\dptxlrtb{\dptxbxtext\ltrpar \pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs22\alang1025 -\ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 Here is a text box -\par }}\dpx4866\dpy1990\dpxsize3727\dpysize668\dpfillfgcr255\dpfillfgcg255\dpfillfgcb255\dpfillbgcr255\dpfillbgcg255\dpfillbgcb255\dpfillpat1\dplinew15\dplinecor0\dplinecog0\dplinecob0}}}}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 Footnote appears here} -{\rtlch\fcs1 \af31507 \ltrch\fcs0 \cs25\super\insrsid10829135 \chftn {\footnote \ltrpar \pard\plain \ltrpar\s23\ql \li0\ri0\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs20\alang1025 \ltrch\fcs0 -\f31506\fs20\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \cs25\super\insrsid10829135 \chftn }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid10829135 This is a footnote.}}}{\rtlch\fcs1 \af31507 \ltrch\fcs0 -\insrsid14292078 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid14556934 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \b\insrsid14556934\charrsid14556934 Bold}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid14556934 }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \i\insrsid14556934\charrsid14556934 italic}{\rtlch\fcs1 \af31507 \ltrch\fcs0 -\insrsid14556934 }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \ul\insrsid14556934\charrsid14556934 underline}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid14556934 }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \super\insrsid14556934\charrsid14556934 superscript}{\rtlch\fcs1 -\af31507 \ltrch\fcs0 \insrsid14556934 }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \sub\insrsid14556934\charrsid14556934 subscript}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid14556934 -\par }\pard \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid10055055 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid14292078 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid10055055 Here is a list: -\par {\listtext\pard\plain\ltrpar \s27 \rtlch\fcs1 \af31507\afs22 \ltrch\fcs0 \f3\fs22\insrsid10055055 \loch\af3\dbch\af31506\hich\f3 \'b7\tab}}\pard\plain \ltrpar\s27\ql \fi-360\li720\ri0\sa200\sl276\slmult1 -\widctlpar\wrapdefault\aspalpha\aspnum\faauto\ls2\adjustright\rin0\lin720\itap0\pararsid10055055\contextualspace \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 -\ltrch\fcs0 \insrsid10055055 Bullet 1 -\par {\listtext\pard\plain\ltrpar \s27 \rtlch\fcs1 \af31507\afs22 \ltrch\fcs0 \f3\fs22\insrsid10055055 \loch\af3\dbch\af31506\hich\f3 \'b7\tab}Bullet 2 -\par {\listtext\pard\plain\ltrpar \s27 \rtlch\fcs1 \af31507\afs22 \ltrch\fcs0 \f3\fs22\insrsid10055055 \loch\af3\dbch\af31506\hich\f3 \'b7\tab}Bullet 3 -\par }\pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid10055055 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 -{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid10055055 Here is a numbered list: -\par {\listtext\pard\plain\ltrpar \s27 \rtlch\fcs1 \af31507\afs22 \ltrch\fcs0 \f31506\fs22\insrsid10055055 \hich\af31506\dbch\af31506\loch\f31506 1)\tab}}\pard\plain \ltrpar\s27\ql \fi-360\li720\ri0\sa200\sl276\slmult1 -\widctlpar\wrapdefault\aspalpha\aspnum\faauto\ls3\adjustright\rin0\lin720\itap0\pararsid10055055\contextualspace \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 -\ltrch\fcs0 \insrsid10055055 Number bullet 1 -\par {\listtext\pard\plain\ltrpar \s27 \rtlch\fcs1 \af31507\afs22 \ltrch\fcs0 \f31506\fs22\insrsid10055055 \hich\af31506\dbch\af31506\loch\f31506 2)\tab}Number bullet 2 -\par {\listtext\pard\plain\ltrpar \s27 \rtlch\fcs1 \af31507\afs22 \ltrch\fcs0 \f31506\fs22\insrsid10055055 \hich\af31506\dbch\af31506\loch\f31506 3)\tab}Number bullet 3 -\par }\pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 -\af31507 \ltrch\fcs0 \insrsid10829135 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536\charrsid4535536 }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 Keyword1 Keyword2}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 }{\rtlch\fcs1 -\af31507 \ltrch\fcs0 \insrsid15481255 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 -\par }{\field{\*\fldinst {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 HYPERLINK "http://tika.apache.org" }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536 {\*\datafield -00d0c9ea79f9bace118c8200aa004ba90b0200000003000000e0c9ea79f9bace118c8200aa004ba90b4800000068007400740070003a002f002f00740069006b0061002e006100700061006300680065002e006f00720067002f000000795881f43b1d7f48af2c825dc485276300000000a5ab0000}}}{\fldrslt { -\rtlch\fcs1 \af31507 \ltrch\fcs0 \cs21\ul\cf17\insrsid4535536\charrsid4535536 This is a hyperlink}}}\sectd \ltrsect\linex0\endnhere\sectlinegrid360\sectdefaultcl\sectrsid16456967\sftnbj {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid14292078 -\par -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid4535536\charrsid4535536 }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid14292078 }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid14292078 Subject is here}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid14292078 }{\rtlch\fcs1 -\af31507 \ltrch\fcs0 \insrsid4535536 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid8288896 -\par \ltrrow}\trowd \irow0\irowband0\ltrrow\ts26\trgaph108\trleft-108\trbrdrt\brdrs\brdrw10 \trbrdrl\brdrs\brdrw10 \trbrdrb\brdrs\brdrw10 \trbrdrr\brdrs\brdrw10 \trbrdrh\brdrs\brdrw10 \trbrdrv\brdrs\brdrw10 -\trftsWidth1\trftsWidthB3\trautofit1\trpaddl108\trpaddr108\trpaddfl3\trpaddft3\trpaddfb3\trpaddfr3\tblrsid8288896\tbllkhdrrows\tbllkhdrcols\tbllknocolband\tblind0\tblindtype3 \clvertalt\clbrdrt\brdrs\brdrw10 \clbrdrl\brdrs\brdrw10 \clbrdrb\brdrs\brdrw10 -\clbrdrr\brdrs\brdrw10 \cltxlrtb\clftsWidth3\clwWidth3192\clshdrawnil \cellx3084\clvertalt\clbrdrt\brdrs\brdrw10 \clbrdrl\brdrs\brdrw10 \clbrdrb\brdrs\brdrw10 \clbrdrr\brdrs\brdrw10 \cltxlrtb\clftsWidth3\clwWidth3192\clshdrawnil \cellx6276\clvertalt -\clbrdrt\brdrs\brdrw10 \clbrdrl\brdrs\brdrw10 \clbrdrb\brdrs\brdrw10 \clbrdrr\brdrs\brdrw10 \cltxlrtb\clftsWidth3\clwWidth3192\clshdrawnil \cellx9468\pard\plain \ltrpar -\ql \li0\ri0\widctlpar\intbl\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\yts26 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid8288896 -Row 1 Col 1\cell Row 1 Col 2\cell Row 1 Col 3\cell }\pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\intbl\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 -\f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid8288896 \trowd \irow0\irowband0\ltrrow\ts26\trgaph108\trleft-108\trbrdrt\brdrs\brdrw10 \trbrdrl\brdrs\brdrw10 \trbrdrb\brdrs\brdrw10 \trbrdrr -\brdrs\brdrw10 \trbrdrh\brdrs\brdrw10 \trbrdrv\brdrs\brdrw10 \trftsWidth1\trftsWidthB3\trautofit1\trpaddl108\trpaddr108\trpaddfl3\trpaddft3\trpaddfb3\trpaddfr3\tblrsid8288896\tbllkhdrrows\tbllkhdrcols\tbllknocolband\tblind0\tblindtype3 \clvertalt\clbrdrt -\brdrs\brdrw10 \clbrdrl\brdrs\brdrw10 \clbrdrb\brdrs\brdrw10 \clbrdrr\brdrs\brdrw10 \cltxlrtb\clftsWidth3\clwWidth3192\clshdrawnil \cellx3084\clvertalt\clbrdrt\brdrs\brdrw10 \clbrdrl\brdrs\brdrw10 \clbrdrb\brdrs\brdrw10 \clbrdrr\brdrs\brdrw10 -\cltxlrtb\clftsWidth3\clwWidth3192\clshdrawnil \cellx6276\clvertalt\clbrdrt\brdrs\brdrw10 \clbrdrl\brdrs\brdrw10 \clbrdrb\brdrs\brdrw10 \clbrdrr\brdrs\brdrw10 \cltxlrtb\clftsWidth3\clwWidth3192\clshdrawnil \cellx9468\row \ltrrow}\pard\plain \ltrpar -\ql \li0\ri0\widctlpar\intbl\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\yts26 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid8288896 -Row 2 Col 1\cell Row 2 Col 2\cell Row 2 Col 3\cell }\pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\intbl\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 -\f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid8288896 \trowd \irow1\irowband1\lastrow \ltrrow\ts26\trgaph108\trleft-108\trbrdrt\brdrs\brdrw10 \trbrdrl\brdrs\brdrw10 \trbrdrb\brdrs\brdrw10 \trbrdrr -\brdrs\brdrw10 \trbrdrh\brdrs\brdrw10 \trbrdrv\brdrs\brdrw10 \trftsWidth1\trftsWidthB3\trautofit1\trpaddl108\trpaddr108\trpaddfl3\trpaddft3\trpaddfb3\trpaddfr3\tblrsid8288896\tbllkhdrrows\tbllkhdrcols\tbllknocolband\tblind0\tblindtype3 \clvertalt\clbrdrt -\brdrs\brdrw10 \clbrdrl\brdrs\brdrw10 \clbrdrb\brdrs\brdrw10 \clbrdrr\brdrs\brdrw10 \cltxlrtb\clftsWidth3\clwWidth3192\clshdrawnil \cellx3084\clvertalt\clbrdrt\brdrs\brdrw10 \clbrdrl\brdrs\brdrw10 \clbrdrb\brdrs\brdrw10 \clbrdrr\brdrs\brdrw10 -\cltxlrtb\clftsWidth3\clwWidth3192\clshdrawnil \cellx6276\clvertalt\clbrdrt\brdrs\brdrw10 \clbrdrl\brdrs\brdrw10 \clbrdrb\brdrs\brdrw10 \clbrdrr\brdrs\brdrw10 \cltxlrtb\clftsWidth3\clwWidth3192\clshdrawnil \cellx9468\row }\pard \ltrpar -\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid8288896 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid724479 Suddenly some }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid5706211 J}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid724479 apanese text:}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid9969477 -\par }{\rtlch\fcs1 \af11 \ltrch\fcs0 \loch\af11\hich\af11\dbch\af11\insrsid724479\charrsid724479 \loch\af11\hich\af11\dbch\f11 \uc2\u12478\'83\'5d\u12523\'83\'8b\u12466\'83\'51\u12392\'82\'c6\u23614\'94\'f6\u23822\'8d\'e8\u12289\'81\'41\u28129\'92\'57\u12293 -\'81\'58\u12392\'82\'c6\u26368\'8d\'c5\u26399\'8a\'fa}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid9969477 -\par }{\rtlch\fcs1 \af15 \ltrch\fcs0 \lang1033\langfe1041\loch\af15\hich\af15\dbch\af15\langfenp1041\insrsid5843828 \loch\af15\hich\af15\dbch\f15 \uc2\u-248\'81\'69\u-217\'82\'66\u-216\'82\'67\u-207\'82\'70\u-247\'81\'6a}{\rtlch\fcs1 \af31507 \ltrch\fcs0 -\insrsid9969477 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid5706211 And then some Gothic text: -\par }\pard \ltrpar\ql \li0\ri0\nowidctlpar\wrapdefault\faauto\rin0\lin0\itap0\pararsid14163426 {\rtlch\fcs1 \af1\afs20 \ltrch\fcs0 \f1\fs20\insrsid14163426 \u-10240\'3f\u-8398\'3f\u-10240\'3f\u-8385\'3f\u-10240\'3f\u-8380\'3f\u-10240\'3f\u-8391\'3f\u-10240 -\'3f\u-8381\'3f\u-10240\'3f\u-8390\'3f}{\rtlch\fcs1 \af1\afs20 \ltrch\fcs0 \f1\fs20\insrsid14163426 -\par }\pard \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid9969477 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid7218132 Here is a citation:}{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid9969477 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid12941695 }{\field{\*\fldinst {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid12941695 CITATION Kra \\l 1033 }}{\fldrslt {\rtlch\fcs1 \af31507 \ltrch\fcs0 \lang1024\langfe1024\noproof\insrsid12941695 (Kramer)}}} -\sectd \ltrsect\linex0\endnhere\sectlinegrid360\sectdefaultcl\sectrsid16456967\sftnbj {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid12941695 }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid9969477 -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid11105546 -\par }\pard\plain \ltrpar\s28\ql \li0\ri0\sa200\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid11105546 \rtlch\fcs1 \ab\af31507\afs18\alang1025 \ltrch\fcs0 \b\f31506\fs18\cf18\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 -{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid11105546 Figure }{\field{\*\fldinst {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid11105546 SEQ Figure \\* ARABIC }}{\fldrslt {\rtlch\fcs1 \af31507 \ltrch\fcs0 \lang1024\langfe1024\noproof\insrsid11105546 1}}} -\sectd \ltrsect\linex0\endnhere\sectlinegrid360\sectdefaultcl\sectrsid16456967\sftnbj {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid11105546 This is a caption for Figure 1 -\par }\pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid8152053 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 { -\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid8152053 -\par -\par }{\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid5051464 \sect }\sectd \ltrsect\sbknone\linex0\cols2\endnhere\sectlinegrid360\sectdefaultcl\sectrsid5051464\sftnbj \pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1 -\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid5051464 \rtlch\fcs1 \af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid5051464 -Row 1 column 1 -\par Row 2 column 1 -\par }\pard \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid8152053 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid5051464 Row 1 column 2 -\par Row 2 column 2 -\par \sect }\sectd \ltrsect\sbknone\linex0\endnhere\sectlinegrid360\sectdefaultcl\sectrsid5051464\sftnbj \pard\plain \ltrpar\ql \li0\ri0\sa200\sl276\slmult1\widctlpar\wrapdefault\aspalpha\aspnum\faauto\adjustright\rin0\lin0\itap0\pararsid8152053 \rtlch\fcs1 -\af31507\afs22\alang1025 \ltrch\fcs0 \f31506\fs22\lang1033\langfe1033\cgrid\langnp1033\langfenp1033 {\rtlch\fcs1 \af31507 \ltrch\fcs0 \insrsid5051464\charrsid8152053 -\par }{\*\themedata 504b030414000600080000002100828abc13fa0000001c020000130000005b436f6e74656e745f54797065735d2e786d6cac91cb6ac3301045f785fe83d0b6d8 -72ba28a5d8cea249777d2cd20f18e4b12d6a8f843409c9df77ecb850ba082d74231062ce997b55ae8fe3a00e1893f354e9555e6885647de3a8abf4fbee29bbd7 -2a3150038327acf409935ed7d757e5ee14302999a654e99e393c18936c8f23a4dc072479697d1c81e51a3b13c07e4087e6b628ee8cf5c4489cf1c4d075f92a0b -44d7a07a83c82f308ac7b0a0f0fbf90c2480980b58abc733615aa2d210c2e02cb04430076a7ee833dfb6ce62e3ed7e14693e8317d8cd0433bf5c60f53fea2fe7 -065bd80facb647e9e25c7fc421fd2ddb526b2e9373fed4bb902e182e97b7b461e6bfad3f010000ffff0300504b030414000600080000002100a5d6a7e7c00000 -00360100000b0000005f72656c732f2e72656c73848fcf6ac3300c87ef85bd83d17d51d2c31825762fa590432fa37d00e1287f68221bdb1bebdb4fc7060abb08 -84a4eff7a93dfeae8bf9e194e720169aaa06c3e2433fcb68e1763dbf7f82c985a4a725085b787086a37bdbb55fbc50d1a33ccd311ba548b63095120f88d94fbc -52ae4264d1c910d24a45db3462247fa791715fd71f989e19e0364cd3f51652d73760ae8fa8c9ffb3c330cc9e4fc17faf2ce545046e37944c69e462a1a82fe353 -bd90a865aad41ed0b5b8f9d6fd010000ffff0300504b0304140006000800000021006b799616830000008a0000001c0000007468656d652f7468656d652f7468 -656d654d616e616765722e786d6c0ccc4d0ac3201040e17da17790d93763bb284562b2cbaebbf600439c1a41c7a0d29fdbd7e5e38337cedf14d59b4b0d592c9c -070d8a65cd2e88b7f07c2ca71ba8da481cc52c6ce1c715e6e97818c9b48d13df49c873517d23d59085adb5dd20d6b52bd521ef2cdd5eb9246a3d8b4757e8d3f7 -29e245eb2b260a0238fd010000ffff0300504b03041400060008000000210096b5ade296060000501b0000160000007468656d652f7468656d652f7468656d65 -312e786d6cec594f6fdb3614bf0fd87720746f6327761a07758ad8b19b2d4d1bc46e871e698996d850a240d2497d1bdae38001c3ba618715d86d87615b8116d8 -a5fb34d93a6c1dd0afb0475292c5585e9236d88aad3e2412f9e3fbff1e1fa9abd7eec70c1d1221294fda5efd72cd4324f1794093b0eddd1ef62fad79482a9c04 -98f184b4bd2991deb58df7dfbb8ad755446282607d22d771db8b944ad79796a40fc3585ee62949606ecc458c15bc8a702910f808e8c66c69b9565b5d8a314d3c -94e018c8de1a8fa94fd05093f43672e23d06af89927ac06762a049136785c10607758d9053d965021d62d6f6804fc08f86e4bef210c352c144dbab999fb7b471 -7509af678b985ab0b6b4ae6f7ed9ba6c4170b06c788a705430adf71bad2b5b057d03606a1ed7ebf5babd7a41cf00b0ef83a6569632cd467faddec9699640f671 -9e76b7d6ac355c7c89feca9cccad4ea7d36c65b258a206641f1b73f8b5da6a6373d9c11b90c537e7f08dce66b7bbeae00dc8e257e7f0fd2badd5868b37a088d1 -e4600ead1ddaef67d40bc898b3ed4af81ac0d76a197c86826828a24bb318f3442d8ab518dfe3a20f000d6458d104a9694ac6d88728eee2782428d60cf03ac1a5 -193be4cbb921cd0b495fd054b5bd0f530c1931a3f7eaf9f7af9e3f45c70f9e1d3ff8e9f8e1c3e3073f5a42ceaa6d9c84e5552fbffdeccfc71fa33f9e7ef3f2d1 -17d57859c6fffac327bffcfc793510d26726ce8b2f9ffcf6ecc98baf3efdfdbb4715f04d814765f890c644a29be408edf3181433567125272371be15c308d3f2 -8acd249438c19a4b05fd9e8a1cf4cd296699771c393ac4b5e01d01e5a30a787d72cf1178108989a2159c77a2d801ee72ce3a5c545a6147f32a99793849c26ae6 -6252c6ed637c58c5bb8b13c7bfbd490a75330f4b47f16e441c31f7184e140e494214d273fc80900aedee52ead87597fa824b3e56e82e451d4c2b4d32a423279a -668bb6690c7e9956e90cfe766cb37b077538abd27a8b1cba48c80acc2a841f12e698f13a9e281c57911ce298950d7e03aba84ac8c154f8655c4f2af074481847 -bd804859b5e696007d4b4edfc150b12addbecba6b18b148a1e54d1bc81392f23b7f84137c2715a851dd0242a633f900710a218ed715505dfe56e86e877f0034e -16bafb0e258ebb4faf06b769e888340b103d3311da9750aa9d0a1cd3e4efca31a3508f6d0c5c5c398602f8e2ebc71591f5b616e24dd893aa3261fb44f95d843b -5974bb5c04f4edafb95b7892ec1108f3f98de75dc97d5772bdff7cc95d94cf672db4b3da0a6557f70db629362d72bcb0431e53c6066acac80d699a6409fb44d0 -8741bdce9c0e4971624a2378cceaba830b05366b90e0ea23aaa241845368b0eb9e2612ca8c742851ca251ceccc70256d8d87265dd96361531f186c3d9058edf2 -c00eafe8e1fc5c509031bb4d680e9f39a3154de0accc56ae644441edd76156d7429d995bdd88664a9dc3ad50197c38af1a0c16d684060441db02565e85f3b966 -0d0713cc48a0ed6ef7dedc2dc60b17e92219e180643ed27acffba86e9c94c78ab90980d8a9f0913ee49d62b512b79626fb06dccee2a432bbc60276b9f7dec44b -7904cfbca4f3f6443ab2a49c9c2c41476dafd55c6e7ac8c769db1bc399161ee314bc2e75cf8759081743be1236ec4f4d6693e5336fb672c5dc24a8c33585b5fb -9cc24e1d4885545b58463634cc5416022cd19cacfccb4d30eb45296023fd35a458598360f8d7a4003bbaae25e331f155d9d9a5116d3bfb9a95523e51440ca2e0 -088dd844ec6370bf0e55d027a012ae264c45d02f708fa6ad6da6dce29c255df9f6cae0ec38666984b372ab5334cf640b37795cc860de4ae2816e95b21be5ceaf -8a49f90b52a51cc6ff3355f47e0237052b81f6800fd7b802239daf6d8f0b1571a8426944fdbe80c6c1d40e8816b88b8569082ab84c36ff0539d4ff6dce591a26 -ade1c0a7f669880485fd484582903d284b26fa4e2156cff62e4b9265844c4495c495a9157b440e091bea1ab8aaf7760f4510eaa69a6465c0e04ec69ffb9e65d0 -28d44d4e39df9c1a52ecbd3607fee9cec7263328e5d661d3d0e4f62f44acd855ed7ab33cdf7bcb8ae889599bd5c8b3029895b6825696f6af29c239b75a5bb1e6 -345e6ee6c28117e73586c1a2214ae1be07e93fb0ff51e133fb65426fa843be0fb515c187064d0cc206a2fa926d3c902e907670048d931db4c1a44959d366ad93 -b65abe595f70a75bf03d616c2dd959fc7d4e6317cd99cbcec9c58b34766661c7d6766ca1a9c1b327531486c6f941c638c67cd22a7f75e2a37be0e82db8df9f30 -254d30c1372581a1f51c983c80e4b71ccdd28dbf000000ffff0300504b0304140006000800000021000dd1909fb60000001b010000270000007468656d652f74 -68656d652f5f72656c732f7468656d654d616e616765722e786d6c2e72656c73848f4d0ac2301484f78277086f6fd3ba109126dd88d0add40384e4350d363f24 -51eced0dae2c082e8761be9969bb979dc9136332de3168aa1a083ae995719ac16db8ec8e4052164e89d93b64b060828e6f37ed1567914b284d262452282e3198 -720e274a939cd08a54f980ae38a38f56e422a3a641c8bbd048f7757da0f19b017cc524bd62107bd5001996509affb3fd381a89672f1f165dfe514173d9850528 -a2c6cce0239baa4c04ca5bbabac4df000000ffff0300504b01022d0014000600080000002100828abc13fa0000001c0200001300000000000000000000000000 -000000005b436f6e74656e745f54797065735d2e786d6c504b01022d0014000600080000002100a5d6a7e7c0000000360100000b000000000000000000000000 -002b0100005f72656c732f2e72656c73504b01022d00140006000800000021006b799616830000008a0000001c00000000000000000000000000140200007468 -656d652f7468656d652f7468656d654d616e616765722e786d6c504b01022d001400060008000000210096b5ade296060000501b000016000000000000000000 -00000000d10200007468656d652f7468656d652f7468656d65312e786d6c504b01022d00140006000800000021000dd1909fb60000001b010000270000000000 -00000000000000009b0900007468656d652f7468656d652f5f72656c732f7468656d654d616e616765722e786d6c2e72656c73504b050600000000050005005d010000960a00000000} -{\*\colorschememapping 3c3f786d6c2076657273696f6e3d22312e302220656e636f64696e673d225554462d3822207374616e64616c6f6e653d22796573223f3e0d0a3c613a636c724d -617020786d6c6e733a613d22687474703a2f2f736368656d61732e6f70656e786d6c666f726d6174732e6f72672f64726177696e676d6c2f323030362f6d6169 -6e22206267313d226c743122207478313d22646b3122206267323d226c743222207478323d22646b322220616363656e74313d22616363656e74312220616363 -656e74323d22616363656e74322220616363656e74333d22616363656e74332220616363656e74343d22616363656e74342220616363656e74353d22616363656e74352220616363656e74363d22616363656e74362220686c696e6b3d22686c696e6b2220666f6c486c696e6b3d22666f6c486c696e6b222f3e} -{\*\latentstyles\lsdstimax267\lsdlockeddef0\lsdsemihiddendef1\lsdunhideuseddef1\lsdqformatdef0\lsdprioritydef99{\lsdlockedexcept \lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority0 \lsdlocked0 Normal; -\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority9 \lsdlocked0 heading 1;\lsdqformat1 \lsdpriority9 \lsdlocked0 heading 2;\lsdqformat1 \lsdpriority9 \lsdlocked0 heading 3;\lsdqformat1 \lsdpriority9 \lsdlocked0 heading 4; -\lsdqformat1 \lsdpriority9 \lsdlocked0 heading 5;\lsdqformat1 \lsdpriority9 \lsdlocked0 heading 6;\lsdqformat1 \lsdpriority9 \lsdlocked0 heading 7;\lsdqformat1 \lsdpriority9 \lsdlocked0 heading 8;\lsdqformat1 \lsdpriority9 \lsdlocked0 heading 9; -\lsdpriority39 \lsdlocked0 toc 1;\lsdpriority39 \lsdlocked0 toc 2;\lsdpriority39 \lsdlocked0 toc 3;\lsdpriority39 \lsdlocked0 toc 4;\lsdpriority39 \lsdlocked0 toc 5;\lsdpriority39 \lsdlocked0 toc 6;\lsdpriority39 \lsdlocked0 toc 7; -\lsdpriority39 \lsdlocked0 toc 8;\lsdpriority39 \lsdlocked0 toc 9;\lsdqformat1 \lsdpriority35 \lsdlocked0 caption;\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority10 \lsdlocked0 Title;\lsdpriority1 \lsdlocked0 Default Paragraph Font; -\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority11 \lsdlocked0 Subtitle;\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority22 \lsdlocked0 Strong;\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority20 \lsdlocked0 Emphasis; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority59 \lsdlocked0 Table Grid;\lsdunhideused0 \lsdlocked0 Placeholder Text;\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority1 \lsdlocked0 No Spacing; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority60 \lsdlocked0 Light Shading;\lsdsemihidden0 \lsdunhideused0 \lsdpriority61 \lsdlocked0 Light List;\lsdsemihidden0 \lsdunhideused0 \lsdpriority62 \lsdlocked0 Light Grid; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority63 \lsdlocked0 Medium Shading 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority64 \lsdlocked0 Medium Shading 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority65 \lsdlocked0 Medium List 1; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority66 \lsdlocked0 Medium List 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority67 \lsdlocked0 Medium Grid 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority68 \lsdlocked0 Medium Grid 2; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority69 \lsdlocked0 Medium Grid 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority70 \lsdlocked0 Dark List;\lsdsemihidden0 \lsdunhideused0 \lsdpriority71 \lsdlocked0 Colorful Shading; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority72 \lsdlocked0 Colorful List;\lsdsemihidden0 \lsdunhideused0 \lsdpriority73 \lsdlocked0 Colorful Grid;\lsdsemihidden0 \lsdunhideused0 \lsdpriority60 \lsdlocked0 Light Shading Accent 1; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority61 \lsdlocked0 Light List Accent 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority62 \lsdlocked0 Light Grid Accent 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority63 \lsdlocked0 Medium Shading 1 Accent 1; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority64 \lsdlocked0 Medium Shading 2 Accent 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority65 \lsdlocked0 Medium List 1 Accent 1;\lsdunhideused0 \lsdlocked0 Revision; -\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority34 \lsdlocked0 List Paragraph;\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority29 \lsdlocked0 Quote;\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority30 \lsdlocked0 Intense Quote; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority66 \lsdlocked0 Medium List 2 Accent 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority67 \lsdlocked0 Medium Grid 1 Accent 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority68 \lsdlocked0 Medium Grid 2 Accent 1; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority69 \lsdlocked0 Medium Grid 3 Accent 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority70 \lsdlocked0 Dark List Accent 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority71 \lsdlocked0 Colorful Shading Accent 1; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority72 \lsdlocked0 Colorful List Accent 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority73 \lsdlocked0 Colorful Grid Accent 1;\lsdsemihidden0 \lsdunhideused0 \lsdpriority60 \lsdlocked0 Light Shading Accent 2; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority61 \lsdlocked0 Light List Accent 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority62 \lsdlocked0 Light Grid Accent 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority63 \lsdlocked0 Medium Shading 1 Accent 2; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority64 \lsdlocked0 Medium Shading 2 Accent 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority65 \lsdlocked0 Medium List 1 Accent 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority66 \lsdlocked0 Medium List 2 Accent 2; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority67 \lsdlocked0 Medium Grid 1 Accent 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority68 \lsdlocked0 Medium Grid 2 Accent 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority69 \lsdlocked0 Medium Grid 3 Accent 2; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority70 \lsdlocked0 Dark List Accent 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority71 \lsdlocked0 Colorful Shading Accent 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority72 \lsdlocked0 Colorful List Accent 2; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority73 \lsdlocked0 Colorful Grid Accent 2;\lsdsemihidden0 \lsdunhideused0 \lsdpriority60 \lsdlocked0 Light Shading Accent 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority61 \lsdlocked0 Light List Accent 3; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority62 \lsdlocked0 Light Grid Accent 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority63 \lsdlocked0 Medium Shading 1 Accent 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority64 \lsdlocked0 Medium Shading 2 Accent 3; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority65 \lsdlocked0 Medium List 1 Accent 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority66 \lsdlocked0 Medium List 2 Accent 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority67 \lsdlocked0 Medium Grid 1 Accent 3; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority68 \lsdlocked0 Medium Grid 2 Accent 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority69 \lsdlocked0 Medium Grid 3 Accent 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority70 \lsdlocked0 Dark List Accent 3; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority71 \lsdlocked0 Colorful Shading Accent 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority72 \lsdlocked0 Colorful List Accent 3;\lsdsemihidden0 \lsdunhideused0 \lsdpriority73 \lsdlocked0 Colorful Grid Accent 3; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority60 \lsdlocked0 Light Shading Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority61 \lsdlocked0 Light List Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority62 \lsdlocked0 Light Grid Accent 4; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority63 \lsdlocked0 Medium Shading 1 Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority64 \lsdlocked0 Medium Shading 2 Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority65 \lsdlocked0 Medium List 1 Accent 4; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority66 \lsdlocked0 Medium List 2 Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority67 \lsdlocked0 Medium Grid 1 Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority68 \lsdlocked0 Medium Grid 2 Accent 4; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority69 \lsdlocked0 Medium Grid 3 Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority70 \lsdlocked0 Dark List Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority71 \lsdlocked0 Colorful Shading Accent 4; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority72 \lsdlocked0 Colorful List Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority73 \lsdlocked0 Colorful Grid Accent 4;\lsdsemihidden0 \lsdunhideused0 \lsdpriority60 \lsdlocked0 Light Shading Accent 5; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority61 \lsdlocked0 Light List Accent 5;\lsdsemihidden0 \lsdunhideused0 \lsdpriority62 \lsdlocked0 Light Grid Accent 5;\lsdsemihidden0 \lsdunhideused0 \lsdpriority63 \lsdlocked0 Medium Shading 1 Accent 5; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority64 \lsdlocked0 Medium Shading 2 Accent 5;\lsdsemihidden0 \lsdunhideused0 \lsdpriority65 \lsdlocked0 Medium List 1 Accent 5;\lsdsemihidden0 \lsdunhideused0 \lsdpriority66 \lsdlocked0 Medium List 2 Accent 5; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority67 \lsdlocked0 Medium Grid 1 Accent 5;\lsdsemihidden0 \lsdunhideused0 \lsdpriority68 \lsdlocked0 Medium Grid 2 Accent 5;\lsdsemihidden0 \lsdunhideused0 \lsdpriority69 \lsdlocked0 Medium Grid 3 Accent 5; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority70 \lsdlocked0 Dark List Accent 5;\lsdsemihidden0 \lsdunhideused0 \lsdpriority71 \lsdlocked0 Colorful Shading Accent 5;\lsdsemihidden0 \lsdunhideused0 \lsdpriority72 \lsdlocked0 Colorful List Accent 5; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority73 \lsdlocked0 Colorful Grid Accent 5;\lsdsemihidden0 \lsdunhideused0 \lsdpriority60 \lsdlocked0 Light Shading Accent 6;\lsdsemihidden0 \lsdunhideused0 \lsdpriority61 \lsdlocked0 Light List Accent 6; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority62 \lsdlocked0 Light Grid Accent 6;\lsdsemihidden0 \lsdunhideused0 \lsdpriority63 \lsdlocked0 Medium Shading 1 Accent 6;\lsdsemihidden0 \lsdunhideused0 \lsdpriority64 \lsdlocked0 Medium Shading 2 Accent 6; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority65 \lsdlocked0 Medium List 1 Accent 6;\lsdsemihidden0 \lsdunhideused0 \lsdpriority66 \lsdlocked0 Medium List 2 Accent 6;\lsdsemihidden0 \lsdunhideused0 \lsdpriority67 \lsdlocked0 Medium Grid 1 Accent 6; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority68 \lsdlocked0 Medium Grid 2 Accent 6;\lsdsemihidden0 \lsdunhideused0 \lsdpriority69 \lsdlocked0 Medium Grid 3 Accent 6;\lsdsemihidden0 \lsdunhideused0 \lsdpriority70 \lsdlocked0 Dark List Accent 6; -\lsdsemihidden0 \lsdunhideused0 \lsdpriority71 \lsdlocked0 Colorful Shading Accent 6;\lsdsemihidden0 \lsdunhideused0 \lsdpriority72 \lsdlocked0 Colorful List Accent 6;\lsdsemihidden0 \lsdunhideused0 \lsdpriority73 \lsdlocked0 Colorful Grid Accent 6; -\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority19 \lsdlocked0 Subtle Emphasis;\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority21 \lsdlocked0 Intense Emphasis; -\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority31 \lsdlocked0 Subtle Reference;\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority32 \lsdlocked0 Intense Reference; -\lsdsemihidden0 \lsdunhideused0 \lsdqformat1 \lsdpriority33 \lsdlocked0 Book Title;\lsdpriority37 \lsdlocked0 Bibliography;\lsdqformat1 \lsdpriority39 \lsdlocked0 TOC Heading;}}{\*\datastore 010500000200000018000000 -4d73786d6c322e534158584d4c5265616465722e352e30000000000000000000000e0000 -d0cf11e0a1b11ae1000000000000000000000000000000003e000300feff0900060000000000000000000000010000000100000000000000001000000200000001000000feffffff0000000000000000ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -fffffffffffffffffdffffff05000000feffffff04000000fefffffffeffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffff52006f006f007400200045006e00740072007900000000000000000000000000000000000000000000000000000000000000000000000000000000000000000016000500ffffffffffffffff01000000ec69d9888b8b3d4c859eaf6cd158be0f0000000000000000000000000076 -bb6efd66cc0103000000c0030000000000004d0073006f004400610074006100530074006f0072006500000000000000000000000000000000000000000000000000000000000000000000000000000000001a000101ffffffffffffffff0200000000000000000000000000000000000000000000000076bb6efd66cc01 -0076bb6efd66cc010000000000000000000000003500cb004c0053004a004300ca00d80044005500470056003000cd0045004500d100c3004c00c000cd0051003d003d000000000000000000000000000000000032000101ffffffffffffffff0300000000000000000000000000000000000000000000000076bb6efd66 -cc010076bb6efd66cc010000000000000000000000004900740065006d0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000a000201ffffffff04000000ffffffff000000000000000000000000000000000000000000000000 -0000000000000000000000000000000016020000000000000100000002000000030000000400000005000000060000000700000008000000feffffff0a0000000b0000000c0000000d0000000e000000feffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff -ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff3c623a536f757263657320786d6c6e733a623d22687474703a2f2f736368656d61732e6f70656e786d6c666f726d6174732e6f72672f6f6666696365446f63756d656e742f323030362f6269626c696f6772617068792220786d6c6e733d -22687474703a2f2f736368656d61732e6f70656e786d6c666f726d6174732e6f72672f6f6666696365446f63756d656e742f323030362f6269626c696f677261706879222053656c65637465645374796c653d225c4150412e58534c22205374796c654e616d653d22415041223e3c623a536f757263653e3c623a546167 -3e4b72613c2f623a5461673e3c623a536f75726365547970653e426f6f6b3c2f623a536f75726365547970653e3c623a477569643e7b32313839323034362d453338412d344136382d383931312d3837313145343731453345347d3c2f623a477569643e3c623a4c4349443e303c2f623a4c4349443e3c623a417574686f -723e3c623a417574686f723e3c623a4e616d654c6973743e3c623a506572736f6e3e3c623a4c6173743e4b72616d65723c2f623a4c6173743e3c2f623a506572736f6e3e3c2f623a4e616d654c6973743e3c2f623a417574686f723e3c2f623a417574686f723e3c623a5469746c653e486f7720746f207573652054696b -613c2f623a5469746c653e3c623a5265664f726465723e313c2f623a5265664f726465723e3c2f623a536f757263653e3c2f623a536f75726365733e0d0a68aa1a083ae995719ac16db8ec8e4052164e89d93b64b060828e6f37ed1567914b284d262452282e31983c3f786d6c2076657273696f6e3d22312e302220656e -636f64696e673d225554462d3822207374616e64616c6f6e653d226e6f223f3e0d0a3c64733a6461746173746f72654974656d2064733a6974656d49443d227b32344432423237452d423832412d343130442d393536412d4431303443363332453042357d2220786d6c6e733a64733d22687474703a2f2f736368656d61 -732e6f70656e786d6c666f726d6174732e6f72672f6f6666696365446f63756d656e742f323030362f637573746f6d586d6c223e3c64733a736368656d61526566733e3c64733a736368656d615265662064733a7572693d22687474703a2f2f736368656d61732e6f70656e786d6c666f726d6174732e6f72672f6f6666 -696365446f63756d656e742f323030362f6269626c696f677261706879222f3e3c2f64733a736368656d61526566733e3c2f64733a6461746173746f72654974656d3e68656d65312e786d6c504b01022d00140006000800000021000dd1909fb60000001b01000027000000000000000000000000009b0900007468656d -652f7468656d652f5f72656c732f7468656d654d616e616765722e786d6c2e72656c73504b050600000000050005005d500072006f007000650072007400690065007300000000000000000000000000000000000000000000000000000000000000000000000000000000000000000016000200ffffffffffffffffffff -ffff0000000000000000000000000000000000000000000000000000000000000000000000000900000055010000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000ffffffffffffffff -ffffffff0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000ffffffffffff -ffffffffffff0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000ffffffff -ffffffffffffffff0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000105000000000000}} diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testSVG.svg b/solr/contrib/morphlines-core/src/test-files/test-documents/testSVG.svg deleted file mode 100644 index 8a05a4835b6..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-documents/testSVG.svg +++ /dev/null @@ -1,23 +0,0 @@ - - - - - Test SVG image - - \ No newline at end of file diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testTIFF.tif b/solr/contrib/morphlines-core/src/test-files/test-documents/testTIFF.tif deleted file mode 100644 index 8f6c7abba4243d5562e67566675fd12df14cb1d0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 25584 zcmeFZWpo_bmL^)ad)`~~=G`K*Wm#lNmMpfI88Vg3%qS%&NzBZQDP~rRnPth6Wt17p zE@PRgTz1j)dPjE6ou0LNe)L*z{@rjW@kmrwDtepI2M1k!Qrpw|K)FAo%(;i`YLR92Q2^f^S?k38eYX$s%TJ6WF*^&B37%# zG9fKU&ecfyGPR0^m1~q-K8+p4=8E{Xa;1=_7DuvaYNbXNNmD7gG&NtPrYZTV&;3-N zcWrq8c>C^I93&*fRxK8M^%YFa<;zrjo{?H^q!#gw{6CLsgp~`_F>EE@C`7K2@z`py zTxMk9j}5Uj0=s;rkzB?%k}Hj*a?ph zfy6-qgN+1Y3Ezk(R(*LBxO9wIEdm|HDx(;=GSWx@#$oe#;@yj~B}QTyI3WG<5tMu( zTgd}(fQv@ODaAsO+DINF189lFQMO-w_4mJnKO%q<(BKhhgCz|7?cV&)uU@{fV1AK& zwGoFe6wAC#-pnnT7>Rk_CPZhrJv=IqFA|5vEBV-X1dbaY$#v(M`1<@euUL;*X_S<& zW*fyyB{G#qthWi9C+F}zKzsMFor%%sF6u~c6A~O{6u1kEldY4jy-jeeIQpcC&wn%W zQt|{In4plqh5&osCL*;u%EQhsCML!<#&P#f+BvwpyW82Hv^#my2K2B|#mUs{SR0w@ z*zOQt#t7o8xJq%<=UcKnCYvMIsJ%^0z<< zp9eJZ;&DAx8cqbCtKPl9UzZ4A3%~4rxuQhu!)2?z>~^mP+MoOV`=x&99>U*_|F@B+I-|MMdMDS-b8*FWL< zPa*K1>ikc3{S&VL6axRL&i`cB|8H>pxBn^F_%fiTZDcwzqhqlb^y*B6I6`_3+%n(Xe; zHMCwnvvK3%#XHw8-nh8^2i6jdrlTcX4ENZgg~eY4+m!nbS+_ z%QH*s(+iV>x|v>`uDxe&bo%tl+GNk*sin>7@wGRv-mfoiRFqWJ)wU04JLeXz+`IeJ z{OtO@JNM>BXFBUTe}4PEBd7eq^&88RBdp};keE;h8ch8^S z-afb1H+FVr`O4y%vAT|xuBr9$xvl=y?a}4!(be0F=dX=!-CVeM@xuLgKYs!<`n>${ z8vyB#-`~Cc`NM~g0E)+tp8omgA0OWR^8D7*YqJ-xo%!MF&W~rdZr!>2?C19%UcP+& z>CG_fI&W7gIsoBk?<>i_Ag{j%5>Y9b= zxsBBZwG>l2HQZvMD3JiEQLIoUdJYH<3e zn?GJyT)nh@`pm-Y&7E^U-M=@|+OxB|)l=C#uI=0ETih65I@P&!d*afC!HuQ*(Yv#k zu8yC(dHcnyHy^-Ue);9&Z@>KpCIoo!;>DX^K70ZMd-U+>>sPNoy#Mh0=CiZoXV#`K zuPj_yKXqs4@`J~Zp8fv&Z|~pz0=oV0NcBJP`NFK-Ex6G6Q>)XX<4X$*gS~@Iwe@q8 zbLUTOtpBO<`-Yopo7%PY3lkHIQ?rx(gFSWi z>mw5vR!$GM^qpBfy}fpNZe*gjtMA17m|Nx^8XzKuho1+)7XL zKwo{|XxG%a)A3uHsAb_d;^zh03 z%Xfc%^>(>;Wo_cjc+bN6^7X6No?pB5)6egI{_WS_fSLV=SbaGN=n4h_3jhk-i+69| zpW9elot+=*?(gd8sx7M>85o-yoLrk-Se>35=<2Vlu3sLXSeu&Kn4TN#>YdR|%}q`> z*ELS-hR?1nT|Ixkuc_nQ@~O?mP|p?q0j`^SgJz zvkj%}<{`F3-+u3#x9epIz>oC{4+($u6HBoSSIsS?`;=w6K19 zc;VLO`BOu)r^gmX8@eVtbpV{&?833uu7cRa{+5n{_{`kcl>C%}vb3U#tctdV@%g!n zm(Jf@U%GT=#OS6~qX@EWoJGi2eED!-u)y{;83% z@u5L&Rb6IE+Hn8C$k6EIh;DpjyuU}+Slu|>-FKE1QDIat@(RiPd48JlhySm~Nr=^kI}o1E_(zc{xt-#)l8IMG>I+g97uR;g_) zsH)2;ZmQIlCKnc^6jtR_RTVTg*9|Yu?3`M={KMTRuU>A@$OHrUcLwY0c(Ey_4oI$KfHPU z9z>$G)t!6y9)i$t_x}B-&!7GD;_0uy{sKUI{rdG#Q{zZ~-{_#OtD`$JEvvbvsjp{n zVQPMEa%`xtx3aXnySa60bXwQ1>+9(+FD!1WYXEo-ws(vT>Xzmfx7IdR7gnYQ2T#q; zEsjrJI(_=#z5A2H!^<;^KyTC*Rdu)b4)#wyd-`l{V!ArJ`0lx@r-tVS>N;mS`{qYx z*9T|jdM0$*W}UXPx2*B(==_D*wQC#ObG>7uT?6e^4HcOs4W(7><#km#Wz~7LtrZDOOBefr_*o$a}m?d6S2XD{8o za_7#~JGZXfxwiAewVm5@}dF{fv3m`^xG`5!H z=6AJrS5~%aYnpCcz5)V4M@{3ytG8}$p6jlz>#u8G9GzJkpPT6(1W?Shcb^`aJ3qB_ zdT8p(+SWwZfUdc@F2ACqOxx4g(p}jw+T1(P(5Y*kEYH(+wv1jp`*3At2TTD#w5t|= z{|%Tq5Dj4W<1Zf{-+urA{PokXH$YfEy*)oPIo312y>j->^}DB+w;$YlxO4W(#M7h5|i_Di>4>XmZoOL$A(J^@t%IgTq5ZUG4pY10((Y z-9VGg&CD;&0F36w1_u|$CT53+FK(S19vaS0$!e;t9c=HcEvs&C>6jd!xpU*z%G_*A zMb+uy=`&-~<9$P|CFO%{eLdAJLron6_08+LF%Sgjdj_tqp1!wpqo=lcvb%Snsk6Vn zdAhfIx^w94?DABHu1njm>zx?yUwM50^|fn{fdzpe2q?4*(Z^3e|GX;$z?46mBtYPo zD_iH!uWb&u_H{S*UA}N-Yi;Y|#)S*pmtH=5`R4W8#}6I@cmY=L-hKG;mS2DU^{1bn zzJLGA_!*4Fl(o{qU$-T4da6C<-N_FEj3Lg ziRpulE!Q{CT$ouNs%yKsasJZjbE6&I?bQwA-2+oVdw`qKGqO55-{08RQP(}$Grcgf z_V~e*%R6@go!-0#{`pTZ(-+TPy?p)(Fb%l$&qw|My7%GryA!Q_*48#Q zcj$)uhkDz)TIxqfdwN=%2K)Luni}Rt2N!212l|F9%gWoemFr_OlYN6%F5h09UA=qb z=7p`T>a3jSx9>f;a&xq~XQHF4sj9J|sC>E?h=ca?ORJaHHs-o@H#RTmS~^!ICPq7Y z#yfjQI=Yv}CYHwLCc1{Y8oMUCCoZjDSeaaT`|9o0i?@Kc0h0o=c>Ct<<%`$u-F^tB z{Ncl2i4Oqq^^<3_J-XSUvDWIA$M^4V%&%X+aQ*z&Comv7(y^78e2u)Y7|Ltuj=?XB~}qus5oBmF(Soo%%>bb+vc4wp3Tw_I0<_msD026ltp~``SAi$}6UIqqDlf z(SiQ;l{16Ay5a7Qp@!O}(aHIV>9zT#AFp3sUznR8(cQar>Cv6LQ=Prqg3_GCtW_Z1 z#wLf_y0&H)r+WHF+xljD2WGna&#$ge4~&eqcJ`VT*U*jd_I9UdF*=pFCZt<5cf zc>TkzhY#;Q0U7RtAMQQ;@&4B8_LXzDE^M9$Z+rdn6$l8Q+4U!Y5%3cj_3Jl3-@Ett z#zx|uZ*OmkiHXh4C}=1v1U^TjQRippG*(n}wzc)PH&vCD zwKX@640ktbs|%9S%kztx>gtC2`U?_MI_jEx+gf{?n$}lN_jPnv=H&u6*j`;XH8#J# zcJ9*F_WJb9(&Y4=)8}umZ1$H{Co1CFs+)AJ-Lpf(b3>y;tsOnu#_v?C+kP9O-Uv z9UmF!Xl=^O%uY;5?x?Ss)AcEp^8CE4#_GzJ`i7q7+J@>%ZDmbULuGAMO>RqupmX4lAZD&o*;_%SY%GRCxPiICZF0QREjZgh>e&_7; zVqI=grY5exsjIP|NY~if-_+Vy-!RzFG}b-1J~rFi&^pmOw4fVpt8D15X&US8t;{QH ztLqqO>K$&?&FYrVt(?1e>(RNbTeoiB-#)!_`Qr6oe);9*mAendt03d+h03(N9y z>dULzOA1?SY6jZ6n`>%2wY6V4H{3mRasBjU z-_TfB_u9n5=FIHe@Z>;i?|Ap%g^kTC=gtqdbk7bBUq7>bW&6yF$M@g7e0l%Q-K#s- zZ(hA~|K5|;h4sy~bE_+7K}vq@+KY+4?#1b`j;8vi+H!zcO+{r)bWDChQFL62T#-m3 zQX^zZGEJUDksg;?k(gc`rOApw&P_?HD=KU%D;?_Ts;{ih zOHA#muj#ID)E1Xbk4yuHb75^`du^j8uV}EXqcX2xy05ca+q5!0-%(RH+R@pNl`}sy z0%BiXQQ7L)Sccf$^^X#){h3y2jSJ)}pkc^4zNK`mWZR&ei#?Qw!T~ z-@L!Db!mQT>F(_Zx3ArK`sg{R2uEY$dhyefrnjfHWtsx~&dqm(KpyvCHRgZmeD!-*vPztFvnt zHg{&mW+(b|^SY7V*6#7HuBkrV#m&vm+J;LD^AB&`oa*ixX>FS79R#d9Jv+NTJG(Kz zJl#JqH8`}fv;-J+dU4^$J9pl^cm{wvzjg8Qxt*b&vF)w1AWYo2{6l*~eM3!6e{Wky zTVqR8eRgI}Ol~uoEB6X#31iAgBqD*pz_a9ZmW0DoM#bg|gz@ekco%1^e*lZcQ4qN* zr6N`hn3tSU284QjV_k7kNlsQ%SxHxI%}__zKyzzbMOmk|vA(i4Co#1$C9R>XY_P7b zF1w(myt=8frn;bTvaPEqDSf1~yRW8R*V;MTKUh;#-q%pqTw2~-R@G8boe~udVn=&z zV{K7&j3m0Xs(E2-slBnYvATVAW<%G}yEwTp(mk*;J9m0^^6HtjzQ*RJ%DSnZ!R7I( zrJ>=ezJZGyD;L*Rra*nNrERjOZDW3JUN^KfF*e-OKhxh0T*ReQD?sd{81;W*Z(y zBT%>~0yh}LCQ~DL?8tyXl7+d~iQ`@;?IBpa2#sRI#AV18nuI7tQgocQw6q{OAv-yx zJU6$uu^vFt*U(Uuo?cg6+N*6X$jZr0OlmDJFHB0U%gn6JD{QH%F3-tts%@+*C~2yy zTA3Wzb@eWRY;<_IyG~nIRMJ^hUy+%a8=u--S=U^tttzTaS0yw9VU|}3BIDH1?6_`b zYIwf3tX^AOJ3cTv-QTx8H(Fm<*i=&1t!?P8YwoMBAM5G5yuP$DI;Lyy0yVV$hL+Ke z_SqrbY+vu%^f*Xn7j%6~<6~z)ZaX)9{qlw9Pac7s=hsgkuU*<1?j1P2yms~C4p3{S z=SMqRfFN!j>+jN50soVtO08i<#)dF7ikwa&jZdKn$gD^lPfg${S+odGH<;-$Cv#I* z>l6N1f(S?C#Kxtm)NwIVd1gjZRYguts-hr0xja9+EHAe_D?42sJ1}VRE-aiL8E&nroze|!%c|PyYH|}Yijp!a z^YQ`fTB@7Ma!T^jv(sZzGh{CvdzyQzi)*uDQo8G#W`;(ZD{9(n8|v~3 zf$nW6C<5W*#)S*j1*M?&)md57Qd~CP*V9;1*;QBHR$QrTY3^@qY_F*9Yt-&+ZJb|Q zxq0d0%EHW(AAdY^YVGB-=Re-LJv%%;KQVJ|>kOz)bZF}UqdJ}Y1{=IxDTQAE)RnHptWtU4xE zCdo-kNL5HvRZ)3KiG3}NnK22;Qt5DK*F@g{&WG2%ChrIvI{`a zOpt3D$}0g@75RC|n)vSK#`>bN+LB^$urxiRIwv*DK984$jZyh&C1VAO~|gJaaCBBN}N)UrA6SF5d^Xb0V9P`x$DH&33M4BQxS)oplDpQr3*7B0doSdqn{G=#lK}ynSXU9-iTUAlsaBEX%y|$~a z7Tm|;wCud(l!DCS%A$(l-ab%3%T7+M%+78osmx8yNr+Hn#b?x&RkqbM*OymxRhH!> zWdP+23X&P>xZ1+v_S)Loocz(Q&dJ`vk*?06j?Vsu#y&ud+>-8wR&7c7cyIs8%=lzq zkFK+0PS<;Ob$M}obhx|o+~(;YZr!+b`BHUZ>FCh-+}PO8_I61DFqW)lt+pgTx3nZT zBRyH2Qo@p@5a}YeJOfYV(uJ`kek_G9fnhlarku);vA6d#G_W=_usderYGWNNmLzKw zv2m)HXlbN^&(DijXU50Mxcsa{AUmRldOLu#D$dN!O-NNo$~6+1S|};W&MQhyYc44r zZg0-Z%FT{X1YQe7$-d_Lfv&Eu`i9PiCeY4_Pi(6$t;x;JOUcSg$jpq($W6=Ame)0w z15gUev+^=xld}^uvXiolGRg{4(z0VyQe%=zGK%uz64Dg06>!?CH{K`~PkPan;Wje_Sj zpdhfewEK{&O-n(N2W<*%F_{88dsT3VMbu+(lBCVC`k&#@Ng6{ zl_iB^*hU7{M~_+=8QPm2b91nRh$G@;JW+yNmL9FCFUjtxDHT&Fat1RmF1DhiI3+&5 zEU!Sy6&7b^$4M0#u_;w~`SHrw+5#YG3ljk^)tZu={EE!%iu~N3<_2A718{rYwH5W{ zWku;JwfQ-nO-(@erX{9kCTHg+r4^^=q(vv>Cuhnyk(n_WxhdHN8Rb+$goG=Tab?xH zd9h+y6kAf0Q&5(kRh6CF4qRe(o<^js%FY9yp*lCWEVm*xIyG4lT~}B%qw851(*fCh zRMezMQwRyR(wKYbi6t;GC3wTF(x)T zwV0*KC$l3690`WN#Zn_eu{on>V&9>DDX`>2}hvd z@N&~q%X5ozlhX^*(n~V4v_-jT>eytNJU1bsyRizC%s`Him6Dd9npvKm7agI@Oh~FJ zE|al@zo;ZLLtB(rU6faqm)BEQy)Zu1Qe9J9QdF3h-d0%+d}*pWwzILR zBrCtWrM)O8zaT9!J0nRMp@@~MVpM9eG?}5u3deI1C<2li8BUfU@d5-%1SbffI3AfQ zCi7GXqQKP`Yk1V!$i&IW@T7@}i;YdNTqMtmQDkY<$qJR2MoW>4%aW3qcp3wbDT+@h zNX?Rpq#`OSCL%&9j8O1}38F|%L}YwKq>9BXOwR`GjF|Y^qRh&ItiqI(hSGxKtn{Ax zy5^e3g7mbe+A3{jUU##$CO@|@rywsmvm!4qPNqnWPpc`Z7P2HzU!;>m$O$tS4>ydT zFC7xhg9J0e1BkBnfp+E|0p4gHCqlpy=cQ)nq!y@zvSd|aW_((5bW(XvPJ&buFH_{E z=H{oQ1BMr+XBTB=6{Kf0R8*xTrkCXvr^Y4KRa6#cW@pA~WWq>3gD+rkBKcBDRDwvI zha!r?p{Q^SJ2adeieUs}IKfx}k}kuNgh(_UOW=k>@J=UvkLsT=(6=!%axyV-voQB3 zqnSx!DLx!onU_|VpG8Gt2ynE7$x#TTa-J|cA~Hd$5HgwI>xuF5s6-Kfiw6)(1|c9b zs~pr7ON(psvRljZno4qk3>#>y))wT}lofY2G=eFV7v;59mreFJ=B6drfG8KA$)`$^ zROu!8wXiUX%gHdSOrtCLXp{0Z>0^ib5`?un6I}@PKeKijbm;jp9boQOH;U zH%YCC;__o9vfQ}XtT<&|QC4STO;cHUc4A_6QDJ#bdI?ao2?^DOd1cudplH@uS(ula zR#KGPtgUaaE$^tSO;RUt@T>^F3Lb<>Nvx@Bz2xE;e9X|s=%^)FO^qFnn>tyY@H=km zYHsXkYG4!SPnC)jcnpzPEQv`Z^okV`3T1>URTHa>P%6aoL`{-1 zGBRGNiC3s4Y+Ww4|y zUNq1QWO^iE6dKLKlK51*5E6<)grSjv;a<)@md9-M^^PCaw>o;%#=y{Vmt)2*-cBJt zb{i;0xUIYKU# zD`1F$P4TE)311=NNQG>nm>Ur#j1_jK_?hTxGQxNsk+mAUV?M4YA-*o5e)cYb9svm7pg>2r5O*I;7>Wc%h6iG7Y!Md6 zgDj8xTUZ2GnERTWd4uf9^q7m;F((rv+hfMI)>iIrPJzCj5Hu7Y;)4nez|m8`$gub;Qy@-`W_68DocFsF2Fa_HxHqn7TPT;nc}>F`8th z1jJ#UKt={$loW0x%D~d87&?y`LBUe_G!7k0$Auz!RGxsw1;K(z zq%~HRfWW||)3OtyHDWQ3DIj4P461;DA(F`gJOYcsv(XqTJPZZJP+=%qXc#^uob2Tb zg9IVGT>Ly-{K3}M-u;A$wfk;cCdKP+z%nlzsrgspGV{-VQk=`NWBZrRZ>zN)sVrpby zWo%?~%)|jS3=M!>u{AMviqDzpop@<&6=(#o(ziT%EwqqNr#z1xsLuhhs2E6qU-w z<5?&;85T}}1S4QbQZN*U!!Uw;gIrI#+k@}Lo7>o!+nSpI$7Oc-km>Gv*i28)LjTAK zLjxO&<6bVVLp5I(GPw@fQq_9)9yn%qx2ba@3 zZg9lP!zU;x3~g-{1d>)4Cpd-6qcGGX^RH5~ij^q|nv~?!f zT11RAmPce|Cg&27c$G9UE2A9h2ZIHM;~-cH3X2X!F-Yw3;eo`scshrI4Tp2+92yZDwl$>`~AB@ZsZnhb@jAJ`QMOa?HcZ7Gdj1vOS4&_GFk_hUn{^IDE+L5Wwev z$$5);o+No8Cd zRuh++URn_m69-BovdCBw8x)n|vy$=(vnw)_(=$@?va)N{K=?@F@F;3RbT%>!2N;5Z zgR=mueS41X+k0gHJ^&Z! z0H8a%yYACJxZePnt&-Q7fwKjVx@dBm|bjira~5z zpia+9tpE@GMBuj;>?kst6D3Jx(W96IPB=UW0ttg5kZ>3V2}km10vZmmjme^l=oA43 zMgWf|@X%ok zeSJGq(*R3bgrge;P0N(T)Vuo748OA3s5rNZGaTK-Wm1_+E@qL*F^Z&ERc1y?kyHqB0SOw4Lxf-mNIWzQ0>{CTNE8~5W6*>&ijab) zL`BAs@C*tF6C^g96%`kquZT_*N+m=b3*r@w2tt!FEG&w`q(!0w!K%N{$l{2;m8Df^5L6Wi)7Uu?t?l4cZjn!D#KHZ?00?`(KeFdLuMvJ2@h*^U|*IiSd?>4l?tWiu&RU7(;XlBS6z=|N$@ zB7`K00v65;3M9%E`9+mO%IHL<0EMFk5SjjB1xYLw%T+*tMylfkiD}?>U{WTN#g4#n zA_!5@{73~`s^9}GRq>3JG_@usA~`iWBRfsZj&gVO3i1lZplBgsxNtm-h#deL{gUa`u2D@Eo>^!%``v$pj?D)4?4Tg2BMZL>!la7jns53W6Xe zGx$VGn75y$xxLx(09QW_Euu0I5p(=Ru&XDB7p3(G5FR>Yv2U-@E~&oL`~JH_yP$n$ z7l3OQvF`yUfKDcR_W+LVat~m*i|Q`%K9dkw?tY`7aA{iJa8}`PTuNhfLWMvS&lkmW zcp5y8=j#jc^9?7F02H7Eo0^uEm6((sosi5EMu}unsUk`$13FHlP^vVsiHWJX(%lS_ zp^TBM61V`XgtUarf{eJ-C`Al6Jv${aEjlJepo(FlkaR2@@8ce9Wq!i`_z8Pc^H9$K zZznGTf)ptLm?VfrDi|UZiN}S8BFO|UiNFr_gX6-{VcuaZ95cYl6Y7toUOMNr}!7OAIi1!(hm`T#4v@4q?p z{kMm}_PgDN9 zXcWQ-Zc;{qLd{Vn@M1Fs>J&y)B2N>?Q^m3qQzS77a#eJcN)wTkr4;iNk$gFq8R6mJ z;bP|^Wbu`eDl&!yhv8umlw6)1DUOqgqr)Ls22CW1h>PT@5rIe$ApCvc6f6ts4@ZaM zg54ktvY5wG!9p+)4<8RJ3lUEl5<(9_X#5~iZvG;@BeqBM?T;Pz3xp}%dW?)IDgpLYRhhXHioescf-1^NImfwhAEyLP(w=%ItA7B*oK z%9^Z#!KAFVn8ccd)VkCxZB{{ZY0Z$fb-K8sO{oz{qqtGvSCz4JxtbZ1BvHpHlarHE zlCl!gGG)=6sA#GpmaK`TE2Rn!K3pKi$m2=!cygqQLE{ie3@B&_#XKQMYLqcj1zW_E zM@4Gl6v-Sa2NXAQ(hIWF3o+q1CXGuWaN`q8rLuIPFgjA6s#0eWu}nOc6X**=BN)<% zBpxFwf)`CjFc=`y7Jy$%Ch|EN7h9(wpAa%u;N*<(4vDaHp%|FB9|9ukuoa+|n=cRe zF8!k?_w6+RSbg^$z~SI7IJ1dtRdNpt~HrlE+XEU`3F3V!O! z!f@FH4wp;^L5{-^@`NPtP>?A>gkxz076Hq_6ZkCPy@YW>VH}?y!=ObGh++(i1%(kE zY@LZn3XdW6c6P_Z$Z!bG!^PFd&)v)0(aR&m(G6qggfq8>=pC^?c+ebhN$-e_n?Dy$ zOffO@-oM}Uz<$&3_ZWQp8HaDaIq=Qb`@jBr-#1|W`#u2IH(x;e=D-)wzW)wT3s|E5 zZcx}`aMZxbFEm0KUzeWSlU3NCTdV_*#PdrBN^3^7?Mq#{^SoUtMCC`4=^O-=jli=4 z2#gS>B#a>pp>V^5GF+qzsZOLQ;u#59pr|cSCa}dyv^*9kiN%67Hc2LrCd=Y!vS_j@ zo-S8WHE|*e4a*cG_)-*424e`}OaY9<2t(n5u|x=oiX+oW;iym~27$yQpol;$F`Uf8 zp>UxnQZNi35`^%!urzgdwzocEZfS1p?d|X97aRl)3WE6C*f^S*yPH}B=o_3o2t4^d zll=#d9nrUQ@}!Y?xoAp)g;nrjJFlqBApGTaKZ&qBtaBeC@1iuFcJ-4tj6&b z;j$#GCXJ*@CQDWQkRMQz8a>ZaM`oB|pb$`HVL5*#m*#+QirkwU%%R8TM+0S0(+SyZG*Buh;#%FZau z%&5&wFWs%u%E_ukvNB1aP83Vy8G>kn6jbdKXh4Hi6je%;tOx~_Or-L-5@2jN3K>BN z!cs$MObmemA#;!>U62Vi~s_Z>T;cLK1;$&*54rARep0$B!2kVN6c z`iC+tt%L0x;fHqZ@i4GQkm&%|vanV(h3399X}L8Bau|Z_PUQMf`4EnT&JYj)8Ax0Rfev8_2qFnp zoxoBjaMg+IxJ0R3E{ljzaYbULkSdF2Nz`XQkF;D?Rn3+}x z!vB48eK?Vj&287#xd4uadGl?I&}E>0YIw#ru+68@7-erKr=D( zLXhJ{ilWG$H|$EG#MrqrwBD-_Ws>bSDx?DmYj?yRETlG@3d*5!)EDGV7Ch7N+@Jp&O=;Z)af ziaU{sgc5yx!yEzIag0DB*N-j^ri%mFl2Az;N}WWD)DV=h61hsm7qjUCLZph#lhTE1 zHlBmR6NxMl+0WNQ%#&xNRpDr1R5psjf}rt1L<(FK6~Pwp@DvgN3XXwA@(LC!FqvWrBhh0!cuHz(WHdfq@8Mr2cTCJCy7Mqd6dm-mbpxdioXz zLB#+7wbux^oqhWZ4UJp^pb=bYjzpEu7RC#u>Ac8PUSu*SB9X|@1coy0PlkB~kR4qy z1_rhkC;Ywqh-6NZK$as@7pvkcG>J8dS?$qDHBx1sTvL#c-jrF;n_n?p*|gNudp0_; zAPDN^8F&(k4FM#F5PT5ifH0g(2-?p*z#h~rVZ=ZP-W@~p3dK1isP0sepGX6f#3Gf6 z6ipI2DxN2eX7Z&ZmXLs;hq9ugWHIp~iI~7eNh9J4OFQ5=X96VvLkLC@1Ly)Oo=zZ8 zh!9jLhJs>nDJV)1jvkJogaJ<#PIQLg9K#9T2%!+PGD+CEDfCt+(&{?p2L%*$)ig&Nj355&9P~Xaj6Z78I7sA zt+{3Wl}(G)P4l4o3qlfsj0?pC0-q3qbq&F|(?xiyh=8Vr5!i@exPK7R6N2+aPyzx` zP7u5!g5trFhtYXKARh!#FH%L{O`D}L{sF%3p3W>T7l%hN`8Y9;6RSzW&^*yJS0oXn zPi}Zx2%d&Qkivp7t{~fk;k|Za9LgO}LEtC|2+{|M^+Dl-&?G1v?-u|)iKmCdut7L9 zgiME_Nj^}#GY%ITjIjyE+xeqxjLoeN9xwyp55zAJ*S`My-mm`-B*A`wo_`pZNRK8m zH58VbC5UGU6R2!8jicsCl4%?jofk{z#o#CsB#wtB3ek9BIEn*D^YJtVg$*7L<|`9w z5;8l|3kI@FhVsgF6%BJu-J4vVoC#|90xp|R0<||RBg7-fDGVQoq6dfLgP@o|3Y$cr zBR&0{T>VZ0rUfJ2!qDzWsyCeMfoB9TMKGR(ERSI-G?6@(5Qm|H|B^(aV8mq5GK-=T zGKEPwY627FDz4!~zYvs1Aj~xcZF%_U@n}dfd|0%?pkuMY2SRQL!b7neEx7x)SY7bN_|P=1CYa6o&Vv3CI*Kk`;j= zGNCvchApC^8NO6OIDr#@AcqhcGzc^-2FhW2$6#~NthGN~^ z1FRufNI=xg^MS#gK_*& zqA(QC1b*K;Al!pMhM_Rwp^y+JmFa0^10FkBo0}y@C#9yJ*G|0iGum>mig&*dhTE4-Y}Q0^tNB04L@ajJEgi zvkb<2dqQm;gDjkVZ9GDOcd~T}v~~0`b@a9J2(fbwu|DZ-y>Fk9g$?9{J^ZNA$^D0b zZZ|k)YGrC+qHm;UVrFD%Yi(?9x@V6bC@<*in}G+ov1twJv=*VNfFe-2`MJ7yJBEga zI=eg3cro0_1c?SzJ`+sLj~Sa99=AM>r$$Dqa^utM3u`9Zb=Pn-wx^GaySop862a!j z1wwrtJwZO@=n>!sTrLdbk7wiQpzszCXhtX~Z;OZ$3!*q=0h+)@5*b(w6lG~-;_Tp# zg<Bvof@LUMpa;7Ab8wvX={Gm+6;-~M<}!7 z(zW?j6YZndnL=qG)ZZt-)x*OT3J->%At8{E0EnN9pQCrEgNx_M6OP7yXiqd3E|$~5 zSGwx7qnSBXNtrpIj*gEEM$y9H;TYq?`dlW@#?tzPt+glyCy&L-lxV36Ba$P@f-n~N zQWYhP$_=NoKn@7O(R}fYAPPSi0fHdGIT&vrPI96M;0!TIk)+h*rQkr=#dw4gq0Uej zmmoVIsJES$xucJ@ovXFgNmDydGq=zacAlp8-e!(o_9yIuoP2Nqr~`-f&1?+KtPRa< z42~T?3Ot%Y<+r}H>>=A759%kbdVCEcn z!ol}gFgiq*ScM>}4(VB#SzB1yAGdHYv~)H(?qXtKrl)Um$n=EO3452L#>NJQz&1?| zA31Dbe8kei41C~Y$Bj))4b9BWjLnXkS(+p95_M{QQdUEBa$OjV>g@~jaP#-{0}4>2 z0^f0~7;YcE5st^Y_@49$@v=K~4O|vi5iU|= zXv}aL*9S#*!qffX1fVt@NL)`8#RDXOAoYfmoWn^D{z$6;d{`*R9(Y(MKYN6L4i(a! zg4}FeZ2V!c01Vm2!_2|k!p7Cq%EQRY-PkqI271DITut9zKQFWf5tP|n#i)G-9%9fY(CLAhc%{v=sASI(dbh)g~chJgCJy4spq zaCj7s94?Hcv3X=X&7b!FRd6NEairJrPdF^i5J2y>bnW}T?_JeZy`y(5jfDV60w72Z zMGj|Z4mFa;3VE!tLW=@x)2r&~llX9kb0lNqTBfw=T8FI>BE}g)Z#??)yMO%l@BibQU;Xj=_4m9fpnRM1 zQG{vXI7?9yWvDpM)S8WY=TQUMueCpiuX_+mAUaK?-bS1CW}Sl1Q)@SnZX2iDoFvd3 ziP1R0AvoXa@)#j9*Dt^O?zeyXr$7DoAOHNnSFeA{>NIDPoW=^OYPxY&txnEf&espT z;E@F9#y$yI5Z}>l$rWgu!6J_@=l*QA=vCmzl~HK)N_Tx>&(E#P`{d*%JiE{4XYP2R zkB;@();Zix(t%#hvf&J3FPar{*~l5}=JVSp%h%7hZ(a_co~G;KfAHIA( zy1yUZJ?ULt_Kw#ObI2BznOUOGaSn^J4NUB)A()oI?U#S|@rOTr`|Ce``2JsSzWg0R zAsFALSyq!}mX+&`&S(372?*an51Pnc6Wzm^1|?xAdGGmVl2{XMj#6npP_(3S$6MGOE8DBQn#;WzUoDjBu`xa} z{E`g^?0Butm*dUJY_mJer&&IX7sspX_g}7F+#TLOi9T&&t8|tgo~^}zn?20t5BK@i z#;77CcNv2@J->hc`mYD;g>t$56+5yC0fXny{_5SEzrB0*cG4dwek@arrqGjPZGC2jeFn_BF=&~Q46u>( z8Mja6W7P=>h}Zq%m;e0pZ+|$u*yTezAE)*tDzA?Tg=%26TB~l3(#03AFTQ@idU-#( z-SyAsX+KddEvj-m^!>1GVEcOLXl^LGn)(Uh1<|K8pC+}on?;A)3n1QkZE(i0i22cBvp?i>RTS8LN zNru_v)r+5fasT1f^N*JR+)-(Jvgvm%zsty|*CTAN8}-?^0IerkDdDDUHPZ)!$?o)N zoW_gQJgXp0;jPY&;wtM37((xpYNxt7E1q1AZ@0&9UYx&syM6O~e7Q}_QuSdp`%ypV zUCN3rC-dw~3Ho+Wc(O+d4$hfv&SY&r1mHM+{3Mx91y7cTkv{A5iAT%2AgG3^sH)PX zu_jrocMmA0ZTm7QgPcSOrNwGE#@0z5rFf!49#BdL;pzxer&L7r7(wsquA#Y#U{FRR z*)ip&yrhe!ZkvjdCNZzdoT62O`O|kF`=_hH@ri7D74XO7TiDr6`m^h^XS31f`uzE9 zu$?w+ zFTXy&`tUTrUYnC3zZp;NPUlZgkG_0ZzPg#5oUD$XrTzJAy#{NmT=&gPh=!IO$!?-s ziJ*C$X7O(1yMduOs_E;~6ji}4_jikjyJR$-A3lEeG|o?JQ>j>u9%plqKZul zQb&*wDWhcn_U+Ypd)gmtTswMl_454qYA_gDJ>JR1q%_^!W+4|Nv?Yu6vO+Q$ z&E!m9k(@3_4@O7*)LU(jVdr1=%k9yXVamEENpKEzL~s$!!_&g2RG4%P$t466Wwf?v z;i^r`J|zIrH4xcHoQ#rO)QaIq)6rPX=avz3$9((?%r;VoWt0*)6b%0#%YRK>9d>V z&HML;<7lQNvhv=ey&6*E3`z@eD^Q>j1gFbeT|pOg&yj5z7ix^y5_Q^(HPz?(!>k;Y ztlCx_T=To&_NWf%O58~p$?hs4D%x$uCG?O~9D>(T*&#)vE9*3?kWf8%kd&LUji_PQ z>Cs*#=rQ3}LfF^5f;Ez^Td{7=8!@BDw3)G1${UFYXPagD`Q&mwKG`0g^xW+F;^EoN zm!XxNZl14ZSCG7g#Yv}A$GQmGZIcwr^KF*HIUY_IWBZ&Pa^qQ^SK{^6laF8j@WsQo zAHMtQ`itH3Zyug~fP0XhY;We0N9zGUnj9zTTo7$qrzKZoRYFY}(Is0*vySdHsePb( zq|#(`r~vJRGPITAsj_2}9PllOWYtlwP6;i`Ra{TY%S1E@JtEwK)_u;;xo99-32BC` z?9d$aorM|zdL|+InBbvS!txr%$cUhKpnAOVskUl_Y9S^XNHAp0kdSTMA96v-`h7ku zrEI2pfG{K8$l%32xO}Fkz?O+;q%<5ryyxcf2ft&uO8n%yoasV7_!(y1Dr9%BLoX$l$KzDdNUXf z#y!vINRatzAjvMN#vD+c4qw+o)-7#41z@3#Sk!E~KPckl>|>PF z!U6N8hT>pG21X!4>uoiL!Xk=~y9Jd^WUo&<8EvI>I1}TcC7HCHNUc_zW3=gD(+m2k zR~VB2H*B>FqWedrmIcIiY!|0p4K9LuFXAB?6iingNTl zkYfZCI44TA2?5wu4d1s7CoKK0$m zn$D+bkrde4}7GFfN}FH56*)mwZra=^(NreLPxtZqkJ=Lzvn4 zY;$-Ky94Nx>0q~BLX_jCf?X7+i@a)xrbLo_mr^iDAFz0v0?NV@{ABKoCeCCQFBau| z)q}*s@#!`z?8mqF{h4#P$(Kii@g^8=6Th!TMFPZL(G*?~2h)Y$^Np0#B26~z++4!>8l!i+Qd5jD-bJLSEw~`)>Vku)UYF$BC|Q@__*os@ zXMnCV3e)P;_G_Ot@ckB1gMc$Fbvr~Ir5hOE#Ar}MD8aNyu?>kamcbFa1qV;EDBu;K z-2GgXO~#5Cs9QD-`h(mHwZ-CiHr@2HzM@z>!zB?c&ac_lVKbI9!)O&Q4cwbA#*`@?h3eHAHjp zpx{TVD#`P_7;e@V9$0Ca>-I$>g&sllkns8ftTVh)vI3nB0Jj1e_>vQ;@!7KANI)L4m9O`YOf&DQ?j!J{@_N2xkg2BR8KD4g%0 zbO&MDEv!zfco&p7(~%j9CJ;*M0)z@?*MLiw35vxSG3z8^Hpr*L!?UxU0wx4lSw%lC zGLB&Z=g-gI>~?SFqf1ZmMM`aa-bBGPVs)Mp>yHkQTD@IsHXj`bIJ+FK&KKup6%U6& zui~Pd0l79wJGCJt9UY|8n z7-0aUgx%x4p6Cx{CC06u4AKDza8eoRNbAi)J7t`bheAmXQ2I8^cOxSN8|PK>Q+{7z zMVai9EtV!Y1;MB`czq4z09-6eKw3zni8MOhCdeXM1a{rQH~`>%oI*lBVbKoBp+GHw z#aS6+#f+WsFz7w-BrRc54qrdK$v~Rfz=~}+a-2w8iYZds?&#Ig{B~ZQ4T3RXl&%CK z*dQp=IB3BBd;ha~{YM7}Ki=~#e>GZfAfG+#+dk3jtFu)(oaC#c`C>iYTuecWPgmJ+ zZLN<6oAa~dv(0drSEB-g?bR#>GGix%n~-tIcs({Kz%qirO$|!U?UBVq>>v8DvyO&> z3bQF?C8MJNEf0ZR1M9#wumtr_yUAGkhdXyswyk6Ub1GfJck7}PE)PH=i znHS~2DJMplSW)84=Y#XJljX^<)Wsy)if z2(V0(zLbr;auUUbX-43Ouy8Dg^Ee$yV1+qQOo)h#V^VCfa!b(KdW<{0u4JRU4v4st2mL+mL48+DGn1B?6FwM_e{|~yD2mt^9 diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testVISIO.vsd b/solr/contrib/morphlines-core/src/test-files/test-documents/testVISIO.vsd deleted file mode 100644 index d699e11122bf0ddca65d221828df6f35923ee646..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 45568 zcmeFZ30PA{+dn+#B!RGI0fH!+1O!ph5EccuBy5U`VHMYsunCF=S;Vap1>9{|v|8H= zh_-4~Kyj&Cz`fNL-1Vt#P~2Er!sZH`@17uSU;8}Q)A#+q|L^+V>w5=&x#!H8?c6hS z&pr3dIkh#Mx{`|ikI)Bh0-~WF$^;qxDVzX5*>}xGhyi{a)oS4ol&bd=$#j&2(f}&t4~Rt>Xd#L> z^u#}>{MUYaL=u?)bcllKzmECG9WHpEZ|c2fpm0N(3z5>$_k*D91gJ?W)I$m50oQEN zA4&h>bAj^l9N0Ja{UyRb31C84o>lJ+#$TTMKUW{`_s@@?9lL*0++TuledoOAe{ZT@ z{`JiTczhZ|9867sngYxK<^T(TC4dD06##JnI39i+Yk&=a2e1Y71K0uj0|o%>0RsVp z0DOP|AOtu7907v?P5@`X5P%C{C}0?1IKUM!0x%LF0=NOh0C#`~z!TsF7zG#&@CNt* z#sJ0w#sS6yCIBP=DL@AB1^5B{0RaHqX(s|E0fGRN0aE}|0l|Q2068E85DEwbgaaY~ zk$~xdC;;Zm|JU6gJV%`Y;Q7iGGR65lfzFNR9Uunxp&!SMP^HEu#;BI!2zWf~vokZ4 zg|Bg$*nLO`K_&2b!DA&vIWK;M7{%gxMWiZY7bGp3Cy0n&il4z6;J<8p3}8~{$A2f%q_-r{R~54S;dNRWT@ z)Sziod}ZMQhC6tckDiW$Wm6*p!lub00;b7=1Hui-@gqe?#4k)qR;I@X`TxOx8Qj2m zV)r5c@pDJ`xryC8eMWnGiQPQ#g#N$o|2hZ$)%eBzqb|3o?TtK<{Y^fM-yugI;C`ck z!S>I_D()}1pX2_c2H?K76@bS^F#yje2LZUx>H$~>r$Zrltl9&`dZY``zdKeZspe%W zQ`2$R|4U!Ob-)hyv2g&uF>^vvRr6An3k88m$?<7(f|FuXRcWe(bb&k}At^R~&h(_T zB$Ze^2lu-<{=UI-|EbZzBd3Q)81ld!`or>YfBhpHLwY2Iu<~W|4Qm{%Kf1v~VkwNr z5awqxKnd6l2!jW02ioUW!eIQl3?V^4FsPpda)Gp1kL-JvzIXy)^bANVg$ywt|ATu> zN9z4FgZ}CYHv|C4Za4vv?ZJ=hOtPzf!gW!+2NZ_l05bqMuA7uJ#Op0rE#X62{QTGr zl=jLo+=fsE6bF$^rAR#({8-nos;Z)J-pO+=`5xTv=(iKy_mxK9kqIFTr-=dJyl@|MHKm0iVApo2&j*lJWZ=g=_+m}zDAM#gYez^f~Ii-LafEIxJ;&7uero)Y% znl+hYVE-7?zO>Lc)qQ@5YIvUab(6vMd#C>0kL!l{whw@1CFUDbibyH=Jpq`m{L+;<-l1hE?a zZJ@nyJJSuR{~n*CA+9$7*AG+d`Y7f;o`3(D>*E|4KW!K%TfS;5y(o=>WU{;5Hd1?R|dShoA<&2J+tD$3Lc&G{Subx3{4^d@LvY z5wEw-GFagS!Afx|+!!S#)pChrOA zs4@dZPww_!r@l2%X@6r0zAt^pHc<5u7Xz)@;bx#H+P{|~wWPN|{QGA=mZN=R0Y4}9 zV~2Gi?8jkYfE1`C=21JyBOt%KJY0Ej#xSqT!0M3%5-Pbj=)dsGMM2kB``|45aRtO6 z0QRes9~j24nj;`ivIWE!K=p7B^K2hbYbyaU15jnnMPp~I_rc+PX+mtch)MAcwM+*; zcH38WWG;m~AI$@5`~bZHTKrZBaU6-2Jgy-vE+6{S@3I4Dfb%Kt;uA~X_2!535&(+f zCXU3n+Ui7xCpoj09JzQ zl{5HWEdcZ7U&)zdxcBdP)+cAMe8Fo0eEnzngZ&>~_N*!9)BPagS0CA|S3%A;Y@ ze?5Qy=)Bc$#dv|=U*_*`hW*?5`(O2yaIp2`-3I~8gG#Xa#~H4Bd(BelJJPp_5~P?e zsXMX8K&9J0G0@^!#RiJJ4;rY#=}7Oj^xNL+ftL)l>TRuo3hqBNQ1!7F23oiLZEv_I zh-S#Yb{Ne-)pjNZTDQp3Kw&ujQ!U`{^c5H2yEy>&l`ue`KA``f}NecTWC z852kV`R^kfpht0$=d6y*8Czdx;&~M73E?hJGJAI^V4VQ|i44<)OSJ+dU|)pr-tq;& zBs_=qu2R8HGKh=Lm_E;Z-Sym%4^BthS)GZ~CB6|NO!1hC0R5!!bX6wK>sy8}?qBHx z2O+K+YWEly0cGKJAY6j$9jEwSD_s4%ar*E2D;}HpIq(`AQ_LU6dq2MJ1HRukrg0b( zfZd1VbUlQv1pHUVDVEWbAniB+xZhZGm?m@eC$F zDeyZFq?10(*1(}+*+2sgadYhD*brv}Ew^zsP=&e3K-up+4YaCZjDe=yml`Pa!G9`E zdV4*V6$GFk0Q<4y0k9v3^*ul48RjvryBSob3@3n~4BpV%&B4Zsf9wa;$|_B{^0ooJ7`h_LC9Z-XCScOL4csY6X6Zl&5m5OXE4ZN z2v0fOD}Q$wVu6h&P=EsFc4STl{2JHTO6%9q2CT|P)XO%i{hi*Jc-@LNs2f{(bTtnN zU=LYM8H_)8QG$N>;Rkf|=+PFmbLUPWQlkVaApr#k2P2OxFdFyo-`~(b-xu{S3GT!H z&UKG`-a*1B@JEy{R0AK7iJFI!2o!IEQk(QT5$qZVztN9Jg!rMpb7YgAXleVKv2pRI zfR&s-)dBnau0`Pg;(?sNrqL1LpJ>?dHvfm52!fE2zlTJ4N5FNjQySt}kNGg<_Sg?Y zwvPWW#Ju>!kn1HMhHNhVFl6f94@1nq{xIbFz7In-ANVk2>cI~~%nyGUa=q-skj)i; z4#7OYtiYWTE1OtIFaSmXtU%yN4GRn0yYa+`2Pg=O|8STAE>UFvCOy@rw`|hezi2Oh z+hNRV(!Xqd);alk2YpPF-XB^SMR*Y}+SP5XL9E%Vw${3LV&Yk6V&n4;h#JwPCs9Oh zzb5@SUDsA(E8HOte+HGJjifW0^hO7o^u|qkW|Q8eNpA`lW=(qYCcQ;}X5sSs*P^Rr zB-ruJ6RVyRB#I$Wj~%y@;PXg6j~0W_-k!$R+$wrw>u2UxZq#z?!xJAZ&yj?kqe-82 zULnXyR_xD6!3>^*%>?!c+Z9~eJf4G96g`F%S~K!_M)|yW<9r_T7-t37MEDht+t|AH zBIHH8fLI)rQEiYFbFjk7erQPDJH15B=4 z-D~=>vl-ucmzDVqSBpw9cr2e=>m^d|I#>=h+pySt(QrwSmE>D$&dBZ8o<5ccMk7?$ zrf-p%u}+yrEVsVlO5AWIZ@AKK3{fu>WjRe9l{YwY)Lokb8xxm&-es5P1ByRp)|p$) zlb~vvoppgto?U@W;?M${bNrCn0dFLi9qhPf`8@M{UZh1n&oZCK%IC4+f|JkV=JTw^ z+w7^O-)wC4ARQmS=~j^@bp4p{agGzl)B{_UD6(gE7@>?Lm60@MWF%b~$xucbDI<-Q zA$3LaifMP{FZ`PblC6mNhBG^iqm1M#Bds8!wKCF18Oc*d+A1SE`xQsp<@5T-n9#$1 z2_Nt@Yz5bz7@KpO?%vqS5zB*YQ0aZT6GOmaj|nYKS~yl0hMtD$!nP3?=aCoZ(Vm7i zwtg*mNMD#ZCiKkw#?}*c&=Ol-nOg}2^LWFv=JA*u=J>2xS-wLALS9zNE{}bCZHor3 z;L;0LGWOaV`Nc2`18=32I2NolDOl-bTCmb=uWe)N*(Z=ocSUNFUb5b5y;ZS~Ts6+l z_7jii)^&BBqK%iqY&X4-MB^Le>8<87++ z^f?7CGcFKPYN~VLcX96DN(xpV`mSJKu*c0k`_>#zV?=&m^7QIg=|zoYYl^lOp|6Tg z7JXmzu;{lUR3fl3FWWO{&k8R4#80l)#WqRdx6@_e|CoV)fwBb&h!&} zw|Dn0D?8^a0j`L06k-8952hJGR!pUKZgJr8Z^{L2lpgJ{o!H{$avP>-KYx2~%y9 zlVE2}kCG?JSNI8a(k%h`j*WNyb<=h1qvn>!Y%DB~Sy~=rS(XD5E*R`$wwYN)@r{vSkFNGdOZS7{y&7Tp-CxU!_ zzw9BECdrj-m+bdmzw6~U=|43yetItc&yCtfNSh5%LlTX-q>(oDihn|Tx2d+jforF zh7cFoO%n$jE3ZPU7bD-~tR~K$4;VSs=_3cn<1dxDZ&xRd5NEq#VT7eo<;CPBUajOu z>#Zk0eyLoW+Hc~UoI?u_#R@i4tqi!`Wjt>nq90F=*A~ z5uF;{4OTsC!CMRAN{aajRZRVIx91GSl@FG)ccMJsZSh{6!}#;%)A$Phu5t9$Hb;en zvq#U^rn_@gSP+C9Y%cQe@?Y@Z^36qqL~bJFD~b>$h<=o86n!Du5f?5;@$&8&?UAgc zPvy%7dbM^6P_9>7{Os97RPAy`v~BdzIUQv}m#1Hdv)%2lEH=8MI4HTTdY0f)-z?&* z0@<*K^DJS6=ctwBsFeh5ROa1C{v&68NAxoJCOKNfSM!K`^G9nvH;B0nL_U-F?4%w4 z5hFvjPW74U8&$gJ-1QOe#r-H`J1aVUFpDaUUGh7%BdudliC66 zGxDq5E?q*y%eu-;i|JqWuQV&&q+2hs-YKd6C6#7NUz`2pb{CDPYSYnMcr7kX$Jc!P zob&v6)?Y={zqdu`(jYHltE{q%p&>6@6x>sa!;$ z{$EsH@f|?!ue7)usaw1R5nD6@d#}TnQ(t`J#J-f^S;Wvew8fFXu8u3x?{65iASsGqoY|ICwxVoH zS@kcgId^r8Azm40YReuUo_x2fY@i2WeC@6sZ*a-uXbbj3?F%h>s~w?|s6th-su?x& zFE6}>mi>4*chINaGY9UOGX3G2Ggn+vZY4#@*U_^sGYc;b;xLOJ9Tloq(tKMYS`u3L zm(l8$!j`=)CtAL15qGd{n}0)FFIFeLtC6!>Rbri=ZqyWYP+OvIxo)#=m+sJ^st2jQ zQDpTk-BaBwozZDza{}>ri${e9TwvY7eOU6n1U;1eHeRMDTP+i+@3KM}k^eG`)s7qui5lGx6!4foGk|MTFeXGcP!W1`ce(c0)8(FdZ>ME@9F7u^<3S0SFt zMdhbbus%|ysMe_VvDja!R%U&Y^>D$`vdpCwZ&X!gSp%~~S*DE#*IcR{Ek$8jaasF{ zP8IFUI+#^aR-1($XLV$$imbH5G~=>8sslA>hGxEIrREb&vF3>8qUNp!z0kbXm=_Hy z@+q3CiYQ7bLQ9Lbs=g>%TDGz5X3>+PmqmzK*1ycPY(iN`SxnhiH79G(jdM;67S`~zbQkTMRtt$XR2!?!(5};>&$QoY&uVM5kF@PtMvZL^8d@`w3a*K+SzMD{ zb4apShq86sbYJTpQ`dCEZ2nqO-4HHg$%>zsAoy2(0Yc4VMI zW2Av{_@X`GZ3m*Ywpp?$sWq;#;H6U`AEwm@k{Y9xCiSm# zv8HyGZpmNG=xi0-Y-l@?3$wL+FHIm=3<}{S{wyAoDSOswzlb-BPqeeD$PI8`nYWh~ zO^oKMt*a!*2FBN39(4cypfml_B6~QOo_6ubFKui@0|uZmbE&PcsJrURoV1sA^(_6g z(JWEzS-P=}UNZf#H8K|RSQ-aaF5P!MVgHfwBa2d}$8Yx4zYbWkm)6~;|5fNtBi6%O zk@!`ZLw3E?J8d8-g+&-xFOreYaIAWlU=jK`Ed;FaDNb$(ecofXfgltjv8B5y(^}#@ zrm;1C1O02!w<2AaO?Lfj)4eoC(BLy384V3>f_6RZMZl7=gc8>QpX-8*8Hmm>Gof3X zSeVdzT+pj*%JiM#G;mk`d0{TiKTC**03@@S%%0Z1fkE4LY+lXXk!%8P{iw}IZG1y}gJW=A$-|!%3=p z39J?jmS!pzEdWL=LexCeYi*8PN<)Ho`ZNM%uuxPyoRA|`1X6bE0~Ya7dZ(UFkVK-G zj0|I-b{>COf^l(N)TU#r~g5wm`nJWgg6qz2?pb3tMT)W&+A`zvt_V^iXAyGvk^Jjgc#JMZ)@o=Asjz}Nxii#n@eoq!=^;eu70o-n*ihKOU@be$6Bo5MLP^V zM}p+ApLeXa-U1P()L9YNt$mEHTb2K2?d8%5>ps0ihAf@Oa+N5~sg{DfO|X!m?hF)S?AQEchEQqv}VA*=-R*35S(p3F=nvRr+P z)wDf_jbNGk@iS4+b*@Q1eTYYdJokeXOjC7=jqx8T)MEh(W=iMA8c9`f1VWl%=^I9gQ%UJqnbK04gru() zp^*!qbIVX<@^q=dXj0lN-}sTzZSh9ZCCSpbX;Q7{yfqW(=8p%*(|5`@`aSFXZWgH7 zTgV9D*T5QobdS4*vxUQPh;nQ4^&pc8luQJZmBLD)3AstUNgO~{eaZQf^P}uM_X6jd z5IGGS9bwkkT6w$QgsXnZ^q&Sx%iIakYY(pYu)S1(#xFn-zP_>7&;q7(k*Rc9Xgm@w zwS-Dc4COd=Ma**YxJRI0w~!f4dWCxA3W0R@!?VIKn)FAVD}v4nkBMsqrQ@y(6(Tc# z2y=B370d03|=xdQ^eh3Bx3cn)e*Ta-!B&= z(qqz>`*vn96{TiEZ{HcKhs%;jN=K%l8M6rUT&DVrlD(a&FiP3PT&2m|IMZ_1Mg@H+ zPujYAHu8@_`{#Lu-OYND^)|~~Gf3m6p-+#Uq)DesKhmUVE^D@G&{vw1nyR9Qn%^{N zhi_P2;Io;LOzFw+Y9W|O7_>N-g;>y6UujovZSsu(g>jKI(^Wg+TlXp2xmuN$NXnXPvNIyqBLfSayXygx+7;&{j3t`B`VBaBRU@@$QDU21h50@-Yl` z{p)0JlK!&wte8NNiNuCO(wD6PjP)$j(XYhMI&Ez{!wK3P&rK6H-tHGsXJy{W>*PU~ z;G7Q`z^N4kx>%8fl$(85mT8)K(`CEk?wj|xn=D;CKe~O>2rbg3aw1dK{c`e-2jl85 zaI)-?ZvBIn8z=7sGp^o=zdN`^t=3IyGWTk)Q0gX-v8AaW$p}+a)f2m$_`cqesG9FE zS7`3VjiEj3v`GIX!W7-P-=qiGpVOotf?f?rWJBA|N zCgKD%u!|Wz?*JXC@`YSW_<_wWS%B`oLyk@Q+hhn)Rt9G9YZB75y;YVKLM(@a3NcEf zqRp$N#6uM_Cj5Ke!F2!Rq#se&rb4qNWLxX?200-a3l{S?j1b}wu`K{59x&IncQmxQ z96)rku~lS2r!}^Y9oC-?vekSD-S}bado2wnsrOnM{jXZu1SV)`BCzja6>SvD-!u}d zXH9zA^A1qZy0L~9xJj0Gr|gQLQ}T_G1QfF#(98la<@lC_Omx`;npqx0*9D4M=zGnK z%0mKZA=*>IQ>Il`$FzkLGHG8sV4R zp>)p!dE_vTC$F*9Y>=m|Z)2;)aL*$STRB@fc8#r(oB+@MjjdK;o&y?N11dfHeN^u_ z!m3FRlgB%PI18GHqQBb#SN3>UcF-Grz^{678SJ=?b!EG6@eHF3Dj#TkT2OFcZ>Pd7 z8H}Dii3F|LK@4;2zJ|6K@s~EW^^VFVV=F$&^?+6Sl0$B(M$)Jx6nDri&6S-Vg6_n- zvQrO@GpLJ^DX#2AC*0C2yajA&WXd78*kf)9huq>^*`YCy1XdHb`UpoMmc#G(0Bifv*y`zIfBTV&~sgsf(V3NZaZS$;i-8 z#p~V9erl?|Qqb7y9hygY6-_MCb!C$Vz5iF?RH0jG69-gCgXT}`X=y8tQm;2t z^f#|5>h(s-NRK|t*s+*Ksc1Ak6@z|RNxsg^T$*c4=-+6HSo@7OP}%G0uQYnpk{*5d{1@nXwpZ5Iuwthv09(l z*%sERf0c}sZ}fODL@sLACz_#@SNd2BGGkGvKK6|sOs%ilwy9p}<2v;TX2^jMgc&1& z4;n_qo1uBI77(BXo%&_3^f8@!+jNxKsgDmQ;|O;mGTj(vLKNSwf3Nue98>?gPCKi! zT~gWA-O<&dz3S##w=Md9z1Bg@BSF!(kl53;vpc)GIz1WhRe94tsq)6Xs=N`irNhzW zcTL`eb`Rc{tbfg(Kt>#+F4ax`BKRifH1!B6m|ByV#4u3dnmXbU$WJ$2*FH6&1`Qtz zP?FBr*Q9qOObA#5nE2AoU{(NaztN!VQ&M*js#gS0??~E=XEn6xX*{j;)Qk2ynoXTH zWRxix?_EidT7zbL_9S6B8=X-{Y=F-OV z7!!F!D6f>87|ku^CUFTBmyvM-OmhZMszEC#sRmJ_6%gSA5dUFt&fa&U8)KDd7+IZR7>XK zcL)jOt`Z$XrChJioJVgMGU+q!krD4#312nSPPi@RbPDq&4Q(b7JZl;;473m#!AjwN z6GLyZj-Tj|swLV^qbh;GteDrNPu)Lqgy$WUe}dS3Mp(RRlmKrBl@4ptzbZ_19pOnJ zv)!))g<+#!wid>i!0Cy+9INaL+;LvJRdVs(p*~_!f#<@}gO+;P)u6o6z&Y-&-n%_^ z$w$#qsq-B7QVB0+sTW%9mhI`nA&v4z*^HBmxjy0%BRpn#UiC3uI2tYW8n$|rlDO=6 z6pp_wbBv^wpoMqhIQRBF1hvS!F<7^Mk zSr7COZ?~0N#?^3-z=g)PW6s?n#p*MV*CwI`)jK_LoEj%`3T|l2!JB9Un`vGaLlJ{B z)J<66IYln!iJR0@*Ro6PWrYf}ZvO~auJmpBmMh{tz0UiYM7r^m?+wX6 zzD>Vdolz}u=`Th5yv1C>k^w!^Ro~!g7 zjjhhI7Q`fv5+!~;gW6z2(}aFwb!4QE(UcL6iQ%Q!fqVB4_^k#Fz$E|1>Idt_GHxl! z7I-&kdZi)_MZbktvVV*j3G2heO=#h73P(h)@>N}ueV z;v&r(>Yd_K=;57mC%94(dq$yh&r_kG1XY0|KQ%V*i+rUj?@p33Z_$K2fp-dCL!QVJ z%-fN6x**;?Z{um@l9P(K33)YeVnD0*Wj+$8B-6^INC|4_*QahXhtswb$jPP9A~Nix zB6;tQG~c|W*gQts&)epe(*uQ@i1qa4Ww+;-8uTM(GZ|rxl9Xk8o;Zvw8Clw-SBjNl zWF6)#bCxBBX^Cfhe<~O+MjqsI@pG{WsUbAP@im*2f=$Y|Yi=oTDFaA%apgS{-4@># zmy$b#JA~ol3Wp;?)W%uouwKX$pLRGSwA($JLyqQfKu0YZTOyPhr!p6p|IBenbzJN% zMwO3TrVT46Bbbp=uieBu;henC{(fbVrK$OWpBKa@71%8DPMLC2u`Dz1m#xPuHkJg$ zT{w+y9!*uw$#eQRa$HIhT8uJIDk24yinx=CMPC;rR4T$wD+N=lYV+f~Q)1%tA_c0v z#A)Z15Ax!MsP`Q#w7zN`dHBLnrRbW*d1j$+_(?@de!4O*w3=A-Aa8yCSNZblWtXn! zKhdVVtPtmCRWqxHprDBF1C~|B%=1pE(?*sK`Ki!O1{7J>OrF-*YRe8zj=MUz`=UgjZp6@*%d8dqT^iBziNe#TN02OonvVv*#E2JyY?Gq4)+5I}7)c5H zO7~x{m{}Muzf&MfomUW9eKo(Xrt$0|WnN_Z-W>w>yp)rQ*g5B=3v0rfe!7#~aw>me zciK;d#8){jpSJ94In{Fgw_C&wkSs0krYE4J+hU@7M}|UaKag^ACln*9YHrrZDfP>m znEdnxUtM=w<bc(tS0&BM@Xw37 zz|FFfcQdc%C$HtMe6;P>>V${4msOLu9b8dk(;vN@%$mb$ZAr0U3TQ`; zz0I|Ew|?Su^Qc2f>YTi3m1Y%IDwKJWTW~!;eu(H^mypeIf zW1kb7e(QSvmCh44l$9}yTaU%6x3nHpW?uRBMr5@qhkHID@6k)Un3mUeG4%H5k4p5& z%cVc*&n)!tTk&`U=Y~~4JNtt8dj5vi>-meMdGhMw ze8CIHZ*5(4ZO48jma6g|7BJgOxb-#_CXA?lF`^2~@Pt8OggxJHNNiQAE__p!dV9H) zcRA?DkL5pG=SjSavu75LJ0#IcZb(e)C4QRWXND&DSeQo7KQRGO2uxSE$-{gq^cT>82EkX$RjA+MKrPICA)?H`jDXFMua&_!#E zk*bRJBtulLpalyqfvQw+Ty@3P(A zY4P^+g;`x$mKwD{FeNxc;csU(XPtQJI&t(3&H?tt8Nw&qX;h%+Mx*|3&fbwr0^iI! znjKcd5V-{oZPTOip|3)o$E6Z!Emff|BUTnjXzK34fgyngMWyKJO$mecW$Ehk#KE!* zO^w>Qv_@K3O5RDOl~xs&(%LT&RbTcjmXS<_#Q@rFl0ESzpG(_aU4bOk({E0?I{WjL zjcI}Y5f`+t3eG{ZA;%?7M7am?7w!pZ@XLXb=T|W==`~k zfsr3s&EC$%irz5OI7bxdJZ=r2MKe%8tdq_nu^(1WP0RkfdA8B*Wsw)OxYJK4t-xLzhNiXR5 zRk1r|LajnD=AsQ__YsoF7*W-2hn8`=zqI{&R4LzK_&x5|c41dXmL9b6yENyl_3Y9* z2~9dn?RcC2jQ`p?%gK0n`KciXZ6@3~V>7Q|O1+e3)%ayQ-Zp(>)dn}wh|&BZrcr~U z#)tb2v7TcWM)wMzTe8UI$tCSQZG-lmcD+Rnzeap*zw~OYcu8KD+FD^NwiPR2F0OHzLdE6yKT z?mI>~?p*hI1vVw04GH*uTgbe5WW0Ub=@b)Dk2WM2P9K7*xd6GnkczLQsFwvs(g(;U z+%fX4_YE2&pDMpg-c*K8OPtc5Q#`6fVda1G%c1Sl?#x*&HF{LYx$#q858{wCwn~Zm zjg0wx#@h%bF*0W7%>5NhdO~8z!{M}^VPkKcrl&rLGAp*dl9I|B5U1w1OY==+F9Us^ zs+IXm^8Bi~ANf@|M|;l+OirAVBP({zUd;-6EFBmSs}Q3Z>Ct#&nAy8C9Md9w65Ss2 zE}CwUyr%Q#w89BDxY07zsE;0mBu?2?jEYy@P*~ge?efiBz9;bKyib%HRm?9IOeV`z zRlMaFRd?sVyAVX@Whe`hHEYuBBGvr30{$Mm4dXZWz{^oit1qC9(yOCNRtzy6ttOZ= zxUZ}`6%0{<#o+Qab4!vojlI8dlYXs}UR%0}AdYPAT0OU9g)F^wZi##K!ToN5$4t>u zrHPThb>3GV(^@!=`06gU zcW&MtL`3iF4{re)?48t_P-4yigqEFC8Vz0}CGMyY{VmuhvB9(Xl;1VtfeMYl2G29U zQgXrwju8)HA#Cw{Z_9**^?O?;`Bz)!^9~y9I1ldIaW?;B$Jv(&#>|B|MzG!dqW!Am zYkE|bL{ed){hF>XV|=Zb&^=hNbq$-#`@W@{!tOFP6WcIPq#L%EOJI8$wz%y5`O|y1 zm*?}Q^M}Cp@&VY!o>W5n7VMEH8D;$1Z|OAHERH1Ht&IorSWmd|eVfHUTfDY;{08=P z8KxyP#xCn1$EKh@+s;_9?QEt9T{cBT*3i)vI=Vnd)pSHvWm6GVSyUEPCF$Z<@sTlA z#ahX)GS14QJiy;|>_@fYO6f!!KQOn`=Y&EQumH3*CrR=G8(gxx~jb|{n{e^A3Y=*U^r_diLj@?>!^R% z@%UZGKWKE?!@jLr(qP|#ZUa{Em#zACj>XH?w$=^3TeU_;60n&kRzB}w5H4We!8Y(> zT^EC35IzNtPc!F{D1f#0TvRo3fWVJ~REt#N^j0yA;1l`;S z3^HI#Io8ub(%u>|ct!-{SJ-$}2*9QzVcfyTSSK?b_tV_StoJn@N83|IH>QyTVc#^z zm&w_;9~S2gZKe6lulLh>eA}?Kxr|fB@iN`2~aC+vB`y~9bp&*lh~PuNMn-DSg$a+c=pyYd@SVK50ZuUjQ$Zm~vx^1U&+73E@1jgDiG-09%v<-w4SrT>HZVmR?iB2>-{$pXnqv71-2 zjeD)AagP_vh)gx?&v^Ar^Ccquh$=7c9|G#adjYlT346v~5CHFYWZFplfjl~F$pl61 zeFl%wE25}zt#abnaQN~>O_Sarp&lO23Lp8)}NNQHc`fU<9kk`vEVNH+rAec1Mw12Mmgr`q2S#E`IIeOAyXotS?} zC&N~YK{~9BXhB*z>LpiQigRoVTKF`Y9om-(y{pD8);B@OnCragW6 z-7@A{(~roI^~AWh*7rF~QGXF~6-^L@h+;(PLz+DM#FI0KryMH13L!#GW7)8VuzXlk zSaVsi#Ag)s38Jwm))s$qVpXv4v9RZ?<~J-eJCd~&5v*c@`Z*nBHd6koSHj7R zc0INY3s7`0ScBszJ?v_4TllUmozkZg$eK_cutV|PdI6h?CCbfy$04okg+XB6_h6P+WPTE^GUcJ3M7&wy1{r!(lg-1`-=DDy{C{gW@OMlH+$H+ z_Y9Qyth230N;e{(BU1h372OD$&ya}xJ&_)cY`Bs4Xc;tbGd8v!XW+tdXR#9#KcUnF z7+_yFWRxmlj=R}4pV*|oG)sTJ$}<*K|Oj)upf4O{55QQ zL11^g{$v{nUDzi@8=yvim1PvX!wbR;xwK$$HWa>Y=yt#hc??CtLKMr#4W4F)5{8id zh|~e7F@jJIK}mLScxdFgp^!?3l2u4>fX_@zBjYEccs^P}=AyWHNUwj`ir+d}gC~+d zIsA?U26mqVzL+EL9tbPXRqOdX`3L#u__h2x`&1>hOrGEupPZbQ3vYk4j;4284q8$!UXvRB*(@t ze8n+#ZPuHpT||~+-~}9GxiekF6HF70n<~l|AzGsH9a|wfnJU^LI^Y^4P_~GWmXVYu zP$hgXBKRbtscAxzimX7wH$YC@wx@ zY0YS2AiUth^YMB^zvSo*JN15HDpqp5EVYfDC|Ne@b1~W@dC{?5{Pb0hu1{5H7%b!P$J^F@cr@E09iqU)@%KYb`OS2hTlH0zmz&}Lcpj*sAdYzt?9 z6%`wil%5VfZMneJ4@YS-cc!6_7 z(!%(Uqj$`Ro;f5vY?|tLbVGaXOtLwgA*&m&*Ng0l!JTqXnpvp3C(W{1KSlg_6k&35 z+5ms0cw(xmJ|pG4a3q-{Bb-o9oghofUNP#~%a|CRfZc-ny*>84bGj_dY-SM4!Ezv) zBAz{(A6hrZhbQn<4RCOfg(%ZSi8N$)W)wR%LD!vYk}b6Y{w`KcyX;{JER@Zgml{9s zz^5TWH{2LzsGrYhT9-w-H^D}J!O9dW1zvue247rbKalt)xGwQo%<&n$E7qGJIFZYp zY#Eact_NIc873T7MeC|)K){G=_q&{m# z5)-z}Zl`nyqA8QfU+j~lF1XAb@k5}0jz;~dX6Pr;{++3$>axbf$>{YxjpGEn!Sa~- z5z8bg@%}VqJ9Tud@5gMK+d_d-kUm#zvLr^*;KdO5O-lP@i8n!+A-|B+__(B@v7d$v z&a?LMGRu}78j1>o2BATOiOV2@<#KonbtonymmmyU3kWv`FfK|SIfBb{p^>2a(6i2nPU1cpF+`|G=V>)*&xnCXDxpzgc`lj1CGzV^{O z89Q5AXIHGfrCl$x3G}y(L=)4>Lo&6dHTE?|e3B#_LO09KY2`Idtz(voAHV1bMzUAh zwQvl$=9?4y!nmOmYHHx#V`b#oSpNuGaYR~Ga7GQ&TsKH(DkYcZF~iT*$aM#rS_e1l zOIU-$O^P!}qBxdO+*5kmVZCmrZp0lS;jBERy{3EKIef#UK?FNT<`B9vfE(I98jT`G z5in-@D@PG^HtS_|d`U|vi0zh)mUS(kwS3cZwxy=!Q44BsVNkZzPzsw;qN&ArHM5QS z8WtkgsGng@un9CNPF}Y=HCt!NDyb(5i>$Sj|N$?iKDH?)7~s7w9h$5c4Sqgksr!v2Hl;`RAF zrI~C<{=V^xdCH$<14QNaqVj=I}a^V3{xr3RyXc8OA5Q&duie#=tB~h=I6iW6= zPDs9!{3NLp8=-5Tj^6cd>f|r%z}R}BaO*_mC%-0dyz#AkuKY9k)tze;SIgu|`ORIX z85y64hNo+l3sY#QLu<=Dw=GPSAV^kLDuT7~sq^C1p&cRWdoHIdrRaOn4bjWr z&3dEl_xalW{;n;kIGH=KSoMQSr#h~pW!Yr42{L=;(RHnPFyU{KUYHE1NW+#)76bPcGy1Ib6~*Wa{k~m zj)Sib8C*LoD(Ajv@C%Q@*3;FA`JHv5a;d0X%@cjr<#6`xV)Rq#^SDVX&J)}XYP>>Z~gW_v4Yv5h4nnN|(nj1CsHK?=3 zSZAjjt{bnD>y)}QoitasUAJF%TK9uarwdz=w~^!s-W8;;F5X+Yh zX$N(KbD|TDFxanf-ZJ@=_}i^T1Jq>dVyS~rRrWF3TDGl4v5v6!sI0q@y12R};>7f0 zA8in%w)B+dv<#pURy%kUAJ?iUQVwU&?QLt}m9eRPri9=nXZhj@R1Njpv5;7pBS6my z1T~*ZhS$&ILHf3)J%m@E>blr)+&PpQ#T2l@S)hFwz39k#|4s5tkjn4$%iwb?i%>>F zyB-Sxl+>=@uqcA{?4>~p)OC$m(yX6DG_-?cPlFRBD^Nun-mbol3^SdGz2S}xyzJAYU|Vide)QvU`N5;JqMp}prgyNS~Vf02xNbYoBl zKo@G(?=^+n&pFL{(7h~AHS1X-80m1yJm8J5notu_#huhdG|@&(;!f^yBh>T= z`ActFYG6P4F!?z7G&x%#PnGA$Kb=B2hdYH14o64XXZ)qnVbO8XOQP}HT6y?&t=j0i zuCR7J^r=lKARDEx=5tj}_KYqsUK8;_w|eKirT$G7?pLBo=i%s^UOf!BvUhquN^2#% zLg&5H`#WqQl>*fls@V?2F z{a%K|Dh9n5A<0;Rczw3D=GmiGT? z?mNJmOu9yA-Xv5tg<>}aP^=^rvFAxG)F4;`imq)1Y^$p-i0Il@)YaX! ztWj52x1z57LlFfF=n}eE$bTk5(C^>x`|iEZ{qJ+L&+CSh^Uj<(bLPyXz#A zb;Y{lx}S7+bf`x6R@b9UQ07|ZT^3#zUxpTztt`2*#vKwVj%3hTrtBQUVBP!?$ zWkpQI{EAf-=!=R&6=y51Ry?X`sxYhTRfmSujjNkn7g3j1XFAPAaGYEBpiW;$8Eg#$ z4WkTbqG6^X34%s87)apH6$83wFvTm1*zRm!_Ea{S%U;Z0&n{vgg9-T-`xzU(W_RNc z;EVZkei%QFpT$RO_&fNA!C-lvU&U|XTezS;E}e!2mzA)SKnGmTxcuSr(51nJDYch6 zOVJoG7NlK#1&@WPmDw84wiGv)e@K>PeibTayMZTg~aa{3};*J8s*k8IsBfsBs9&j(RdT=N5y^D2GC3xBe)6hh^hNTa|F zlOH9+`J%R$77fvk(@xe#Xw$U0+AZ3BT69YLoA!ZLuch*A^9JUP%0m^@T-+#|keN-YR@nh+Y?VD;-cOE|r&t zmBy83m7+DJJ4z3io-4gxT2MqpXsC!cPst#Eh`WZ$TXoJ#V3eQ9D)094&apZ9-53N2F&a*JO?u29fbX~ujxJ5(ZMH` zu5EhW^1J~PBZPrM#Ly8TnEwQ<5N2?M#pq#;%IoU^sivW!8TQHc?O+)&Vf_F$Q(_JQ zyhVTr)(Td8c|9o*a9chu@IfbP28J?CXwO?}{&l_PY*O}n#Q#7bu^y1!|A!y2;u$k6 z5ZG#XN@C@BwwEabgO4#eNXd#&919R9_|*@sg@3C6vIZc8$tH;7%250*Kn8U}VgPa< zAcH$0B>*w=eqRoN|C>GGy%j^X1#E1!dhj?0&vYLHVjEu9bGjQ{7@NV9J05JB%?e!c z5{wh;NeI@v?sfGo3e59D2a*3IDiJ)@RcPqk27L**Z?-j|kNw@35r229YR=SVFrtHb z6?k&bFo4C{@~jf_Spr|f zQBbYE>bw#szn+DrMw)pqr}5?TSzzt+Mnh}$Gk@^&!oSW!v%s`78EPB`wEqcwV1yWaZtCw#Y-+D>Z-xR(fF$))D3GvrI8}HE zBq7QZwb#o)R)yB=^vV`Q(2XJoO7!zy4gFcx?TtW=G!)laC^OTjD(f5>X} za_B@DAAifop8rdHydH`V0g;nUe7s>h3%~x5kG9a_BgS@j{X>$kkPi*drd9=R25|xp z$j(ec@U`p43B2<~ zx4}gtUyzDZo_SU3*tlwN!ZO|aUZb) zc9rAh^gjjw9N*^Im9>cvk|`?S+M)Chp}r$GT|IwI}z6# zR%ut^yp<5P5rz#b3|PMkQ5KY%kr~I%Vy|KEU>|0G$ND1HY}Iu(c!QuC%$`sO3A@;h zhIc9CoQiW9>uolQ&XB7tT19g1TG()?S};j^1)i7%+mB`I3NH{eFSQyA)xV`(T z&TweruJ~D>T$jGKWs)EZ<@W$&q{tu}kx4Ol=_mE$uDnFRl z!Iv6He%9y<2?Wecu@{WGreFt4Qw9MNa{*Woo5stzQnJ9H@bdrS`P`}^#mh=b{A|jL zF>W@)2afC=N!VHzhN=62117Vw8I#P&rf>^k78n~;2D*b#gOr9*3xl@xx>QDv=4lHl zG->7K%0||6l+@0+CJipK^kS^VZVc))&Vs;dODy3fP(R|aHui_$kiQh2ju14nSnYj| zV8-0x_AF0(4F2R0dA(58yT>7H9*MG>&=k5|dI(cZW+|3N=_#4}RoKoDP zr+>2FX5(}SJ24LPCLHhe+?KHoqN1O#$bg}~kvx2iZDc)S%yEu12K}B8Llud-aBi zQZLOyvoPZV<$$dn3eFy3zH2@xq%N~|S&X^|Q|BQnO?cd%Z60o$JDCJ&A3ZSon6rN%uN54=f?@djAT_bZC)m=l{pckSVi3WG2!p)>mG?1@ z>s_(SUV)-be;Yytj&Wc}XD_(EojbLNCm9(q7F=8Q?9Rp;trh)!EeH8YLXmhg?d6+K ziFTi3T7ieYtphksoROW8T=ZPLhvg-sh=YDF`Zjpzx9MFs%SRwcJP;2VZ-uDB0WMB+ zm(!l8WsszO^&w0=EOcP5NT8%XO$^DEQ%@3^^HMy{e|(OXel(HjOn4OC@L;Z&}QEAv$j1EHaC} z6>iI+4=Qnq5=ZoBK9WIt{(RMj$U>(}%M0g?&Odq`Q(?9@@wFYlR}GB?<6fq4G>A7avn1m`Jh2dzRY@}qZo zv<}a;b~+fktGHX}q=_rs(sikcw!xVtie=i6*nML2g5*|OaA1uxAuG(em5$7$Gs4cM z_1+%2R4K@$lSN9-;DX+=q@3PfF?YqiWtN$%=h+6gMkWq{_T+^yTnESDVZ8A|Y<(I} z;*Q3rsPC{I^&aLnRI=kcPPi?^m;V%3?Vq-A=QjH`X_Jyn^K z6S+(&K#SKLEKY8uGg|4m73X#-vh9Oss1}9TrLGQnw%?EwEfQ}^7HRs1v__`Ozve9W zUmu$eg`P}X^dMK=Eff#235m+#7o5!8uzY#Qf>5S)Mv825;D%{a7Mz$7Vke8rQ%)M@ zb}}U2qu6t-gxW8e|AWYS*=hV8cVKM3M7DH0$Jg_xxa=ru&a*Qz*Y7>>`KSllsFTIM}pHX}r{aarjSo74pR;2~3U5~c>v30KZrKV?JY*3Y8G zIPH!bNDD7mi{>tCaPF65KHForrM1(O=FKqiEIQzM$;IK8&~4;(heqAWkmn5(ZsF7J zUQw%l+Td^tROhb~AC^69Orno7OYtA-%GZTX&9k^>(VQ&&9sFl4+Hc|97GfEZ^Y*;I z|F<^a)PXNhEo30f1`etS2yQA5dVk9ZD z6}cNGQ=){=HWQ=!!#A7sk6#7J54$~mO7fv;jsCxv&qv+6O2_`RVZyECTb56s;B3L; zR|VV`^Jl}rwt5luHUouobD6f|T^*KH+>ZRL^*+QHVV3OL9BZdXO+Vt7t#J}`nYBU^ ziB?EPut&9a1Spyt9oWVg9oWnetFDRLoeQ(%2LXyD*08#OR3k0*?%=~l@-X(RcuPX* z$Frzl5`oCV))>5>z@6>ot^{T0DNR1| z<5}7`LROK`j6fewIxh#Zja)zu*&~*7t#}=gf$!FxU>LKfRr*0L$8n6)X8D-fYI7Lz zs3>gFn_SK_O>{-R6O5~ zjzO*BS?=x<8FXztj%9i&!n%9b)RUz&qz{4lzg5^O+{Od9CCNC^5$+LgjT?1*ORua{ zFFO99)K?pwdtVylc7z*VBJPuUR2=i2IR2#gz&(lJnmmCmPY*+phrI-`Xb@Yjv7@sh zz7x+959e7?o;<40XB5inx6Zy;HqJ>WJSMx%3UcfAJ1!CTjklwZdEQafD6-!w{u0gO z^$N}651i@7S~5M=3k-U;bdZ~Rmz!b;-1X|fSujQLyL_Vn{U%EeO-c-7M*3`O7V9ON zuf>9I#5W`v!Cr#QX^!#%wyfdKb3bEd%Wxtzb85hN);0*RCK}=JZ-b&-PR9w|{)mch z)Xk7QZ@3kQAw<(jxWrL+Iu2CA$}w5V%9W00>s^Xm@Hc~#3Yda?JQ^})jB|7LAU~e*8-7v3&e^YKG)waJYnf(nm;hBH?12qvF^R-6I5O@=NhN zpQxcZ6K9FLCGoNk2Itu%lx8_6CNE8X{0dTDO#N-pR6iG@5iT*#zn=SZRD471tvK`f zmQ#H(6h1m}o5c)Ms*Cg`*+sYrQe8|kFM?DTHlZp{gstgRigH|L2rJvtGYYVBJ1-Q5{ zpLe&wjaB{SZ5C=T+_SCQuA7_L9@X0~AX(|EorSRngIKPM3dgbr*rXjcdR4aC;HSH> z-MM^TkDV6AoEN=UFszUYcK@(M2O~|`v&IZ>qus@dC?FBHdxrvo#db^mftq@-2AOPN zme;}9__Rrd!FC1lx*b`MR9FRuB^Cx#3sFZh7@|QeNC658>CQHzJ~DtpGU}u4rd9b3 zvtQZ4uRpV9Z^KNYhDO{6J=A9>W`y9a_#c?%WRJPqFe_t&1B&Qv!{I?zy3ss5UE0fT z8>Tcn5U8awy=@$~VMa!vjI)C0k+T&KzYgVW=Jw%T2R*bI2KjH7Kn-a-Qp#>*RMna7 zz|;XK4b;%4UbY-ts=7DJ-k!rVJAj9pnxvdO?-Y)clFo|KX`^hSG5{`H+bS8jv)(l0UCPZ&?0Y%otYDz6JRM+YCm zusYXEx1y2TD9ygn91ZU=$2!`f`x7C1657qe8y^nTI)A#8c@Kh)FELza-o~kijuLl* zk>3q@2y6O>8z9v9K6v0iKO$Ujp`~RwR!DL*Y48<bw8YCIp z#z741Wbeoe81E$t!4v96)?wH;@A_&Yyq6%nf{2=1f*OUecqVFYVli*`*A~Zm`|)M% zlfAG0Jhr8NFg-%HdOGMigDFeW0;D$jBu5}kCaN%INy-E#K9%@$+AR|WEvKV%VA;G` z0L$jp@`bXruqE`jvdGnR!Xi59u6ULplc>3Cv>?Y{m3~tmyIByML(hLo2gKZUE#}P~ zXa_S|l6ml2k9^C6Wcb5+;78BI<1W*~g>Lhdjk@8o;O7k);V$%H2Vt3`ZhE-IC(@p? z>R6v_W16dbt1I5Bwmg0n@}nm;J?!cQ&KJ|eo?YNzPd=|=r1ha-Y6rH45HO~bT{kP= zxNgrI5_U4l2~x$|YBHvO^jtXyhBQljlW~6A^mt`~DkD63WOvJzab!#%0TrAnEYjdq zri+%JZd}19QroC775+=|m*gPtmE@HKtruP3Uf{Y>b+`^^yWYo+_wgxL`5p~N?sZIa zgxUx5DV|0h-`6W^RyzGe;&w*L_qyN;OXG<&h=yE z^~}p2oLB9t<@qdM!w+2t>iN*D^em=;=6k8ry@osm?X2MJ{ZxPg1|rz1wWaP@p0q7j zQZU%(GJQd&(b9qy^j?`TV+mcdxgcFz^rej{Ir4cr+AeMShEJZ- z3r5K9xt5w)cc+}}x>vaFz}~xq@MAsep2IxZ?PXC+)SR;eU%HN>x})qQRz4kNmATs@ z#{m!4hOa*5@yD$6$b)u#`8NDLG}%A&DZNIfnYf4+>|7RuN+xZLKOy&nbRU=GtX(%` ziSiBkl7br9p`5cmNh@f-&x0M%q^iQNp?ULsw?6U0nr1I8SNwHYad=c%F~sDSrE+}M z>?-}HXiB=*z_7EWR}V`3jtT1IY3JpU(JxC_yBod=)FJB#9i8_55!MRb=eqs6fX%L# zX&iG!cTe|1H!A{{rF}n*#a;D9;WtHnHwVOC(ERvq4!T@;cYD*G#=>t`w8;6qV2Vk9 zT~nX5hv|Twv?RY16}(`;tn?kAH)Z%BF9>f6!s|u*_w|}TD?KP6vIs=mM-{Y*8ZMIz z^s?dlJiBnjr*^ZTO)t4Qva+8Goe)VevhQT?vbk_=*?K4bR&>2Q%kpK#ujBaP1HG(5 z?-yrgqCopw!(8Y;t~2Hz5*b49eO%)!Nc*)ad;3E=dnsJ1bMD!`)UwLDs(Vj$&rhW8 zE333^y7#Dx(bzr@Q?bnJ)kWJYTdIqAbp0>3O$FAtU@N}&sJ(a17+xCXbuNnu+n_i2V}sA3?3UnNYT>n<%}oi> zx8AI{n{uZ4>bt;tM=`9KQ+Sb7OZ8ITUv1gTkSc?D?W)EyM%c$T?QALwTb@fS&J@sj z?C@@XNeq2Zx1RPc&d$xBy)7fQ=OW@G& z*u$g~#-Oz>SJHR8oN&49Qr5oy{9g_ML+{%px98|P2gmSBWdTsHSx|E{$`eD5JO;=r{tlcYgDI0k{E96-VgzNp5_?T7go>n5e zcs7vd!}?Q^?Uy7hvT5QJ42|6_Ox+koA#J`HBR{{2Ieh}YlQ@2=pY%~*n^eVe#byQi zN^ugL&+aRJQe(A^+9@3OkYx!|wGrbo)l3Y>4zS>cI{-4kp&CwSe&F2R!#BlSfK&6H zH2KDRk2eb-k)-iJaN||o4Daak4Zl8Wvv_UHh2>ryiCgQyB)_Pg1YVFM!n^>S7Kmz5 ztnomwTeY$l-=2kLM4H8{rJrKgpck;jhZDiGaAfvNC?1&u2QX3!EQh&IB604M9OPgg zj2r3~KZ}&&ufkxNkc$p642NnW@t|pNTG)LIiOksi%6R&Au^hb8Sv6o6eA)!ZhTbGG zm%BY&R9UN^@&^43WwxpDk2OQlKsZJfQg3S&QtSv}A_Xp#2upv{EAS&i(bjGs&`|3Aso^81*`>df19vrCEhdytBwGVKM z3I8)^GRfh|zx$Ab=l}G%|ClVY>>j^*YcM=Fyrn+HOw?wuLe(wjoOr9f4Wa@LErKWr zK9wVa%wmRm{TPPb9Ai|0Xb5{YJB7V&8T(P2L7K;Qw>-y&!)5IZ=w%AQGUgqy$q!7g zuMO;_m_l`WONz^OU?UH1z(Fm>Lvz}9nxW%t*w>)>>2ZD5xUdx=``+n!*-@^O?`pD#?|bn^ZoVGKl@+lmEIedJ!Wuj zVr!l`7c=d@y$@D)Yfgk^< zOAn+CGSi6eE_V*ph@Rit>}<6nhu*MAg)y|c{i=q<<#SBvJii_@lnnK0%+ zyEy%USOYGg55#Gw+Q+4r!LUksAWk}9L2D>~K_s2hE{?7-qvP7eF%QI1Xn}3CfAwgA z1)cU<+D*Pv;RlJ~II>cYH>@ZR@8iep%G^S4xjBZG(Hc>-Jo$k*TQ*xJD5M#(kl43^ zE3(6~7&AJ(kQPms%`>B|Ws;Dz#-N3+Agb%TsjrK*x7@}IRv#W43g>8gSKrfNks2#{ zpDf^4Wh~0N9JannAwZcxj4U!luR>1q8+;S5a&+tDwb>fWB z$iT2kW%OZL`cAnZ->*e&k$lAcB5VGTlTaLzX5Fi}9183iaiw z`gzon2DM20%Kxxyu(vi`8?TM_kH0KWaE?HhvUReJ3wCM0$&3m(6+&rQaZ~8z*}*Y> zYh>}VUt~L@EojL8HanQHJlT0SeI?Xx^@3)R_d$Kt_w`_;3*vhYT z+ToPV)^;o7Nlx%I@$mf{xnUG7i;{|86kaLh$Zi!|m6xvitJG}Y8`j*?(({1BU%7vy zm6v?FaQW5k>Bk@!tnTwei&dwhLv&krI_=*m?EXj4QlHqbn~qt%)^+=(C}ndb%R2Nr z{MdK@MhpL*pIzOabHVD+r5y=ghfJF?x#4x!WWk zIkrKd*)?;npJYL|iwtuvnks2%&S(3u+kgG({FCPE7Y!YuC^ADwC`z8-Dio3?;lU8M zRkBlE=`)soFrMA79e~}V&uQEC&mGzz#nd9w($94u&zCEu zZB>uVaPu{3Yo9j^KKCMPn`D&(=kq>Kn!l2)+AEoL5j!q*J(Q0n1tFp$r%{hF9Vc3Z z69u0L6CJ8s&i?$2)q6bLqjZ1Hv#U{y-7TfTeaC1e!=#7ygt=eqKa43j#%ns-^o{?h zH&l;M=0aC2j9`B1y02hg!9_|^ASuW`{%C+~fb1zXOgK!~jp7M*G1p1jC$5wHIJ!M( zS?_tuAG)F6t&hdVopXD(eRIq26q>Sso%D;`iVGWls8AFjn1xG56#onmfuA=8w(<|+p+Mdow(LU3B=T`TBJ-bm{ z&rpps(39T?W5Efs?eMN+vEwTrmlBh*-)Q{HZYc-w!uz-Fe5miQx*GWkqlf~AOHfUT z-H_q?3ZBAW76zF?_KKQN7`C^#e@Ooj{g8&EILPA%CVgDBo39h@ekR@0^C;fCM{q@@ z@!8UI($SdB{Tw_mp0M>9{Z0SPbDk|a?AKo%9oJXrHb1ISx1j#2d}&+(-fS0*$3nXP zxNEZKXqC(Px%0(y)hy?oQA^`W z<1B}d#MpLkF^aerQQ&?pMrZ^1XO53|u0~4=Cqf7OR$bd0+qHXGS0XD#zlp;~kM{Y? z!$$!(!}ue?-kyioD_j~vE`6<-{07_IxT%=mwxV{FL*--fLY>J77afh!n7!l2iBn(U zQQC>xnc5^RmAy1%gEly?QhP;vPy0fP)}I#a0q@-PmvsSob9KWtRGppUtFM;YEnn0N zT&EvBve4v)>4vSi`h_m%2fIDR=TO6GajNo?XIVhmjIw!U+A`|F*=M`UPLwHbHJq-j zf4rGfnH*Hd(%!!qW?i{i{gv8cnfJ4!Ev_<7)hAS3Wn3jSeuLMLs=r(haSw4G|^(CeaW|G^#;+Q>d%_vcIWmq%^g?dn!;5Nl^etonQf0}c4p zX8BQ{A-ogIk12&{`H9(%O9rP7%8$-*VVn`CqSgZ9zy02Q`!Yit@Hji=*}-4__`YcF z{ZD*nS*PUus7`y$8Rpr>N{eSyA?1a*}46trP0_>^1W-eI-O{Wmd^4o3r7y2%*td_4)0u zMR?$7j%-f%#~v2n4{VZBp8D=$TgAYD!za*~$>*Iy6|su%{jUA+ctWuPnSq!1+aDfF z0~OuWDT@A8ua=+qvmiS=D(Jl4!_dElUbk?5h5YiW2Ln=aRtIFS3Pm@QzeTaL1!>2@ zZDxPW&&nSdS!dC(*@9%-8xbcNDaX;A-_XwGgEET9O+0D*T>h5cs)(pdg8lr{1Lmx} z*(=={_D`m(*uVy7{BTwCjk`XulEcDXWipb(+7m45D)3~4lPRzxgRiKHV{~Zb{gcZm z_%o`&{0fUhlRu-%-Zj*#AOXr`R{mUYOBzge zOCk%5sh-B25_1WAE%=?N&cMl=J*65>S(bJ`ytoJ7%;I^&&#Ns4$Olp(!?Dh)H)}ZM z59v?Gvte>&C)l*qXZAnJ&S&o{W*=u)zJ`MkAA2f<^O{ZaFbs1+@8V#M1U!%-12L>e zBDh0@Wamg~(?_EU$ee&;4f-H-H?-UUe*cR|_MA7_uw^%3EPw!WX8~nx#-u1EW;$Y0 z77NM>MTQ07`4R(d92C>qv|0brEDkmvqt8GPK+?5g&u~!g4+EFx4<2HAlL*tP)w2l(bzy?8$|pO zA_-qA7Gdf?5~)P}BU|<0zewT5m`%vE`Uh%NCYRy zK+@67_d;jTqgM2vgswwweL)BC(?BAiG9vYhDi*ziL2rE9G-jLyYU7|cjx{1eei%e3 z1Mw4~c92-Ys0!3E)GXTH4>f0F`T3H6g_|(%e?!-D#$~lI5V~ z5z4)##B~568r=lhbW@4@r<+1zOpuZrN(^zJ;LP_P=lGvWoC%Q8osf&ymG}riAW*5J zoEz7a_$5HRIw4h8l~}C#@U5mYB~Ac{XD1F5AVmP_D#!K@CB6@kQJpxBzbmm_hAFVe zM4|g{N<0oAu#oAf-TPN1o(&L(PDt1<_%T4bT5;oo688nOHUt|$$>am# z41Z;SFp*$hgqmU0!HIqtXi{w~jJISI4L1UtfWaWGT5@ssa73;N78x-iGYitD8x?3; zs~)B^hz*83Kd3uI{Ff|zK_vdPO>*>Cj(qVu7YAEywN9>>lyI-dt6t7 z3v)P_)d?=xc8O6p6M+EuzJuKInsTs_ccHrxbcnwJ%8f<)o7&T?huJP2MWeJg`gil7 z5@>)1bx*bg^bcQmC$Bqq)zRaU>a#PEP0GfdSCthaLBuaz)3ussPjY=2vr*Nn3M?Y?a^8z;NO#umn~xBm)PKh0{#cz%Nrk zh(O_Jpc(=4NNm$55#`n)H!|o^_>q(-K-y@)NkSS>vuO+z10_Jyp@ff+Dl||Na3&FX z#vZIT!EWQz|vxG2U9vlM@l_x;6q6zbXG0-GTYB9CoBb&r|R0iZ^z;q5r7+Nx6X@~?9 z-n|!=&cflYHB$3`EBs?PDIEIdpBvN~3)u;Q=JXCDIADRO0c*%jc@=T6R)VQF76Eeu z5JYT!@S`CPbQpRvVzeT^Dg2D{?ETSfw;ZzaE7ik7i zYU;mtaM6WHXBveF`WMIz00n9=3NetI83(-!qBaL;G>M3@gS~(*vTnEy{r83>Nz)*_ z08ri!q%^V=%>QKVoa7hl)?L?})(X={=#^R-ze9tWtuE zXvz42;nS=Kp2itxD99E^hAfzwfn>r-SP93#KmqvzLwgF_6MYKW4<3lT>z40t?|tliHsGs4?(+ zJdq5tK85R}NK+tjBfv<1nG`Je+dx*lW(d(Cqn#hyA@wj?;SDl57(J}WLfSYDfIR@< zMiF^0mui9)YS4qegUuM3wvxLe50fT0weFwa0G2_>7g6>sP|qC`^uI@P1%g5L04Wd} z27@iuI2KtMZF*=6S=LeJ-~<34_-g!+Z9aL(hxBJ4Y}ZFFfMp?k_c1~i1H=~~b^!4+ zLJk3B#z&B{unJZg~XcZNdz1hfOOGZ1vpZG4CySVWY$ca0FWL4F}14->dFL&uoI`sCkk%` zNZ(F~4%IUV` zLms{boS^{e>Wg9AKs*{CUFFOKNXSPJ+r&vY0U%wqS3${n0O{XZyW<`R0R9M40=@Gy zK)NJ>dD0&@0>q{BttRLl(Nc(n?S#ZYUEu&BTLDwMVxZ(KfOJX1alrW;AY^l3!YKiq zQvm7P38@0eO@MUuvg6)RTnCWeoj5T-kA)Uu*t;Nr;|7o}oQr@H2#~>@I3;uBI0GOg z%HJf<7Xgy<5u^$rpM3;j#>?@xk02#b*B*d$@sK%h621VCE(s3<$P<7#chwG%9@!t_ zIG*yyt^ny81!rKyMF6C0RNeqc3P8Fvi6e}Z6#(g~{RQ9@1Efn5hW!P?9Etw1F%*c<|W3V+#*yg*`ky;UUzL-Xi@>dcPk$q)h|h;lm?PKn+iGcw@;(b7axJ>VG$583vPf@d^5W8fJJ z&p3F-!!rRMDLgbhGI-?hc){Zh5BaDDR=2<$23{+$_)tL{2238{=uGDHz8{9+lQwo{ zW&qPT<8LxE?4cnuLr4CBTmy(h^rygp*D$pYr`Ew}&H=+U7}8CN8BEVae!m=%$kUe1$6#Hy6S)SLqqX5yj3_42n&Mm z%KzEw{trs0;T>-u6@CU367eum_>tNFqd$l!|G(#>57g@DtHOb+KRjIk;Qu!tLg6N$ zaN0){F1iS7E0DVH8}*aHT?)a1-HHW!Cp#Dv;IbzwH<0{GSyS0Z=}CPJey$!Mm)aNv$om;D8C^u&LAwXP<&i zD0ny`7?Xi6cNF*rq)%b+x^Zuw;c{Nt*&Ay0hYa@>tt}BGEPUAA+X|f7sNd2E+tej6 zdcE%{FxLtdx`INxwTV5SkKVy~(~hSK{8waoAe>``BgZ%R4fsyxMuE0s*gkl-L343<5v4Sp-7?@ H|Niq|w~)S& diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testWAV.wav b/solr/contrib/morphlines-core/src/test-files/test-documents/testWAV.wav deleted file mode 100644 index 59a063ece0114867f9447c30c3c3476a3f185a9b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3884 zcmZ`*eQ1|e6n^eIXHzVf<-(*`ZmF#jvY;SJGq6T6_sL+R>tyZC-gki0aM3&CartbGUJ!fafd*5H{h4XvQJ@+}!dCs}_Wt;uL zj2X2ffk&r4a^F*nTB<7n1Z0^02~fEy!w^*B$>x@35ECGxMo2%(J&_DWny(FsesshJ zVvNCJ`QMZyUq{)LU9+9pR9M`rsbz3q$WO+5S4PXX*`#ywYR+4V^|Mf}sKG0n9>u6C zJ0Y#*YDMDlUc#L?JcoVGq$d%8GxWT$`Ri!zD>+)W_ z3saRwUb8GS z?b%}5k?s4pVLH>>ns(?~J?4tEVr0~n@}%de88$UVD#E9lpJkkrVbTg7ajmv4t?W^I zm(D7`>nlT9cdFW-(tbT2-*Kk-O0GMN#wtF6e7CxVnAWV!%A&&T#J)S_A$2S@Ub+s_3gV(tn&aoB6)F zSd_&@ph!%CR>S=Eg2>zCtK>bZBKHN_23jYzEa1DE9LqF^xlNo*8~us`(8b)J+1p0) zH1NHZ{9~!%558A1?^kk8VeUgA;4WgmEoL}e$nfHYfXRIUWm$&GETHi`=XWmPt)j>* zazEP#Ol96)a`dv!Y~~LqZu{XWPxS)Rhj13OZgSko zy!Et(e}Hkl0hO#V>1e>Azkvri;O5~XchH}EL}W*=$U@dy$irL8z1zw&tf1b}<-q6E zaqCHu4)Xj^7VsMNe{>uee~f3=Bl00@%^|+xEYRL9@;dP~!&sm5xuqhY^-5qR^SVz5 zEIk5zyjx`bK9Ln;fVEfg{_G5B*%mPUB+yCDm0iFT*4@Vo&~-e+hRZUXxjbP1A#i>o zP1iAhH4mp1*!3Q;u12K2Po(^yNO82tnvnsovd?YouZpv&-OpOb_#K=Ud0{oMb4|c+ z*9P1>9@w)L`1AvyV~a@RPr%J1fCZC9num!zrhSYPSBCwppuFB{>%QsD*H%5!T84_R4EGMtQE^M^GQ9k~VJ*+JsGc{+@~-hb z)pt{jay9UK5Z=`^=TL|3b@C1w-n4AmU^v6NpW(ol?EQArG39r!;kg#~IV#_rdXG7W z)wA9*?C3HMty4EsXQgYy_}j3auQPGq-hYm<8UJ-K#;IICm5rq?KC*0^HzSN}ipT4`->%#;d;`i40!DchX+H$G8{sJobN_HkXJ`&)aH52E)2auWA{3i+o@6 myo;igUEI+so*gHgk3=O}H?0x<6r+{AuEjNCkM|Z=rt)8!%e+Sb diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testWORD_various.doc b/solr/contrib/morphlines-core/src/test-files/test-documents/testWORD_various.doc deleted file mode 100644 index a2ad2364565a197d52d678ce221ad941224bbc5c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 35328 zcmeHw30#!L|Nrc=z;YuZBBHK{iU&6WlA;JGs0e~*o(nAO>cZl(D0n2^rj?jxT4aP; zW~r1{d7F1$XsMW%c$IlYWgdC(dw-s1AJ`>GwD0fx|9$`8oqfHZnfc7jXJ$S#&)oBD z>4^sAAAII^iPccy%#1y)v|<(pbP>#51-m0--C-u$)5^+9B6k5`G^zg}3EWz>hneMD zNf>+Jw2KOnF_bKnl(D6*jM=l~G09^#4%#?~qZ&P|Bra^9J7Z_Mh&UvR8EXt5kw(54 zYoxbVRytI<)MXYtY~c*^tav-=$6Oe*hV?9TTj+rlRSi9eCvi_I1Tv)S{Tb>?&`*Hw z4E+l9jt2e)doVT~IdMSly+99vPO2`wqzUg{Q@`AjF|c7*p_8hwe$JDzVg#Jjma)(E z;k-Ex#t$L?LcGSVjJ<($AuCL9uc-^^5yD-8TuBMKkYAq4=YKF>_Xc1}UR~XImzp}o zL*+nxLZJ(Mg!Bq_l8>Ph_!!daV;1sN7eC@V2#hAm_!g@v%3EKg^LIG5kQA?kdUsVXm#s3s{tE8L!H4Q$TgcT9I@Lc?2V{)pLieUh2qnme?38{X ze$Me<@DSn=SPA7t@f3h4$XU>Z^ayrA7sA)23;t^ALVXnUy21-|A%4MLmtHd-!7j*I zu-Byve1-C@sXxoTrvHDHT_~@*(p?u_NbmFNb@8LN*$NHvA~f@1sP*Q|2~X-#OzQu> z1f)GR8eOJFr;y3BvJ`S{woIweD#UhnJv8c6nMx;Dt5Re+nW+k`T9v7gW#?omwAm?I zRhACsWZq(IdVRk)o)M|xJ<}Vlm1Vf;K*UX%ZWQ0rAhv~;zWywNN zV|##|u|3evPS!_}H$kIK^%o2Qc6JSAF+&rg6Z^*Y>m&0}>U3FQzP>uuShh03{O3rB|7U3?XY{8-Za`PVyKOs$!S1G^9ccAI4n$xir{e44Me{32^rZQeajiO&OeY$dBL}@VBmi4IA*cVae4N z5QA%Nd7v~?d1P`=1@nPlJIv+iDfx?e05{GTNim`W8UwCC6QCK;9Pk8M13rK+5C*&i zL;=x2Phcys4cHDG1C9eHfKuQt@CR@YutHB^4cG#9fIH9(Xbva=6)*wF1!e&>-2d&) z)$$A9UO0MT--YcLHeM(%`E=Et5AH0x^Tx7S%gR@k-sPiq3Yjt>vGrsF8j{Aa)WtdGgQd+e50o>^s(_UP|E;6!*w>P#2 zOQ#vRcaly6f8s>(WIBJxSVJLI6suqFCX`aRQHq$G;1=t0fan=9Op*!YNJVXFo+p&K zP@aQ88wDtValky_9bgS0l(kUCHYg`5PoW&!!yF9s0G=3@O5K(RhTe=U6^1T#Ta3b7 zHw;s^rS9bXW0z9H6x3}o^z}Dp%vPLIE3nSuyb^Dzrzkc}1mEeYRD;zA98sN(l(Z?c zZo_PPn!yWIJi#v3+tCOJCgS5lH64{S>)1sgR!b7&05D;k)+`nKZLHNnCh!L z--G8y!J@|Z%qz<3}RC;~nQjOvAkvLdJ#v5XP|lU`2-hCw(p7L=;r4u4M;;>kiiS&S#@ z_~mA0-0~z>{=FbgQ~5Jd&GNTqC|6g2%H0#FS@z38qw+5Ss1AGu>;o!*-+;ToJs=nj zQ5dip_yX7pTn2swt^>D#JHT(iUEl%mC-4;Dmh)V(3J3TA0YDI-1k}Krz!HGgBY(O1 z-OV$HZtlCe?dF#CTh@QLeoOK4w_ksC))f6OZ$e&n-YojPZBSXz6HX7=!w87`AyZY)U0aWek;5M;lhKREILC251)Zp@feSaTFz8;Z;;_#7Sl=U)iF{7Y+s_Pq zBKB`vVHo4)qNIu&+1r`L_ZGAMz(_z2d(}9mrHfM6VJO@sECJ{bj07@c8b~x=A{{hv z<3>I$8YyYSX5fIAtD$G=- zHvlTr3V_P=9zbRK7@#usLs<#sm^ulm5CXxQ)dv)nQCW@M=+10!!3uT?9fv|X;I z*GToh)}`Dq9wUoUaxlcJTZtN=g+mjFcDuS2Oi$$oeIj&|Twl*sQO?9wQi(g-My?W4 zZ`6e5ZPl|Netsw8W_fbV_-Kwsvp#-K$Im6~{xa_nLJ4z^+C$Yo=M#9#STZ_+TyBO! za}|^N_eg-=O=@Sy076^47-pfpHL8q{4Ab-?mV1T_Oy+;L1gcvBdiSaIuHAj)bc-Cm4IHG_Y*82^8Yk0wS@|1?9Thi~LP+K$LzFBQk zKw_&t!DGJBfL^qOv6kbRnNrm0j5!mT<=R3QiFsXY2E7$rb8W*0*)W>?{o{!lMMG@) ziPZn8#E^(g$|M1k1WXbzNx&ollLSl>FiF590h0tw5->@?BmqMSjFB{BPS{Jn1(rVp!MGssER4r`z_U;nk8p&o%k6)Iz8d)z_~HO_m06(G)W4{Q zpPFHmSUQUVYSRB3?IlH?lP zj|84_n2E0;Pk1*_Q7mHgZbI|+h#m-KMPyzddaazrRL5o8mHNvai=t@4p>wjx!_J(h#9s+ zagK4U>~7^Ax>2G}nV*FvmvG&2?tqkuW1I>+MdIoWK=27*PkoD-NEEF=I~-B-EKu_-kOvEFlRHADl=lb`YMEI>J{Rq z>xlY?LpWIKR-gpv)Tpq}TAfcj)}y@x&n7=VAH4-@hO*I$Vjw41#vSpqpmVUsXFZK2 z)!0B%Qtm(j9U_R~q%(~;nKjmPa{3!ihVrrHxKOf)Qxt0hxhxd%?;4+`+BhX5pJ|+= z=;Scv8t*=iK#GVFc6{F%y@9O|<#IKj4q}#=SsT4;y&z*1WQ3bUENqY)dx67L;O%B0 z4Zgldu?>*v=ahc5Ms!wACbqA77i()J=+$k?9A=h^M*Ay6E=-KMx$KoRvD|%IZ(< zKDyIx+u8R131&0eetOqx(a5uB2VcA6?ALDDhXa0I-nonGBK3k+D=%1j=BcK*PrK&! zQp4S^bf5F%X0JQ1{jo2m@Yj8%ISddQLQ@j(NDV}qF}gEQ#(ql zP-px4&~xU36A+Q@oL^?`m~nDS|5JNiTaWi$RS@aBVhX$DHC5@+^nBBs=i?7rSZ|Zg zT=~oWkGnoyH~-15nwRPN~RH_!>Op_oEIRf6`Loxq%m_Jt(fRbdee8yIEVQ(nr#R*_cT2_OjvqB${auXv{*LNP z2OItPbFS;*jjuPn7<(wY|CCmz3udj|^u6rNrVgiD{?hULkzaa_={mpH;OpPqZ+AKR zW`{P`;?UX?zhu+e%S$jAOK6Idc(oj%`TF9!`9G0#7)%i?m^LqE7J_&B~aVbR@S?d7f(>tB6y;_me1qS%J% zv-Vh6=Dc2zbWR~|mVD-3e*EqQv#;*_&2ewyo~ZMmEPH>;#&J1T>(l2ieYs@w#&w!k z&TUmkWR<9M=VcA>TYJCh+783ar@yyD=%*=k_!$*dOv!#p}n@ zwrsmmZg=SN;GZgwBR%ekIjXKX9N_N;31($ynhwmfDNJF0ZR zrX3DRwoUw3+`sYg{`%$Jq7OPOZ`SR>&kru`y<*wzVl*W<@|og7c4w>duF_x;EBZXKS#KK#j(ZM_;*#7*m^cu;U4 zs`%Zj(%ZkgItLCK*Sc-1aXTCBm45Tvyknk=L-)MX$Nb6idG2?QwM$zz!M$_XzD?!L zMAueW=T6a{Xprmp>Z`?#3MUkoiSB5RhYcU_vb@*t?U&_+#(p*C$K=EVw^Jq$*}rXm z!-Kna#9HSBI`rxFYi5+Y|9#hf&i1iAoKIyx&M&aOeJAloS$a`%c>Y%_`ehmJRyZ_UU-_hDnw=p`VBl8(N}qlg_auDQ{`;LrKKyLH^StH* zPHvyp*|*h}2N5A(jeB`kvr^CYDcuSz9gAIWj~i%y@ml8W>k}%>`nhab4D++`Ya2%VD{1-fAHI{o(6BjW|4Iuy#k~wQyB;Ef{k9x83&ZuU~pRJfXmU zTb$DV{mv^I#g%`0bkMrgsEFHV<%7=lcyHOT@8_oVesAyEgeH?i*Sp!eoSA7Mi8wDF zG;-GOU#8hvHar?rbUX9@4;Q=-*fjpxqsgWG)bLXiPdr+kS-AM%yf3Zt%Ra7H7=HD= zyrhh{u0<|A&2j@+2p=ul)U9RM_V$3ky3&^vYO&dU)oRGW~z;fF^5(EIYV%(p&odhPi7d9e>|5?8q&s0>KsgzPCPZG%3p7s_6~4FV@^2?-1>HytO0P^6AgV54H*Px^_nXP3hHpzEPi+4ZJ;ZLzjz_j?BCM z>d<#jAKD;Z`qt*@gO=P|-|nlF&o?u=6!;X3&yu{psM*FnD`SU#leTuz=9hO49hLRb zu!^p7wfDK9kEirYX?$$ik>iK=&kq`(CO>#)y!wNW_iSjJ_yy}!8lzF+G&zda$Xr+x5;`_eA?SGTwyJN5c5^izKwZ`hqIcz-ionxd1&9ety2 zm>+v^U9v%EtRIWSS%2l{JoEna2~v&bo2`xu+k8K_bnUlhGp}~C3-0FG|JH50bseQX zbL2ahskXiS-rGee$JGlU*8o?-tW01*>Tco z)?`V-JCz%rgu1<&?%Tp;#MAx}+2>}NdlerzTHfm2HGL&t58JwR-^Hq|i^SfzHCNZT z6y6L}Eu++gKE@1cj@GI}cYr(%#%?tO)*Ss*2Xw|5*4KtgI7qEm{0BwI;)z(~0L5m{kH$HU_AH!6=6drCn5CSVXtbJ1#Z5pR;U>Ja+(n>v zgwW$305|<8I_}yeU6r7CxC#XtUGt)=K6Kolj{K6lrQSQGh2~APE_DL<1F(V!e*&J8 zDE0YlIAn34`(<6`^oKEEQnI#E1jlI8YD*<+nR){;MN&UUHR;Rx|F?N{Is# zSJ=asBa>E8nz_Iz4hLpd@hR*MOj6+mUBs*^h(S#yX4YIdEM9TD7UZ8+(28w&I#;yF zY#_6#Z~{Fa5Mky-^j(Nej61^~zcvSTml+)McUu-9HG3=%fwGSz!}fDeq_6@(^r``8 zrp!THdb)^9YY$MI^l1`N0Mv>k#LdOPIUlshoDXV=pq6}44JYHqbq39n@WS8rjiJ7JDO&-1sLH;HAb%io+Aox%s96Klf;^}pT zqPC%L<;a=THbCd%66-^;I^!~K7WhzR`cQSJ(6~k5LrI>QLTU^sh13`bDKvj(3XR(V zJ}!%ArqH+v;6qtHGllf_Qwr(r7g8uiC>kD6KLV{vQJ+&~fr0uGLpu%HsugkI6;0Bp z|HWuQVg{H4b(!nys{wd40h$61b(pDlrv%do<{ChXQbLz5>3(%Tl?O}e(w<$!7f5LL zhc3v_rzq(9CSCWW8+o+nvC8soMi(||7vklVLT+D%_7dpQDqR$%i>b569$|DfbfWz^ zMxT73&r#4PAn4;0w3pg^^kXjmWV{Zk^h!Wcy{_yorhycx*|5)s6OCH4XzbOU)43vm zod$9ksZk(HHjvxYqR)s-fQhJ-NkQ?x_;H=E4d62JKz->E>9Z;cdNNyO$-Gs!M7D|+yeNgZA&I|d4mLf$UzKQpeQ7qEaV^< zl!S;2Nn6~M8h2>KY0ireH9$`GH4vFFN-<81J#&OA~; z^ue4Q)3L=z@=QlPU@U^KI}&pw*g~Jvn=)>CL_wHgPN~c(cKU=+9G-L?l|FqaVIt{V zS5{Gq+Rk?&^BX+;kT3)gGuqo923W#f%)KfYHapVmT$2hC)MpaLO+|QF@if!3z7i-h zFEZcAHnM5cocxT#(6oi{Pn1FMfEhS_fN3RrOdCIcU@0u2G3Vk-pm_jLY;@Tu?k>v2LhMzcaqt<{v-$h4Pvhk~*FCk@yI&~1V zs=#-=ws>c3@#fI_g-QK~O5iCvZFDDpNxcY^@>%cy_Ap-Q_{l;k>(F+?IeKq5!hxzS zxi1AMWD!7Bg4S)Q`P>JfezS6bstb)vsMnSv5Ox0U(CO8nGiAgp3OWrKQlZmAR2Foq zRFk38qSRdISWaP!p=05R6+(A}z7jf}jrbHgrQ<3h7e-{Ta%S>Z6T9J*Ohr*NFU%o8}T20=R#(uPvs`g3M=rXFp*2R}(L$4|Ep;rn1w3=*9 znobsC|ZOs2W zK3o?Cnt2UcN6fvigHCf{7`dfanxkg{v<5I4pf!M*0KPZIUIXYr&0>J&`h@^Dmxs>H z<)PETos9s^{kH?e`!GOhsyWA}IWEnUNvHTpr^Q5m?rY1%0aBfMb3{(z2n=_vxJYP% zPKh3%(y0|ytXT<2JYU9SFjJUNfYQ|+AUl~Uy`vIhqejKh{}3c`G)i{_=(IE~ETMJ6 zqMX3bP$#*y0BDWic|Vs$;V2#a$_i&NJxmfXNx&ollLSl>FiF590h0tw5->@?Bmt8I zOcJP?v(o&N=Fv3wr1>egUk{!3>1l7C=IS)Br8zn6#nXR2)4ZML^t6Xhb9dS|$KPgg z^LKKm`72$(aRYGco4EsY=!&M-G*6WQEddXp72pZ92HF5^0WY8(&>rXj5KnLDbehc< z@B{n-+EWh%$h{NvU?2pbxp`+G40s73_a{I&%-!^MqV<3|Qg4U#ze>?TOZvWx4F8}< z-@nn~kkRuf7iKOJ1V(kp!irWv7~9P~@nI1ICHymP1Zdr66zLKKj z9qaLQL;mUgr1!SI`6ssYzulzFxr1jJ+>uWz`&h)D#-)i4feQb`$osAB>GcBg??{ZO z@qjW%4KU6*9fYfYdO{IDRi)>Ve-K5ik3S9v;xtvYFbO%P19l4XQ7`3&(ih^UHnTRh z+kZLtkI+K>H}tayUjsUg^Wpv3Nwh(MtP}^Y{n$_(gzm$>W_v+RVn48N*mrR0ieF6? zfUi6W@k?tu>g|6Qe=KsWfV^K+3GW~E6VFrs=*}*w1V~$>|B@2bHaepCJjw;{pOF9h z#!qP-NG~mv2vVm5&si*;H5@{iCctznFifmbuLXAH^R-`8B^3=FhK0G2J!e>ZQUk@2a=qJwr z5i}3|?egUM+%DH}2OFa_DLEOqD9&FoPgW(XRho3IJWH9UXZ9~)6<(DWG7XTYhxgIS zyZG|PDmVImR(KDMW-PhzS?5Ub&?`rk8a^q&KeTgzUr?}jbYN(NcThxdsCQ^*e}C`L z5P$#Zpb-D)!04bU6qxglV0|NFqr&}&!kIvc$k8b^T1rK=iR9N8e~wMm1l;Y{WRj_` z926uoL!spukd+LA4;ZTh!Xof9Fd~1nO%)!4|Ekt$WI5Ri*#P|GHF4n`5KW>YEnbWN z@DBGUa+QrNya>2R^%a4J$Msgv{jq{f4galz)aF?b+G%(0JA|fI%vQt3M9)Zz4rU(@=HTy-YPf8A=&lr-S z_6YBS==rZCKswpRR!UlqR`sH>7}a-QNV-N;WH%DU^Avyxvl@IbkL!DCj=gCPRqGjJ vS?3UMeC;7nCcK#36fZ7`&L4xsjg83cBe?s7vb&u>A6M^MVe - - - - Tika test document - - Rida Benjelloun - - Java - - XML - - XSLT - - JDOM - - Indexation - - Framework d'indexation des documents XML, HTML, PDF etc.. - - http://www.apache.org - - 2000-12-01T00:00:00.000Z - - test - - application/msword - - Fr - - Archimède et Lius à Châteauneuf testing chars en été - - \ No newline at end of file diff --git a/solr/contrib/morphlines-core/src/test-files/test-documents/testXML2.xml b/solr/contrib/morphlines-core/src/test-files/test-documents/testXML2.xml deleted file mode 100644 index 6611ee14957..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-documents/testXML2.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - 123 - Hello World - Solr rocks - diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/loadSolrBasic.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/loadSolrBasic.conf deleted file mode 100644 index 1c02a9ae181..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/loadSolrBasic.conf +++ /dev/null @@ -1,68 +0,0 @@ -# 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. - - -# morphline.conf example file -# this is a comment -// this is yet another comment - -# for details see https://github.com/typesafehub/config#optional-system-or-env-variable-overrides -SOLR_COLLECTION : "collection1" -SOLR_COLLECTION : ${?ENV_SOLR_COLLECTION} - -ZK_HOST : "127.0.0.1:2181/solr" -ZK_HOST : ${?ENV_ZK_HOST} - -SOLR_HOME_DIR : "example/solr/collection1" -SOLR_HOME_DIR : ${?ENV_SOLR_HOME_DIR} - -SOLR_LOCATOR : { - collection : ${SOLR_COLLECTION} - zkHost : ${ZK_HOST} - solrHomeDir : ${SOLR_HOME_DIR} - # batchSize : 1000 -} -SOLR_LOCATOR : ${?ENV_SOLR_LOCATOR} - -morphlines : [ - { - id : morphline1 - - # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations - # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876) - importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder", - "org.apache.solr.morphlines.solr.SanitizeUnknownSolrFieldsBuilder", - "org.apache.solr.morphlines.solr.LoadSolrBuilder"] - - commands : [ - { - sanitizeUnknownSolrFields { - solrLocator : ${SOLR_LOCATOR} - } - } - - { - loadSolr { - solrLocator : ${SOLR_LOCATOR} - boosts : { - id : 1.0 - } - } - } - - { logDebug { format : "output record: {}", args : ["@{}"] } } - ] - } -] diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellDocumentTypes.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellDocumentTypes.conf deleted file mode 100644 index 4d382563d4a..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellDocumentTypes.conf +++ /dev/null @@ -1,304 +0,0 @@ -# Licensed 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. - -# Application configuration file in HOCON format (Human-Optimized Config Object Notation). -# HOCON syntax is defined at http://github.com/typesafehub/config/blob/master/HOCON.md -# and also used by Akka (http://www.akka.io) and Play (http://www.playframework.org/). -# For more examples see http://doc.akka.io/docs/akka/2.1.2/general/configuration.html - -# morphline.conf example file -# this is a comment -// this is yet another comment - -morphlines : [ - { - id : morphline1 - - # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations - # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876) - importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder", - "org.kitesdk.morphline.stdlib.SetValuesBuilder", - "org.kitesdk.morphline.stdlib.CallParentPipeBuilder", - "org.kitesdk.morphline.stdlib.GenerateUUIDBuilder", - "org.kitesdk.morphline.stdlib.JavaBuilder", - "org.kitesdk.morphline.stdlib.TryRulesBuilder", - "org.kitesdk.morphline.stdlib.SeparateAttachmentsBuilder", - "org.kitesdk.morphline.stdio.ReadCSVBuilder", - "org.kitesdk.morphline.avro.ReadAvroContainerBuilder", - "org.kitesdk.morphline.avro.ExtractAvroPathsBuilder", - "org.kitesdk.morphline.avro.ExtractAvroTreeBuilder", - "org.kitesdk.morphline.tika.DetectMimeTypeBuilder", - "org.kitesdk.morphline.tika.decompress.DecompressBuilder", - "org.kitesdk.morphline.tika.decompress.UnpackBuilder", - "org.kitesdk.morphline.twitter.ReadJsonTestTweetsBuilder", - "org.apache.solr.morphlines.cell.SolrCellBuilder", - "org.apache.solr.morphlines.solr.SanitizeUnknownSolrFieldsBuilder", - "org.apache.solr.morphlines.solr.GenerateSolrSequenceKeyBuilder", - "org.apache.solr.morphlines.solr.LoadSolrBuilder"] - - commands : [ - { separateAttachments {} } - - # java command that doesn't do anything except for test compilation - { - java { - imports : "import java.util.*;" - code: """ - List tags = record.get("javaWithImports"); - return child.process(record); - """ - } - } - - # java command that doesn't do anything except for test compilation - { - java { - code: """ - List tags = record.get("javaWithoutImports"); - return child.process(record); - """ - } - } - - { - # used for auto-detection if MIME type isn't explicitly supplied - detectMimeType { - includeDefaultMimeTypes : true - mimeTypesFiles : ["RESOURCES_DIR/custom-mimetypes.xml"] - } - } - - { - tryRules { - throwExceptionIfAllRulesFailed : true - rules : [ - # next top-level rule: - { - commands : [ - { logDebug { format : "hello unpack" } } - { unpack {} } - { generateUUID {} } - { callParentPipe {} } - ] - } - - { - commands : [ - { logDebug { format : "hello decompress" } } - { decompress {} } - { callParentPipe {} } - ] - } - - { - commands : [ - { - readCSV { - supportedMimeTypes : [text/csv] - charset : UTF-8 - ignoreFirstLine : false - columns : [ user_screen_name, text ] - } - } - - { - generateUUID { - field : id - preserveExisting : false - } - } - - { - sanitizeUnknownSolrFields { - solrLocator : ${SOLR_LOCATOR} - } - } - ] - } - - { - commands : [ - { - readAvroContainer { - supportedMimeTypes : [avro/binary] - # readerSchemaString : "" # optional, avro json schema blurb for getSchema() - # readerSchemaFile : /path/to/syslog.avsc - } - } - - { extractAvroTree {} } - - { - setValues { - id : "@{/id}" - user_screen_name : "@{/user_screen_name}" - text : "@{/text}" - } - } - - { - sanitizeUnknownSolrFields { - solrLocator : ${SOLR_LOCATOR} - } - } - ] - } - - { - commands : [ - { - readJsonTestTweets { - supportedMimeTypes : ["mytwittertest/json+delimited+length"] - } - } - - { - sanitizeUnknownSolrFields { - solrLocator : ${SOLR_LOCATOR} - } - } - ] - } - - # next top-level rule: - { - commands : [ - { logDebug { format : "hello solrcell" } } - { - # wrap SolrCell around an HTML Tika parser - solrCell { - solrLocator : ${SOLR_LOCATOR} - # captureAttr : true # default is false - capture : [ - - # twitter feed schema - user_friends_count - user_location - user_description - user_statuses_count - user_followers_count - user_name - user_screen_name - created_at - text - retweet_count - retweeted - in_reply_to_user_id - source - in_reply_to_status_id - media_url_https - expanded_url - - # file metadata - file_download_url - file_upload_url - file_scheme - file_host - file_port - file_path - file_name - file_length - file_last_modified - file_owner - file_group - file_permissions_user - file_permissions_group - file_permissions_other - file_permissions_stickybit - ] - - fmap : { content : text, content-type : content_type } # rename "content" field to "text" fields - dateFormats : [ "yyyy-MM-dd'T'HH:mm:ss", "yyyy-MM-dd"] # various java.text.SimpleDateFormat - # xpath : "/xhtml:html/xhtml:body/xhtml:div/descendant:node()" - uprefix : "ignored_" - lowernames : true - # solrContentHandlerFactory : org.apache.solr.tika.TrimSolrContentHandlerFactory - - # Tika parsers to be registered. If multiple parsers support the same MIME type, - # the parser is chosen that is closest to the bottom in this list: - parsers : [ - { parser : org.apache.tika.parser.asm.ClassParser } - # { parser : org.apache.tika.parser.AutoDetectParser } - # { parser : org.gagravarr.tika.OggParser, additionalSupportedMimeTypes : [audio/ogg] } - { parser : org.gagravarr.tika.FlacParser } - { parser : org.apache.tika.parser.audio.AudioParser } - { parser : org.apache.tika.parser.audio.MidiParser } - { parser : org.apache.tika.parser.crypto.Pkcs7Parser } - { parser : org.apache.tika.parser.dwg.DWGParser } - { parser : org.apache.tika.parser.epub.EpubParser } - { parser : org.apache.tika.parser.executable.ExecutableParser } - { parser : org.apache.tika.parser.feed.FeedParser } - { parser : org.apache.tika.parser.font.AdobeFontMetricParser } - { parser : org.apache.tika.parser.font.TrueTypeParser } - { parser : org.apache.tika.parser.xml.XMLParser } - { parser : org.apache.tika.parser.html.HtmlParser } - { parser : org.apache.tika.parser.image.ImageParser } - { parser : org.apache.tika.parser.image.PSDParser } - { parser : org.apache.tika.parser.image.TiffParser } - { parser : org.apache.tika.parser.iptc.IptcAnpaParser } - { parser : org.apache.tika.parser.iwork.IWorkPackageParser } - { parser : org.apache.tika.parser.jpeg.JpegParser } - { parser : org.apache.tika.parser.mail.RFC822Parser } - { parser : org.apache.tika.parser.mbox.MboxParser, additionalSupportedMimeTypes : [message/x-emlx] } - { parser : org.apache.tika.parser.microsoft.OfficeParser } - { parser : org.apache.tika.parser.microsoft.TNEFParser } - { parser : org.apache.tika.parser.microsoft.ooxml.OOXMLParser } - { parser : org.apache.tika.parser.mp3.Mp3Parser } - { parser : org.apache.tika.parser.mp4.MP4Parser } - { parser : org.apache.tika.parser.hdf.HDFParser } - { parser : org.apache.tika.parser.netcdf.NetCDFParser } - { parser : org.apache.tika.parser.odf.OpenDocumentParser } - { parser : org.apache.tika.parser.pdf.PDFParser } - { parser : org.apache.tika.parser.pkg.CompressorParser } - { parser : org.apache.tika.parser.pkg.PackageParser } - { parser : org.apache.tika.parser.rtf.RTFParser } - { parser : org.apache.tika.parser.txt.TXTParser } - { parser : org.apache.tika.parser.video.FLVParser } - { parser : org.apache.tika.parser.xml.DcXMLParser } - { parser : org.apache.tika.parser.xml.FictionBookParser } - { parser : org.apache.tika.parser.chm.ChmParser } - #{ parser : org.apache.tika.parser.AutoDetectParser } - ] - } - } - - { generateUUID { field : ignored_base_id } } - - { - generateSolrSequenceKey { - baseIdField: ignored_base_id - solrLocator : ${SOLR_LOCATOR} - } - } - - ] - } - ] - } - } - - { - loadSolr { - solrLocator : ${SOLR_LOCATOR} - } - } - - { - logDebug { - format : "My output record: {}" - args : ["@{}"] - } - } - - ] - } -] diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellJPGCompressed.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellJPGCompressed.conf deleted file mode 100644 index 85cb2a7f7fa..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellJPGCompressed.conf +++ /dev/null @@ -1,150 +0,0 @@ -# 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. - -# Application configuration file in HOCON format (Human-Optimized Config Object Notation). -# HOCON syntax is defined at http://github.com/typesafehub/config/blob/master/HOCON.md -# and also used by Akka (http://www.akka.io) and Play (http://www.playframework.org/). -# For more examples see http://doc.akka.io/docs/akka/2.1.2/general/configuration.html - -# morphline.conf example file -# this is a comment -// this is yet another comment - -morphlines : [ - { - id : morphline1 - - # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations - # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876) - importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder", - "org.kitesdk.morphline.stdlib.CallParentPipeBuilder", - "org.kitesdk.morphline.stdlib.GenerateUUIDBuilder", - "org.kitesdk.morphline.stdlib.JavaBuilder", - "org.kitesdk.morphline.stdlib.TryRulesBuilder", - "org.kitesdk.morphline.stdlib.SeparateAttachmentsBuilder", - "org.kitesdk.morphline.tika.DetectMimeTypeBuilder", - "org.kitesdk.morphline.tika.decompress.DecompressBuilder", - "org.kitesdk.morphline.tika.decompress.UnpackBuilder", - "org.apache.solr.morphlines.cell.SolrCellBuilder", - "org.apache.solr.morphlines.solr.GenerateSolrSequenceKeyBuilder", - "org.apache.solr.morphlines.solr.LoadSolrBuilder"] - - - commands : [ - { separateAttachments {} } - - # java command that doesn't do anything except for test compilation - { - java { - imports : "import java.util.*;" - code: """ - List tags = record.get("javaWithImports"); - return child.process(record); - """ - } - } - - # java command that doesn't do anything except for test compilation - { - java { - code: """ - List tags = record.get("javaWithoutImports"); - return child.process(record); - """ - } - } - - { - # auto-detect MIME type if it isn't explicitly supplied - detectMimeType { - includeDefaultMimeTypes : true - } - } - - { - tryRules { - throwExceptionIfAllRulesFailed : true - rules : [ - # next top-level rule: - { - commands : [ - { logDebug { format : "hello unpack" } } - { unpack {} } - { callParentPipe {} } - ] - } - - { - commands : [ - { logDebug { format : "hello decompress" } } - { decompress {} } - { callParentPipe {} } - ] - } - - # next top-level rule: - { - commands : [ - { logDebug { format : "hello solrcell" } } - { - # wrap SolrCell around a JPG Tika parser - solrCell { - solrLocator : ${SOLR_LOCATOR} - captureAttr : true # default is false - capture : [content, a, h1, h2] # extract some fields - fmap : { exif_image_height : text, a : anchor, h1 : heading1 } # rename some fields - dateFormats : [ "yyyy-MM-dd'T'HH:mm:ss", "yyyy-MM-dd"] # various java.text.SimpleDateFormat - xpath : "/xhtml:html/xhtml:body/xhtml:div/descendant:node()" - uprefix : "ignored_" - lowernames : true - solrContentHandlerFactory : org.apache.solr.morphlines.cell.TrimSolrContentHandlerFactory - parsers : [ # nested Tika parsers - { parser : org.apache.tika.parser.jpeg.JpegParser } - ] - } - } - - { logDebug { format : "solrcell output: {}", args : ["@{}"] } } - ] - } - ] - } - } - - { generateUUID { field : ignored_base_id } } - - { - generateSolrSequenceKey { - baseIdField: ignored_base_id - solrLocator : ${SOLR_LOCATOR} - } - } - - { - loadSolr { - solrLocator : ${SOLR_LOCATOR} - } - } - - { - logDebug { - format : "My output record: {}" - args : ["@{}"] - } - } - - ] - } -] diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellXML.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellXML.conf deleted file mode 100644 index 9e840a9f59c..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/solrCellXML.conf +++ /dev/null @@ -1,78 +0,0 @@ -# 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. - -# Application configuration file in HOCON format (Human-Optimized Config Object Notation). -# HOCON syntax is defined at http://github.com/typesafehub/config/blob/master/HOCON.md -# and also used by Akka (http://www.akka.io) and Play (http://www.playframework.org/). -# For more examples see http://doc.akka.io/docs/akka/2.1.2/general/configuration.html - -# morphline.conf example file -# this is a comment -// this is yet another comment - -morphlines : [ - { - id : morphline1 - - # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations - # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876) - importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder", - "org.kitesdk.morphline.stdlib.AddValuesBuilder", - "org.apache.solr.morphlines.cell.SolrCellBuilder", - "org.apache.solr.morphlines.solr.GenerateSolrSequenceKeyBuilder", - "org.apache.solr.morphlines.solr.SanitizeUnknownSolrFieldsBuilder", - "org.apache.solr.morphlines.solr.LoadSolrBuilder"] - - - commands : [ - { - addValues { _attachment_mimetype : application/xml } - # alternatively, consider using detectMimeTypes command - } - - { - # wrap SolrCell around a JPG Tika parser - solrCell { - solrLocator : ${SOLR_LOCATOR} - parsers : [ # nested Tika parsers - { parser : org.apache.tika.parser.xml.XMLParser } - ] - } - } - - { - generateSolrSequenceKey { - baseIdField: base_id - solrLocator : ${SOLR_LOCATOR} - } - } - - { - sanitizeUnknownSolrFields { - solrLocator : ${SOLR_LOCATOR} - } - } - - { logDebug { format : "solrcell output: {}", args : ["@{}"] } } - - { - loadSolr { - solrLocator : ${SOLR_LOCATOR} - } - } - - ] - } -] diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/tokenizeText.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/tokenizeText.conf deleted file mode 100644 index d9354c420ca..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/tokenizeText.conf +++ /dev/null @@ -1,38 +0,0 @@ -# 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. - -morphlines : [ - { - id : morphline1 - - # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations - # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876) - importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder", - "org.apache.solr.morphlines.solr.TokenizeTextBuilder"] - - commands : [ - { - tokenizeText { - inputField : message - outputField : tokens - solrFieldType : text_en - solrLocator : ${SOLR_LOCATOR} - } - } - - { logDebug { format : "output record {}", args : ["@{}"] } } - ] - } -] diff --git a/solr/contrib/morphlines-core/src/test-files/test-morphlines/tutorialReadAvroContainer.conf b/solr/contrib/morphlines-core/src/test-files/test-morphlines/tutorialReadAvroContainer.conf deleted file mode 100644 index eee4ba52c03..00000000000 --- a/solr/contrib/morphlines-core/src/test-files/test-morphlines/tutorialReadAvroContainer.conf +++ /dev/null @@ -1,145 +0,0 @@ -# Licensed 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. - -# Application configuration file in HOCON format (Human-Optimized Config Object Notation). -# HOCON syntax is defined at http://github.com/typesafehub/config/blob/master/HOCON.md -# and also used by Akka (http://www.akka.io) and Play (http://www.playframework.org/). -# For more examples see http://doc.akka.io/docs/akka/2.1.2/general/configuration.html - -# morphline.conf example file -# this is a comment - -# Specify server locations in a SOLR_LOCATOR variable; used later in variable substitutions: -SOLR_LOCATOR : { - # Name of solr collection - collection : collection1 - - # ZooKeeper ensemble - zkHost : "127.0.0.1:2181/solr" - - # The maximum number of documents to send to Solr per network batch (throughput knob) - # batchSize : 1000 -} - -# Specify an array of one or more morphlines, each of which defines an ETL -# transformation chain. A morphline consists of one or more (potentially -# nested) commands. A morphline is a way to consume records (e.g. Flume events, -# HDFS files or blocks), turn them into a stream of records, and pipe the stream -# of records through a set of easily configurable transformations on its way to -# Solr. -morphlines : [ - { - # Name used to identify a morphline. E.g. used if there are multiple morphlines in a - # morphline config file - id : morphline1 - - # using globs (foo.bar.* or foo.bar.**) will not work in Java9 due to classpath scanning limitations - # so we enumarate every command (builder) we know this config uses below. (see SOLR-8876) - importCommands : ["org.kitesdk.morphline.stdlib.LogDebugBuilder", - "org.kitesdk.morphline.avro.ReadAvroContainerBuilder", - "org.kitesdk.morphline.avro.ExtractAvroPathsBuilder", - "org.kitesdk.morphline.stdlib.ConvertTimestampBuilder", - "org.apache.solr.morphlines.solr.SanitizeUnknownSolrFieldsBuilder", - "org.apache.solr.morphlines.solr.LoadSolrBuilder"] - - commands : [ - { - # Parse Avro container file and emit a record for each avro object - readAvroContainer { - # Optionally, require the input record to match one of these MIME types: - # supportedMimeTypes : [avro/binary] - - # Optionally, use a custom Avro schema in JSON format inline: - # readerSchemaString : """""" - - # Optionally, use a custom Avro schema file in JSON format: - # readerSchemaFile : /path/to/syslog.avsc - } - } - - { - # Consume the output record of the previous command and pipe another record downstream. - # - # extractAvroPaths is a command that uses zero or more avro path expressions to extract - # values from an Avro object. Each expression consists of a record output field name (on - # the left side of the colon ':') as well as zero or more path steps (on the right hand - # side), each path step separated by a '/' slash. Avro arrays are traversed with the '[]' - # notation. - # - # The result of a path expression is a list of objects, each of which is added to the - # given record output field. - # - # The path language supports all Avro concepts, including nested structures, records, - # arrays, maps, unions, etc, as well as a flatten option that collects the primitives in - # a subtree into a flat list. - extractAvroPaths { - flatten : false - paths : { - id : /id - text : /text - user_friends_count : /user_friends_count - user_location : /user_location - user_description : /user_description - user_statuses_count : /user_statuses_count - user_followers_count : /user_followers_count - user_name : /user_name - user_screen_name : /user_screen_name - created_at : /created_at - retweet_count : /retweet_count - retweeted : /retweeted - in_reply_to_user_id : /in_reply_to_user_id - source : /source - in_reply_to_status_id : /in_reply_to_status_id - media_url_https : /media_url_https - expanded_url : /expanded_url - } - } - } - - # Consume the output record of the previous command and pipe another record downstream. - # - # convert timestamp field to native Solr timestamp format - # e.g. 2012-09-06T07:14:34Z to 2012-09-06T07:14:34.000Z - { - convertTimestamp { - field : created_at - inputFormats : ["yyyy-MM-dd'T'HH:mm:ss'Z'", "yyyy-MM-dd"] - inputTimezone : UTC -# outputFormat : "yyyy-MM-dd'T'HH:mm:ss.SSSZ" - outputTimezone : America/Los_Angeles - } - } - - # Consume the output record of the previous command and pipe another record downstream. - # - # This command sanitizes record fields that are unknown to Solr schema.xml by deleting - # them. Recall that Solr throws an exception on any attempt to load a document that - # contains a field that isn't specified in schema.xml. - { - sanitizeUnknownSolrFields { - # Location from which to fetch Solr schema - solrLocator : ${SOLR_LOCATOR} - } - } - - # log the record at DEBUG level to SLF4J - { logDebug { format : "output record: {}", args : ["@{}"] } } - - # load the record into a Solr server or MapReduce Reducer. - { - loadSolr { - solrLocator : ${SOLR_LOCATOR} - } - } - ] - } -] diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineTestBase.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineTestBase.java deleted file mode 100644 index c91f31b70d6..00000000000 --- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineTestBase.java +++ /dev/null @@ -1,318 +0,0 @@ -/* - * 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.morphlines.solr; - -import com.codahale.metrics.MetricRegistry; -import com.google.common.io.Files; -import com.typesafe.config.Config; -import org.apache.commons.io.FileUtils; -import org.apache.solr.SolrTestCaseJ4; -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.impl.HttpSolrClient; -import org.apache.solr.client.solrj.impl.XMLResponseParser; -import org.apache.solr.client.solrj.response.QueryResponse; -import org.apache.solr.common.SolrDocument; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.kitesdk.morphline.api.Collector; -import org.kitesdk.morphline.api.Command; -import org.kitesdk.morphline.api.MorphlineContext; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.Compiler; -import org.kitesdk.morphline.base.FaultTolerance; -import org.kitesdk.morphline.base.Fields; -import org.kitesdk.morphline.base.Notifications; -import org.kitesdk.morphline.stdlib.PipeBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.util.Arrays; -import java.util.Calendar; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Locale; -import java.util.Map; -import java.util.Map.Entry; -import java.util.TimeZone; -import java.util.concurrent.atomic.AtomicInteger; - -public class AbstractSolrMorphlineTestBase extends SolrTestCaseJ4 { - private static Locale savedLocale; - protected Collector collector; - protected Command morphline; - protected SolrClient solrClient; - protected DocumentLoader testServer; - - protected static final boolean TEST_WITH_EMBEDDED_SOLR_SERVER = true; - protected static final String EXTERNAL_SOLR_SERVER_URL = System.getProperty("externalSolrServer"); -// protected static final String EXTERNAL_SOLR_SERVER_URL = "http://127.0.0.1:8983/solr"; - - protected static final String RESOURCES_DIR = getFile("morphlines-core.marker").getParent(); - protected static final String DEFAULT_BASE_DIR = "solr"; - protected static final AtomicInteger SEQ_NUM = new AtomicInteger(); - protected static final AtomicInteger SEQ_NUM2 = new AtomicInteger(); - - protected static final Object NON_EMPTY_FIELD = new Object(); - - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - protected String tempDir; - - @BeforeClass - public static void beforeClass() throws Exception { - - // TODO: test doesn't work with some Locales, see SOLR-6458 - savedLocale = Locale.getDefault(); - Locale.setDefault(Locale.ENGLISH); - - // we leave this in case the above is addressed - assumeFalse("This test fails on UNIX with Turkish default locale (https://issues.apache.org/jira/browse/SOLR-6387)", - new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage())); - - myInitCore(DEFAULT_BASE_DIR); - } - - @AfterClass - public static void afterClass() throws Exception { - if (savedLocale != null) { - Locale.setDefault(savedLocale); - } - savedLocale = null; - } - - protected static void myInitCore(String baseDirName) throws Exception { - String solrHome = RESOURCES_DIR + File.separator + baseDirName; - initCore( - "solrconfig.xml", "schema.xml", solrHome - ); - } - - @Before - public void setUp() throws Exception { - super.setUp(); - collector = new Collector(); - - if (EXTERNAL_SOLR_SERVER_URL != null) { - //solrServer = new ConcurrentUpdateSolrServer(EXTERNAL_SOLR_SERVER_URL, 2, 2); - //solrServer = new SafeConcurrentUpdateSolrServer(EXTERNAL_SOLR_SERVER_URL, 2, 2); - solrClient = getHttpSolrClient(EXTERNAL_SOLR_SERVER_URL); - ((HttpSolrClient) solrClient).setParser(new XMLResponseParser()); - } else { - if (TEST_WITH_EMBEDDED_SOLR_SERVER) { - solrClient = new EmbeddedTestSolrServer(h.getCoreContainer(), DEFAULT_TEST_CORENAME); - } else { - throw new RuntimeException("Not yet implemented"); - //solrServer = new TestSolrServer(getSolrClient()); - } - } - - int batchSize = SEQ_NUM2.incrementAndGet() % 2 == 0 ? 100 : 1; //SolrInspector.DEFAULT_SOLR_SERVER_BATCH_SIZE : 1; - testServer = new SolrClientDocumentLoader(solrClient, batchSize); - deleteAllDocuments(); - - tempDir = createTempDir().toFile().getAbsolutePath(); - } - - @After - public void tearDown() throws Exception { - collector = null; - solrClient.close(); - solrClient = null; - super.tearDown(); - } - - protected void testDocumentTypesInternal( - String[] files, - Map expectedRecords, - Map> expectedRecordContents) throws Exception { - - assumeTrue("This test has issues with this locale: https://issues.apache.org/jira/browse/SOLR-5778", - "GregorianCalendar".equals(Calendar.getInstance(TimeZone.getDefault(), Locale.getDefault()).getClass().getSimpleName())); - deleteAllDocuments(); - int numDocs = 0; - for (int i = 0; i < 1; i++) { - - for (String file : files) { - File f = new File(file); - byte[] body = Files.toByteArray(f); - Record event = new Record(); - //event.put(Fields.ID, docId++); - event.getFields().put(Fields.ATTACHMENT_BODY, new ByteArrayInputStream(body)); - event.getFields().put(Fields.ATTACHMENT_NAME, f.getName()); - event.getFields().put(Fields.BASE_ID, f.getName()); - collector.reset(); - load(event); - Integer count = expectedRecords.get(file); - if (count != null) { - numDocs += count; - } else { - numDocs++; - } - assertEquals("unexpected results in " + file, numDocs, queryResultSetSize("*:*")); - Map expectedContents = expectedRecordContents.get(file); - if (expectedContents != null) { - Record actual = collector.getFirstRecord(); - for (Map.Entry entry : expectedContents.entrySet()) { - if (entry.getValue() == NON_EMPTY_FIELD) { - assertNotNull(entry.getKey()); - assertTrue(actual.getFirstValue(entry.getKey()).toString().length() > 0); - } else if (entry.getValue() == null) { - assertEquals("key:" + entry.getKey(), 0, actual.get(entry.getKey()).size()); - } else { - assertEquals("key:" + entry.getKey(), Arrays.asList(entry.getValue()), actual.get(entry.getKey())); - } - } - } - } - } - assertEquals(numDocs, queryResultSetSize("*:*")); - } - - private boolean load(Record record) { - Notifications.notifyStartSession(morphline); - return morphline.process(record); - } - - protected int queryResultSetSize(String query) { -// return collector.getRecords().size(); - try { - testServer.commitTransaction(); - solrClient.commit(false, true, true); - QueryResponse rsp = solrClient.query(new SolrQuery(query).setRows(Integer.MAX_VALUE)); - LOGGER.debug("rsp: {}", rsp); - int i = 0; - for (SolrDocument doc : rsp.getResults()) { - LOGGER.debug("rspDoc #{}: {}", i++, doc); - } - int size = rsp.getResults().size(); - return size; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - private void deleteAllDocuments() throws SolrServerException, IOException { - collector.reset(); - SolrClient s = solrClient; - s.deleteByQuery("*:*"); // delete everything! - s.commit(); - } - - protected Command createMorphline(String file) throws IOException { - return new PipeBuilder().build(parse(file), null, collector, createMorphlineContext()); - } - - private MorphlineContext createMorphlineContext() { - return new SolrMorphlineContext.Builder() - .setDocumentLoader(testServer) -// .setDocumentLoader(new CollectingDocumentLoader(100)) - .setExceptionHandler(new FaultTolerance(false, false, SolrServerException.class.getName())) - .setMetricRegistry(new MetricRegistry()) - .build(); - } - - private Config parse(String file) throws IOException { - SolrLocator locator = new SolrLocator(createMorphlineContext()); - locator.setSolrHomeDir(testSolrHome + "/collection1"); - File morphlineFile; - if (new File(file).isAbsolute()) { - morphlineFile = new File(file + ".conf"); - } else { - morphlineFile = new File(RESOURCES_DIR + "/" + file + ".conf"); - } - Config config = new Compiler().parse(morphlineFile, locator.toConfig("SOLR_LOCATOR")); - config = config.getConfigList("morphlines").get(0); - return config; - } - - protected void startSession() { - Notifications.notifyStartSession(morphline); - } - - protected void testDocumentContent(HashMap expectedResultMap) - throws Exception { - QueryResponse rsp = solrClient.query(new SolrQuery("*:*").setRows(Integer.MAX_VALUE)); - // Check that every expected field/values shows up in the actual query - for (Entry current : expectedResultMap.entrySet()) { - String field = current.getKey(); - for (String expectedFieldValue : current.getValue().getFieldValues()) { - ExpectedResult.CompareType compareType = current.getValue().getCompareType(); - boolean foundField = false; - - for (SolrDocument doc : rsp.getResults()) { - Collection actualFieldValues = doc.getFieldValues(field); - if (compareType == ExpectedResult.CompareType.equals) { - if (actualFieldValues != null && actualFieldValues.contains(expectedFieldValue)) { - foundField = true; - break; - } - } - else { - for (Iterator it = actualFieldValues.iterator(); it.hasNext(); ) { - String actualValue = it.next().toString(); // test only supports string comparison - if (actualFieldValues != null && actualValue.contains(expectedFieldValue)) { - foundField = true; - break; - } - } - } - } - assert(foundField); // didn't find expected field/value in query - } - } - } - - /** - * Representation of the expected output of a SolrQuery. - */ - protected static class ExpectedResult { - private HashSet fieldValues; - public enum CompareType { - equals, // Compare with equals, i.e. actual.equals(expected) - contains; // Compare with contains, i.e. actual.contains(expected) - } - private CompareType compareType; - - public ExpectedResult(HashSet fieldValues, CompareType compareType) { - this.fieldValues = fieldValues; - this.compareType = compareType; - } - public HashSet getFieldValues() { return fieldValues; } - public CompareType getCompareType() { return compareType; } - } - - public static void setupMorphline(String tempDir, String file, boolean replaceSolrLocator) throws IOException { - String morphlineText = FileUtils.readFileToString(new File(RESOURCES_DIR + "/" + file + ".conf"), "UTF-8"); - morphlineText = morphlineText.replace("RESOURCES_DIR", new File(tempDir).getAbsolutePath()); - if (replaceSolrLocator) { - morphlineText = morphlineText.replace("${SOLR_LOCATOR}", - "{ collection : collection1 }"); - } - new File(tempDir + "/" + file + ".conf").getParentFile().mkdirs(); - FileUtils.writeStringToFile(new File(tempDir + "/" + file + ".conf"), morphlineText, "UTF-8"); - } -} diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineZkTestBase.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineZkTestBase.java deleted file mode 100644 index 31e7ebfe84b..00000000000 --- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/AbstractSolrMorphlineZkTestBase.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import java.util.Locale; - -import com.codahale.metrics.MetricRegistry; -import com.google.common.collect.ListMultimap; -import com.typesafe.config.Config; -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.request.CollectionAdminRequest; -import org.apache.solr.cloud.AbstractDistribZkTestBase; -import org.apache.solr.cloud.SolrCloudTestCase; -import org.apache.solr.common.SolrDocument; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.kitesdk.morphline.api.Collector; -import org.kitesdk.morphline.api.Command; -import org.kitesdk.morphline.api.MorphlineContext; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.Compiler; -import org.kitesdk.morphline.base.FaultTolerance; -import org.kitesdk.morphline.base.Notifications; -import org.kitesdk.morphline.stdlib.PipeBuilder; - -public abstract class AbstractSolrMorphlineZkTestBase extends SolrCloudTestCase { - - protected static final String COLLECTION = "collection1"; - - protected static final int TIMEOUT = 30; - - @BeforeClass - public static void setupCluster() throws Exception { - // set some system properties for use by tests - System.setProperty("solr.test.sys.prop1", "propone"); - System.setProperty("solr.test.sys.prop2", "proptwo"); - - configureCluster(2) - .addConfig("conf", SOLR_CONF_DIR.toPath()) - .configure(); - - CollectionAdminRequest.createCollection(COLLECTION, "conf", 2, 1) - .processAndWait(cluster.getSolrClient(), TIMEOUT); - AbstractDistribZkTestBase.waitForRecoveriesToFinish(COLLECTION, cluster.getSolrClient().getZkStateReader(), - false, true, TIMEOUT); - } - - @AfterClass - public static void afterClass() { - System.clearProperty("solr.test.sys.prop1"); - System.clearProperty("solr.test.sys.prop2"); - } - - protected static final String RESOURCES_DIR = getFile("morphlines-core.marker").getParent(); - private static final File SOLR_CONF_DIR = new File(RESOURCES_DIR + "/solr/collection1/conf"); - - protected Collector collector; - protected Command morphline; - - @BeforeClass - public static void setupClass() throws Exception { - - assumeFalse("This test fails on UNIX with Turkish default locale (https://issues.apache.org/jira/browse/SOLR-6387)", - new Locale("tr").getLanguage().equals(Locale.getDefault().getLanguage())); - - } - - @Before - public void setup() throws Exception { - collector = new Collector(); - cluster.waitForAllNodes(DEFAULT_TIMEOUT); - } - - protected void commit() throws Exception { - Notifications.notifyCommitTransaction(morphline); - } - - protected Command parse(String file) throws IOException { - return parse(file, COLLECTION); - } - - protected Command parse(String file, String collection) throws IOException { - SolrLocator locator = new SolrLocator(createMorphlineContext()); - locator.setCollectionName(collection); - locator.setZkHost(cluster.getZkServer().getZkAddress()); - //locator.setServerUrl(cloudJettys.get(0).url); // TODO: download IndexSchema from solrUrl not yet implemented - //locator.setSolrHomeDir(SOLR_HOME_DIR.getPath()); - Config config = new Compiler().parse(new File(RESOURCES_DIR + "/" + file + ".conf"), locator.toConfig("SOLR_LOCATOR")); - config = config.getConfigList("morphlines").get(0); - return createMorphline(config); - } - - private Command createMorphline(Config config) { - return new PipeBuilder().build(config, null, collector, createMorphlineContext()); - } - - private MorphlineContext createMorphlineContext() { - return new MorphlineContext.Builder() - .setExceptionHandler(new FaultTolerance(false, false, SolrServerException.class.getName())) - .setMetricRegistry(new MetricRegistry()) - .build(); - } - - protected void startSession() { - Notifications.notifyStartSession(morphline); - } - - protected ListMultimap next(Iterator iter) { - SolrDocument doc = iter.next(); - Record record = toRecord(doc); - record.removeAll("_version_"); // the values of this field are unknown and internal to solr - return record.getFields(); - } - - private Record toRecord(SolrDocument doc) { - Record record = new Record(); - for (String key : doc.keySet()) { - record.getFields().replaceValues(key, doc.getFieldValues(key)); - } - return record; - } - -} diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/CollectingDocumentLoader.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/CollectingDocumentLoader.java deleted file mode 100644 index 3864e6d19c4..00000000000 --- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/CollectingDocumentLoader.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.lang.invoke.MethodHandles; -import java.util.ArrayList; -import java.util.List; - -import org.apache.solr.client.solrj.response.SolrPingResponse; -import org.apache.solr.client.solrj.response.UpdateResponse; -import org.apache.solr.common.SolrInputDocument; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A mockup DocumentLoader implementation for unit tests; collects all documents into a main memory list. - */ -class CollectingDocumentLoader implements DocumentLoader { - - private final int batchSize; - private final List batch = new ArrayList<> (); - private List results = new ArrayList<> (); - - private static final Logger LOGGER = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - - public CollectingDocumentLoader(int batchSize) { - if (batchSize <= 0) { - throw new IllegalArgumentException("batchSize must be a positive number: " + batchSize); - } - this.batchSize = batchSize; - } - - @Override - public void beginTransaction() { - LOGGER.trace("beginTransaction"); - batch.clear(); - } - - @Override - public void load(SolrInputDocument doc) { - LOGGER.trace("load doc: {}", doc); - batch.add(doc); - if (batch.size() >= batchSize) { - loadBatch(); - } - } - - @Override - public void commitTransaction() { - LOGGER.trace("commitTransaction"); - if (batch.size() > 0) { - loadBatch(); - } - } - - private void loadBatch() { - try { - results.addAll(batch); - } finally { - batch.clear(); - } - } - - @Override - public UpdateResponse rollbackTransaction() { - LOGGER.trace("rollback"); - return new UpdateResponse(); - } - - @Override - public void shutdown() { - LOGGER.trace("shutdown"); - } - - @Override - public SolrPingResponse ping() { - LOGGER.trace("ping"); - return new SolrPingResponse(); - } - -} diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/EmbeddedTestSolrServer.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/EmbeddedTestSolrServer.java deleted file mode 100644 index 9599511d8f9..00000000000 --- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/EmbeddedTestSolrServer.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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.morphlines.solr; - -import org.apache.solr.client.solrj.SolrServerException; -import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer; -import org.apache.solr.client.solrj.response.UpdateResponse; -import org.apache.solr.core.CoreContainer; - -import java.io.IOException; - -/** - * An EmbeddedSolrServer that supresses close and rollback requests as - * necessary for testing - */ -public class EmbeddedTestSolrServer extends EmbeddedSolrServer { - - public EmbeddedTestSolrServer(CoreContainer coreContainer, String coreName) { - super(coreContainer, coreName); - } - - @Override - public void close() { - ; // NOP - } - - @Override - public UpdateResponse rollback() throws SolrServerException, IOException { - return new UpdateResponse(); - } - -} diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineTest.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineTest.java deleted file mode 100644 index 64fa2ec6d70..00000000000 --- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineTest.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.morphlines.solr; - -import org.junit.Test; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.Fields; -import org.kitesdk.morphline.base.Notifications; - -import java.io.File; -import java.util.Arrays; - -public class SolrMorphlineTest extends AbstractSolrMorphlineTestBase { - - @Test - public void testLoadSolrBasic() throws Exception { - //System.setProperty("ENV_SOLR_HOME", testSolrHome + "/collection1"); - morphline = createMorphline("test-morphlines/loadSolrBasic"); - //System.clearProperty("ENV_SOLR_HOME"); - Record record = new Record(); - record.put(Fields.ID, "id0"); - record.put("first_name", "Nadja"); // will be sanitized - startSession(); - Notifications.notifyBeginTransaction(morphline); - assertTrue(morphline.process(record)); - assertEquals(1, collector.getNumStartEvents()); - Notifications.notifyCommitTransaction(morphline); - Record expected = new Record(); - expected.put(Fields.ID, "id0"); - assertEquals(Arrays.asList(expected), collector.getRecords()); - assertEquals(1, queryResultSetSize("*:*")); - Notifications.notifyRollbackTransaction(morphline); - Notifications.notifyShutdown(morphline); - } - - @Test - public void testTokenizeText() throws Exception { - morphline = createMorphline("test-morphlines" + File.separator + "tokenizeText"); - for (int i = 0; i < 3; i++) { - Record record = new Record(); - record.put(Fields.MESSAGE, "Hello World!"); - record.put(Fields.MESSAGE, "\nFoo@Bar.com #%()123"); - Record expected = record.copy(); - expected.getFields().putAll("tokens", Arrays.asList("hello", "world", "foo", "bar.com", "123")); - collector.reset(); - startSession(); - Notifications.notifyBeginTransaction(morphline); - assertTrue(morphline.process(record)); - assertEquals(1, collector.getNumStartEvents()); - Notifications.notifyCommitTransaction(morphline); - assertEquals(expected, collector.getFirstRecord()); - } - } - -} diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAliasTest.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAliasTest.java deleted file mode 100644 index 74c8824ddc8..00000000000 --- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAliasTest.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.io.File; -import java.util.Iterator; - -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; -import org.apache.lucene.util.LuceneTestCase.Slow; -import org.apache.solr.client.solrj.SolrQuery; -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.util.BadHdfsThreadsFilter; -import org.junit.Test; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.Fields; -import org.kitesdk.morphline.base.Notifications; - -@ThreadLeakFilters(defaultFilters = true, filters = { - BadHdfsThreadsFilter.class // hdfs currently leaks thread(s) -}) -@Slow -public class SolrMorphlineZkAliasTest extends AbstractSolrMorphlineZkTestBase { - - @Test - public void test() throws Exception { - - CollectionAdminRequest.createAlias("aliascollection", "collection1") - .process(cluster.getSolrClient()); - - morphline = parse("test-morphlines" + File.separator + "loadSolrBasic", "aliascollection"); - Record record = new Record(); - record.put(Fields.ID, "id0-innsbruck"); - record.put("text", "mytext"); - record.put("user_screen_name", "foo"); - record.put("first_name", "Nadja"); // will be sanitized - startSession(); - assertEquals(1, collector.getNumStartEvents()); - Notifications.notifyBeginTransaction(morphline); - assertTrue(morphline.process(record)); - - record = new Record(); - record.put(Fields.ID, "id1-innsbruck"); - record.put("text", "mytext1"); - record.put("user_screen_name", "foo1"); - record.put("first_name", "Nadja1"); // will be sanitized - assertTrue(morphline.process(record)); - - Record expected = new Record(); - expected.put(Fields.ID, "id0-innsbruck"); - expected.put("text", "mytext"); - expected.put("user_screen_name", "foo"); - Iterator citer = collector.getRecords().iterator(); - assertEquals(expected, citer.next()); - - Record expected2 = new Record(); - expected2.put(Fields.ID, "id1-innsbruck"); - expected2.put("text", "mytext1"); - expected2.put("user_screen_name", "foo1"); - assertEquals(expected2, citer.next()); - - assertFalse(citer.hasNext()); - - Notifications.notifyCommitTransaction(morphline); - new UpdateRequest().commit(cluster.getSolrClient(), COLLECTION); - - QueryResponse rsp = cluster.getSolrClient() - .query(COLLECTION, new SolrQuery("*:*").setRows(100000).addSort(Fields.ID, SolrQuery.ORDER.asc)); - //System.out.println(rsp); - Iterator iter = rsp.getResults().iterator(); - assertEquals(expected.getFields(), next(iter)); - assertEquals(expected2.getFields(), next(iter)); - assertFalse(iter.hasNext()); - - Notifications.notifyRollbackTransaction(morphline); - Notifications.notifyShutdown(morphline); - - } - -} diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAvroTest.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAvroTest.java deleted file mode 100644 index 49e2a0ea066..00000000000 --- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkAvroTest.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.io.File; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; - -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.file.FileReader; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.lucene.util.LuceneTestCase.Slow; -import org.apache.solr.client.solrj.SolrQuery; -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.util.BadHdfsThreadsFilter; -import org.junit.Test; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.Fields; -import org.kitesdk.morphline.base.Notifications; - -@ThreadLeakFilters(defaultFilters = true, filters = { - BadHdfsThreadsFilter.class // hdfs currently leaks thread(s) -}) -@Slow -public class SolrMorphlineZkAvroTest extends AbstractSolrMorphlineZkTestBase { - - - @Test - public void test() throws Exception { - - Path avro = Paths.get(RESOURCES_DIR).resolve("test-documents").resolve("sample-statuses-20120906-141433-medium.avro"); - - // load avro records via morphline and zk into solr - morphline = parse("test-morphlines" + File.separator + "tutorialReadAvroContainer"); - Record record = new Record(); - byte[] body = Files.readAllBytes(avro); - record.put(Fields.ATTACHMENT_BODY, body); - startSession(); - Notifications.notifyBeginTransaction(morphline); - assertTrue(morphline.process(record)); - assertEquals(1, collector.getNumStartEvents()); - - Notifications.notifyCommitTransaction(morphline); - new UpdateRequest().commit(cluster.getSolrClient(), COLLECTION); - - // fetch sorted result set from solr - QueryResponse rsp = cluster.getSolrClient() - .query(COLLECTION, new SolrQuery("*:*").setRows(100000).addSort("id", SolrQuery.ORDER.asc)); - assertEquals(2104, collector.getRecords().size()); - assertEquals(collector.getRecords().size(), rsp.getResults().size()); - - Collections.sort(collector.getRecords(), (r1, r2) -> r1.get("id").toString().compareTo(r2.get("id").toString())); - - // fetch test input data and sort like solr result set - List records = new ArrayList<>(); - FileReader reader = new DataFileReader(avro.toFile(), new GenericDatumReader()); - while (reader.hasNext()) { - GenericData.Record expected = reader.next(); - records.add(expected); - } - assertEquals(collector.getRecords().size(), records.size()); - Collections.sort(records, (r1, r2) -> r1.get("id").toString().compareTo(r2.get("id").toString())); - - Object lastId = null; - for (int i = 0; i < records.size(); i++) { - //System.out.println("myrec" + i + ":" + records.get(i)); - Object id = records.get(i); - if (id != null && id.equals(lastId)) { - throw new IllegalStateException("Detected duplicate id. Test input data must not contain duplicate ids!"); - } - lastId = id; - } - - for (int i = 0; i < records.size(); i++) { - //System.out.println("myrsp" + i + ":" + rsp.getResults().get(i)); - } - - Iterator rspIter = rsp.getResults().iterator(); - for (int i = 0; i < records.size(); i++) { - // verify morphline spat out expected data - Record actual = collector.getRecords().get(i); - GenericData.Record expected = Objects.requireNonNull(records.get(i)); - assertTweetEquals(expected, actual, i); - - // verify Solr result set contains expected data - actual = new Record(); - actual.getFields().putAll(next(rspIter)); - assertTweetEquals(expected, actual, i); - } - - Notifications.notifyRollbackTransaction(morphline); - Notifications.notifyShutdown(morphline); - - } - - private void assertTweetEquals(GenericData.Record expected, Record actual, int i) { - Objects.requireNonNull(expected); - Objects.requireNonNull(actual); -// System.out.println("\n\nexpected: " + toString(expected)); -// System.out.println("actual: " + actual); - String[] fieldNames = new String[] { - "id", - "in_reply_to_status_id", - "in_reply_to_user_id", - "retweet_count", - "text", - }; - for (String fieldName : fieldNames) { - assertEquals( - i + " fieldName: " + fieldName, - expected.get(fieldName).toString(), - actual.getFirstValue(fieldName).toString()); - } - } - -} diff --git a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkTest.java b/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkTest.java deleted file mode 100644 index 24d8682003b..00000000000 --- a/solr/contrib/morphlines-core/src/test/org/apache/solr/morphlines/solr/SolrMorphlineZkTest.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.morphlines.solr; - -import java.io.File; -import java.util.Iterator; - -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; -import org.apache.lucene.util.LuceneTestCase.Slow; -import org.apache.solr.client.solrj.SolrQuery; -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.util.BadHdfsThreadsFilter; -import org.junit.Test; -import org.kitesdk.morphline.api.Record; -import org.kitesdk.morphline.base.Fields; -import org.kitesdk.morphline.base.Notifications; - -@ThreadLeakFilters(defaultFilters = true, filters = { - BadHdfsThreadsFilter.class // hdfs currently leaks thread(s) -}) -@Slow -public class SolrMorphlineZkTest extends AbstractSolrMorphlineZkTestBase { - - @Test - public void test() throws Exception { - - morphline = parse("test-morphlines" + File.separator + "loadSolrBasic"); - Record record = new Record(); - record.put(Fields.ID, "id0-innsbruck"); - record.put("text", "mytext"); - record.put("user_screen_name", "foo"); - record.put("first_name", "Nadja"); // will be sanitized - startSession(); - assertEquals(1, collector.getNumStartEvents()); - Notifications.notifyBeginTransaction(morphline); - assertTrue(morphline.process(record)); - - record = new Record(); - record.put(Fields.ID, "id1-innsbruck"); - record.put("text", "mytext1"); - record.put("user_screen_name", "foo1"); - record.put("first_name", "Nadja1"); // will be sanitized - assertTrue(morphline.process(record)); - - Record expected = new Record(); - expected.put(Fields.ID, "id0-innsbruck"); - expected.put("text", "mytext"); - expected.put("user_screen_name", "foo"); - Iterator citer = collector.getRecords().iterator(); - assertEquals(expected, citer.next()); - - Record expected2 = new Record(); - expected2.put(Fields.ID, "id1-innsbruck"); - expected2.put("text", "mytext1"); - expected2.put("user_screen_name", "foo1"); - assertEquals(expected2, citer.next()); - - assertFalse(citer.hasNext()); - - Notifications.notifyCommitTransaction(morphline); - new UpdateRequest().commit(cluster.getSolrClient(), COLLECTION); - - QueryResponse rsp = cluster.getSolrClient() - .query(COLLECTION, new SolrQuery("*:*").setRows(100000).addSort(Fields.ID, SolrQuery.ORDER.asc)); - //System.out.println(rsp); - Iterator iter = rsp.getResults().iterator(); - assertEquals(expected.getFields(), next(iter)); - assertEquals(expected2.getFields(), next(iter)); - assertFalse(iter.hasNext()); - - Notifications.notifyRollbackTransaction(morphline); - Notifications.notifyShutdown(morphline); - - } - -} diff --git a/solr/licenses/Saxon-HE-9.6.0-2.jar.sha1 b/solr/licenses/Saxon-HE-9.6.0-2.jar.sha1 deleted file mode 100644 index 56981f2d3bf..00000000000 --- a/solr/licenses/Saxon-HE-9.6.0-2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -826a6c32fef050346b37e91b36fde16cf932da12 diff --git a/solr/licenses/Saxon-HE-LICENSE-MPL.txt b/solr/licenses/Saxon-HE-LICENSE-MPL.txt deleted file mode 100644 index 608cd2ef524..00000000000 --- a/solr/licenses/Saxon-HE-LICENSE-MPL.txt +++ /dev/null @@ -1,108 +0,0 @@ -MOZILLA PUBLIC LICENSE -Version 1.0 - -1. Definitions. - -1.1. ``Contributor'' means each entity that creates or contributes to the creation of Modifications. -1.2. ``Contributor Version'' means the combination of the Original Code, prior Modifications used by a Contributor, and the Modifications made by that particular Contributor. - -1.3. ``Covered Code'' means the Original Code or Modifications or the combination of the Original Code and Modifications, in each case including portions thereof. - -1.4. ``Electronic Distribution Mechanism'' means a mechanism generally accepted in the software development community for the electronic transfer of data. - -1.5. ``Executable'' means Covered Code in any form other than Source Code. - -1.6. ``Initial Developer'' means the individual or entity identified as the Initial Developer in the Source Code notice required by Exhibit A. - -1.7. ``Larger Work'' means a work which combines Covered Code or portions thereof with code not governed by the terms of this License. - -1.8. ``License'' means this document. - -1.9. ``Modifications'' means any addition to or deletion from the substance or structure of either the Original Code or any previous Modifications. When Covered Code is released as a series of files, a Modification is: - -A. Any addition to or deletion from the contents of a file containing Original Code or previous Modifications. - -B. Any new file that contains any part of the Original Code or previous Modifications. - -1.10. ``Original Code'' means Source Code of computer software code which is described in the Source Code notice required by Exhibit A as Original Code, and which, at the time of its release under this License is not already Covered Code governed by this License. - -1.11. ``Source Code'' means the preferred form of the Covered Code for making modifications to it, including all modules it contains, plus any associated interface definition files, scripts used to control compilation and installation of an Executable, or a list of source code differential comparisons against either the Original Code or another well known, available Covered Code of the Contributor's choice. The Source Code can be in a compressed or archival form, provided the appropriate decompression or de-archiving software is widely available for no charge. - -1.12. ``You'' means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License or a future version of this License issued under Section 6.1. For legal entities, ``You'' includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, ``control'' means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of fifty percent (50%) or more of the outstanding shares or beneficial ownership of such entity. - -2. Source Code License. -2.1. The Initial Developer Grant. -The Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license, subject to third party intellectual property claims: -(a) to use, reproduce, modify, display, perform, sublicense and distribute the Original Code (or portions thereof) with or without Modifications, or as part of a Larger Work; and - -(b) under patents now or hereafter owned or controlled by Initial Developer, to make, have made, use and sell (``Utilize'') the Original Code (or portions thereof), but solely to the extent that any such patent is reasonably necessary to enable You to Utilize the Original Code (or portions thereof) and not to any greater extent that may be necessary to Utilize further Modifications or combinations. - -2.2. Contributor Grant. -Each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license, subject to third party intellectual property claims: - -(a) to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof) either on an unmodified basis, with other Modifications, as Covered Code or as part of a Larger Work; and - -(b) under patents now or hereafter owned or controlled by Contributor, to Utilize the Contributor Version (or portions thereof), but solely to the extent that any such patent is reasonably necessary to enable You to Utilize the Contributor Version (or portions thereof), and not to any greater extent that may be necessary to Utilize further Modifications or combinations. - -3. Distribution Obligations. -3.1. Application of License. -The Modifications which You create or to which You contribute are governed by the terms of this License, including without limitation Section 2.2. The Source Code version of Covered Code may be distributed only under the terms of this License or a future version of this License released under Section 6.1, and You must include a copy of this License with every copy of the Source Code You distribute. You may not offer or impose any terms on any Source Code version that alters or restricts the applicable version of this License or the recipients' rights hereunder. However, You may include an additional document offering the additional rights described in Section 3.5. -3.2. Availability of Source Code. -Any Modification which You create or to which You contribute must be made available in Source Code form under the terms of this License either on the same media as an Executable version or via an accepted Electronic Distribution Mechanism to anyone to whom you made an Executable version available; and if made available via Electronic Distribution Mechanism, must remain available for at least twelve (12) months after the date it initially became available, or at least six (6) months after a subsequent version of that particular Modification has been made available to such recipients. You are responsible for ensuring that the Source Code version remains available even if the Electronic Distribution Mechanism is maintained by a third party. - -3.3. Description of Modifications. -You must cause all Covered Code to which you contribute to contain a file documenting the changes You made to create that Covered Code and the date of any change. You must include a prominent statement that the Modification is derived, directly or indirectly, from Original Code provided by the Initial Developer and including the name of the Initial Developer in (a) the Source Code, and (b) in any notice in an Executable version or related documentation in which You describe the origin or ownership of the Covered Code. - -3.4. Intellectual Property Matters - -(a) Third Party Claims. -If You have knowledge that a party claims an intellectual property right in particular functionality or code (or its utilization under this License), you must include a text file with the source code distribution titled ``LEGAL'' which describes the claim and the party making the claim in sufficient detail that a recipient will know whom to contact. If you obtain such knowledge after You make Your Modification available as described in Section 3.2, You shall promptly modify the LEGAL file in all copies You make available thereafter and shall take other steps (such as notifying appropriate mailing lists or newsgroups) reasonably calculated to inform those who received the Covered Code that new knowledge has been obtained. - -(b) Contributor APIs. -If Your Modification is an application programming interface and You own or control patents which are reasonably necessary to implement that API, you must also include this information in the LEGAL file. - -3.5. Required Notices. -You must duplicate the notice in Exhibit A in each file of the Source Code, and this License in any documentation for the Source Code, where You describe recipients' rights relating to Covered Code. If You created one or more Modification(s), You may add your name as a Contributor to the notice described in Exhibit A. If it is not possible to put such notice in a particular Source Code file due to its structure, then you must include such notice in a location (such as a relevant directory file) where a user would be likely to look for such a notice. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Code. However, You may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear than any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. - -3.6. Distribution of Executable Versions. -You may distribute Covered Code in Executable form only if the requirements of Section 3.1-3.5 have been met for that Covered Code, and if You include a notice stating that the Source Code version of the Covered Code is available under the terms of this License, including a description of how and where You have fulfilled the obligations of Section 3.2. The notice must be conspicuously included in any notice in an Executable version, related documentation or collateral in which You describe recipients' rights relating to the Covered Code. You may distribute the Executable version of Covered Code under a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable version does not attempt to limit or alter the recipient's rights in the Source Code version from the rights set forth in this License. If You distribute the Executable version under a different license You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or any Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. - -3.7. Larger Works. -You may create a Larger Work by combining Covered Code with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Code. - -4. Inability to Comply Due to Statute or Regulation. -If it is impossible for You to comply with any of the terms of this License with respect to some or all of the Covered Code due to statute or regulation then You must: (a) comply with the terms of this License to the maximum extent possible; and (b) describe the limitations and the code they affect. Such description must be included in the LEGAL file described in Section 3.4 and must be included with all distributions of the Source Code. Except to the extent prohibited by statute or regulation, such description must be sufficiently detailed for a recipient of ordinary skill to be able to understand it. - -5. Application of this License. -This License applies to code to which the Initial Developer has attached the notice in Exhibit A, and to related Covered Code. -6. Versions of the License. -6.1. New Versions. -Netscape Communications Corporation (``Netscape'') may publish revised and/or new versions of the License from time to time. Each version will be given a distinguishing version number. -6.2. Effect of New Versions. -Once Covered Code has been published under a particular version of the License, You may always continue to use it under the terms of that version. You may also choose to use such Covered Code under the terms of any subsequent version of the License published by Netscape. No one other than Netscape has the right to modify the terms applicable to Covered Code created under this License. - -6.3. Derivative Works. -If you create or use a modified version of this License (which you may only do in order to apply it to code which is not already Covered Code governed by this License), you must (a) rename Your license so that the phrases ``Mozilla'', ``MOZILLAPL'', ``MOZPL'', ``Netscape'', ``NPL'' or any confusingly similar phrase do not appear anywhere in your license and (b) otherwise make it clear that your version of the license contains terms which differ from the Mozilla Public License and Netscape Public License. (Filling in the name of the Initial Developer, Original Code or Contributor in the notice described in Exhibit A shall not of themselves be deemed to be modifications of this License.) - -7. DISCLAIMER OF WARRANTY. -COVERED CODE IS PROVIDED UNDER THIS LICENSE ON AN ``AS IS'' BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED CODE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED CODE IS WITH YOU. SHOULD ANY COVERED CODE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED CODE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. -8. TERMINATION. -This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. All sublicenses to the Covered Code which are properly granted shall survive any termination of this License. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. -9. LIMITATION OF LIABILITY. -UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED CODE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO YOU OR ANY OTHER PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THAT EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. -10. U.S. GOVERNMENT END USERS. -The Covered Code is a ``commercial item,'' as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of ``commercial computer software'' and ``commercial computer software documentation,'' as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Code with only those rights set forth herein. -11. MISCELLANEOUS. -This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by California law provisions (except to the extent applicable law, if any, provides otherwise), excluding its conflict-of-law provisions. With respect to disputes in which at least one party is a citizen of, or an entity chartered or registered to do business in, the United States of America: (a) unless otherwise agreed in writing, all disputes relating to this License (excepting any dispute relating to intellectual property rights) shall be subject to final and binding arbitration, with the losing party paying all costs of arbitration; (b) any arbitration relating to this Agreement shall be held in Santa Clara County, California, under the auspices of JAMS/EndDispute; and (c) any litigation relating to this Agreement shall be subject to the jurisdiction of the Federal Courts of the Northern District of California, with venue lying in Santa Clara County, California, with the losing party responsible for costs, including without limitation, court costs and reasonable attorneys fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. -12. RESPONSIBILITY FOR CLAIMS. -Except in cases where another Contributor has failed to comply with Section 3.4, You are responsible for damages arising, directly or indirectly, out of Your utilization of rights under this License, based on the number of copies of Covered Code you made available, the revenues you received from utilizing such rights, and other relevant factors. You agree to work with affected parties to distribute responsibility on an equitable basis. -EXHIBIT A. -``The contents of this file are subject to the Mozilla Public License Version 1.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.mozilla.org/MPL/ -Software distributed under the License is distributed on an "AS IS" basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See the License for the specific language governing rights and limitations under the License. - -The Original Code is ______________________________________. - -The Initial Developer of the Original Code is ________________________. Portions created by ______________________ are Copyright (C) ______ _______________________. All Rights Reserved. - -Contributor(s): ______________________________________.'' - diff --git a/solr/licenses/aopalliance-1.0.jar.sha1 b/solr/licenses/aopalliance-1.0.jar.sha1 deleted file mode 100644 index 5da3c21c7f4..00000000000 --- a/solr/licenses/aopalliance-1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0235ba8b489512805ac13a8f9ea77a1ca5ebe3e8 diff --git a/solr/licenses/aopalliance-LICENSE-PD.txt b/solr/licenses/aopalliance-LICENSE-PD.txt deleted file mode 100644 index c75d4e6d9aa..00000000000 --- a/solr/licenses/aopalliance-LICENSE-PD.txt +++ /dev/null @@ -1 +0,0 @@ -Released to Public Domain \ No newline at end of file diff --git a/solr/licenses/argparse4j-0.4.3.jar.sha1 b/solr/licenses/argparse4j-0.4.3.jar.sha1 deleted file mode 100644 index 3223c06e88a..00000000000 --- a/solr/licenses/argparse4j-0.4.3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f4c256934d79940477a35b6c4c182b47ee5f0a6f diff --git a/solr/licenses/argparse4j-LICENSE-MIT.txt b/solr/licenses/argparse4j-LICENSE-MIT.txt deleted file mode 100644 index 42612cfa84a..00000000000 --- a/solr/licenses/argparse4j-LICENSE-MIT.txt +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Copyright (C) 2011, 2013 Tatsuhiro Tsujikawa - * - * Permission is hereby granted, free of charge, to any person - * obtaining a copy of this software and associated documentation - * files (the "Software"), to deal in the Software without - * restriction, including without limitation the rights to use, copy, - * modify, merge, publish, distribute, sublicense, and/or sell copies - * of the Software, and to permit persons to whom the Software is - * furnished to do so, subject to the following conditions: - * - * The above copyright notice and this permission notice shall be - * included in all copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, - * EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF - * MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND - * NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS - * BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN - * ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN - * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE - * SOFTWARE. - */ \ No newline at end of file diff --git a/solr/licenses/avro-1.7.5.jar.sha1 b/solr/licenses/avro-1.7.5.jar.sha1 deleted file mode 100644 index d19293c78e0..00000000000 --- a/solr/licenses/avro-1.7.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8343a5b33f56fa16306ed27fa7b1a79278c26c2d diff --git a/solr/licenses/avro-LICENSE-ASL.txt b/solr/licenses/avro-LICENSE-ASL.txt deleted file mode 100644 index 2f23f979d2d..00000000000 --- a/solr/licenses/avro-LICENSE-ASL.txt +++ /dev/null @@ -1,308 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - ----------------------------------------------------------------------- -License for the Jansson C JSON parser used in the C implementation: - -Copyright (c) 2009 Petri Lehtinen - -Permission is hereby granted, free of charge, to any person obtaining a copy -of this software and associated documentation files (the "Software"), to deal -in the Software without restriction, including without limitation the rights -to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -copies of the Software, and to permit persons to whom the Software is -furnished to do so, subject to the following conditions: - -The above copyright notice and this permission notice shall be included in -all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -THE SOFTWARE. - ----------------------------------------------------------------------- -License for the Json.NET used in the C# implementation: - -Copyright (c) 2007 James Newton-King - -Permission is hereby granted, free of charge, to any person obtaining -a copy of this software and associated documentation files (the -"Software"), to deal in the Software without restriction, including -without limitation the rights to use, copy, modify, merge, publish, -distribute, sublicense, and/or sell copies of the Software, and to -permit persons to whom the Software is furnished to do so, subject to -the following conditions: - -The above copyright notice and this permission notice shall be -included in all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, -EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF -MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND -NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE -LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION -OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION -WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - ----------------------------------------------------------------------- -License for msinttypes used in the C implementation: -Source from: -http://code.google.com/p/msinttypes/downloads/detail?name=msinttypes-r26.zip - -Copyright (c) 2006-2008 Alexander Chemeris - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - - 1. Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - - 2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - - 3. The name of the author may be used to endorse or promote products - derived from this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR IMPLIED -WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF -MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO -EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, -PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; -OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, -WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR -OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF -ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - ----------------------------------------------------------------------- -License for Dirent API for Microsoft Visual Studio used in the C implementation: -Source from: -http://www.softagalleria.net/download/dirent/dirent-1.11.zip - -Copyright (C) 2006 Toni Ronkko - -Permission is hereby granted, free of charge, to any person obtaining -a copy of this software and associated documentation files (the -``Software''), to deal in the Software without restriction, including -without limitation the rights to use, copy, modify, merge, publish, -distribute, sublicense, and/or sell copies of the Software, and to -permit persons to whom the Software is furnished to do so, subject to -the following conditions: - -The above copyright notice and this permission notice shall be included -in all copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED ``AS IS'', WITHOUT WARRANTY OF ANY KIND, EXPRESS -OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF -MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. -IN NO EVENT SHALL TONI RONKKO BE LIABLE FOR ANY CLAIM, DAMAGES OR -OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, -ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR -OTHER DEALINGS IN THE SOFTWARE. - ----------------------------------------------------------------------- \ No newline at end of file diff --git a/solr/licenses/avro-NOTICE.txt b/solr/licenses/avro-NOTICE.txt deleted file mode 100644 index da479fec1be..00000000000 --- a/solr/licenses/avro-NOTICE.txt +++ /dev/null @@ -1,9 +0,0 @@ -Apache Avro -Copyright 2010 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -C JSON parsing provided by Jansson and -written by Petri Lehtinen. The original software is -available from http://www.digip.org/jansson/. \ No newline at end of file diff --git a/solr/licenses/bcpkix-jdk15on-1.47.jar.sha1 b/solr/licenses/bcpkix-jdk15on-1.47.jar.sha1 deleted file mode 100644 index 1a7e7bedcb4..00000000000 --- a/solr/licenses/bcpkix-jdk15on-1.47.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -cd204e6f26d2bbf65ff3a30de8831d3a1344e851 diff --git a/solr/licenses/bcpkix-jdk15on-LICENSE-BSD_LIKE.txt b/solr/licenses/bcpkix-jdk15on-LICENSE-BSD_LIKE.txt deleted file mode 100644 index 9dfdf7c6a12..00000000000 --- a/solr/licenses/bcpkix-jdk15on-LICENSE-BSD_LIKE.txt +++ /dev/null @@ -1,15 +0,0 @@ -Copyright (c) 2000-2010 The Legion Of The Bouncy Castle (http://www.bouncycastle.org) -Permission is hereby granted, free of charge, to any person obtaining a copy of this software and -associated documentation files (the "Software"), to deal in the Software without restriction, -including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, -and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, - subject to the following conditions: - -The above copyright notice and this permission notice shall be included in all copies or substantial portions - of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED - TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL - THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF - CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS - IN THE SOFTWARE. diff --git a/solr/licenses/bcpkix-jdk15on-NOTICE.txt b/solr/licenses/bcpkix-jdk15on-NOTICE.txt deleted file mode 100644 index be0638a2ba8..00000000000 --- a/solr/licenses/bcpkix-jdk15on-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -Copyright (c) 2000-2006 The Legion Of The Bouncy Castle -(http://www.bouncycastle.org) diff --git a/solr/licenses/config-1.0.2.jar.sha1 b/solr/licenses/config-1.0.2.jar.sha1 deleted file mode 100644 index 6dbf80d7f94..00000000000 --- a/solr/licenses/config-1.0.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a0bca82c39f23f75e3afccd6e12840eeabaea123 diff --git a/solr/licenses/config-LICENSE-ASL.txt b/solr/licenses/config-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/config-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/config-NOTICE.txt b/solr/licenses/config-NOTICE.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/solr/licenses/guice-3.0.jar.sha1 b/solr/licenses/guice-3.0.jar.sha1 deleted file mode 100644 index 7ce1a30309c..00000000000 --- a/solr/licenses/guice-3.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d84f15fe35e2c716a02979fb62f50a29f38aefa diff --git a/solr/licenses/guice-LICENSE-ASL.txt b/solr/licenses/guice-LICENSE-ASL.txt deleted file mode 100644 index 7a4a3ea2424..00000000000 --- a/solr/licenses/guice-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. \ No newline at end of file diff --git a/solr/licenses/guice-NOTICE.txt b/solr/licenses/guice-NOTICE.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/solr/licenses/guice-servlet-3.0.jar.sha1 b/solr/licenses/guice-servlet-3.0.jar.sha1 deleted file mode 100644 index a496feb6904..00000000000 --- a/solr/licenses/guice-servlet-3.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -610cde0e8da5a8b7d8efb8f0b8987466ffebaaf9 diff --git a/solr/licenses/guice-servlet-LICENSE-ASL.txt b/solr/licenses/guice-servlet-LICENSE-ASL.txt deleted file mode 100644 index 7a4a3ea2424..00000000000 --- a/solr/licenses/guice-servlet-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. \ No newline at end of file diff --git a/solr/licenses/guice-servlet-NOTICE.txt b/solr/licenses/guice-servlet-NOTICE.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/solr/licenses/hadoop-mapreduce-client-app-2.7.2.jar.sha1 b/solr/licenses/hadoop-mapreduce-client-app-2.7.2.jar.sha1 deleted file mode 100644 index 40efc353168..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-app-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6e4a1d1754c68b5ce3821cea96da77d0fc2067b5 diff --git a/solr/licenses/hadoop-mapreduce-client-app-LICENSE-ASL.txt b/solr/licenses/hadoop-mapreduce-client-app-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-app-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-app-NOTICE.txt b/solr/licenses/hadoop-mapreduce-client-app-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-app-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-common-2.7.2.jar.sha1 b/solr/licenses/hadoop-mapreduce-client-common-2.7.2.jar.sha1 deleted file mode 100644 index 87b5693bc3e..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-common-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8021f146b10053f29c2085d0972b115395da1a33 diff --git a/solr/licenses/hadoop-mapreduce-client-common-LICENSE-ASL.txt b/solr/licenses/hadoop-mapreduce-client-common-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-common-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-common-NOTICE.txt b/solr/licenses/hadoop-mapreduce-client-common-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-common-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-core-2.7.2.jar.sha1 b/solr/licenses/hadoop-mapreduce-client-core-2.7.2.jar.sha1 deleted file mode 100644 index adc7aca66b9..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-core-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -accf0f40de77a6d8cece5e3bd6bbc2a579102a4f diff --git a/solr/licenses/hadoop-mapreduce-client-core-LICENSE-ASL.txt b/solr/licenses/hadoop-mapreduce-client-core-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-core-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-core-NOTICE.txt b/solr/licenses/hadoop-mapreduce-client-core-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-core-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-hs-2.7.2.jar.sha1 b/solr/licenses/hadoop-mapreduce-client-hs-2.7.2.jar.sha1 deleted file mode 100644 index fedbe7f16cc..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-hs-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -57ccdecef8a4d9075e5f37dc254776e410cc6230 diff --git a/solr/licenses/hadoop-mapreduce-client-hs-LICENSE-ASL.txt b/solr/licenses/hadoop-mapreduce-client-hs-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-hs-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-hs-NOTICE.txt b/solr/licenses/hadoop-mapreduce-client-hs-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-hs-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-jobclient-2.7.2-tests.jar.sha1 b/solr/licenses/hadoop-mapreduce-client-jobclient-2.7.2-tests.jar.sha1 deleted file mode 100644 index bb4c7d1c898..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-jobclient-2.7.2-tests.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c686047090283e21bd1d4659e6eabb73d726a4fe diff --git a/solr/licenses/hadoop-mapreduce-client-jobclient-2.7.2.jar.sha1 b/solr/licenses/hadoop-mapreduce-client-jobclient-2.7.2.jar.sha1 deleted file mode 100644 index e9d8e630768..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-jobclient-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -cb2ca1e41075d55ad6dacf2f54cc0f0f97d623b6 diff --git a/solr/licenses/hadoop-mapreduce-client-jobclient-LICENSE-ASL.txt b/solr/licenses/hadoop-mapreduce-client-jobclient-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-jobclient-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-jobclient-NOTICE.txt b/solr/licenses/hadoop-mapreduce-client-jobclient-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-jobclient-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-shuffle-2.7.2.jar.sha1 b/solr/licenses/hadoop-mapreduce-client-shuffle-2.7.2.jar.sha1 deleted file mode 100644 index 3efabe6a156..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-shuffle-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -edaf659eaf16effa6c7f759c603ab3e2c0f976f3 diff --git a/solr/licenses/hadoop-mapreduce-client-shuffle-LICENSE-ASL.txt b/solr/licenses/hadoop-mapreduce-client-shuffle-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-shuffle-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-mapreduce-client-shuffle-NOTICE.txt b/solr/licenses/hadoop-mapreduce-client-shuffle-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-mapreduce-client-shuffle-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-api-2.7.2.jar.sha1 b/solr/licenses/hadoop-yarn-api-2.7.2.jar.sha1 deleted file mode 100644 index 4e5793d459a..00000000000 --- a/solr/licenses/hadoop-yarn-api-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -373d35254d9e292a36c35f48f8a797359b86e1f7 diff --git a/solr/licenses/hadoop-yarn-api-LICENSE-ASL.txt b/solr/licenses/hadoop-yarn-api-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-yarn-api-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-api-NOTICE.txt b/solr/licenses/hadoop-yarn-api-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-yarn-api-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-client-2.7.2.jar.sha1 b/solr/licenses/hadoop-yarn-client-2.7.2.jar.sha1 deleted file mode 100644 index 56af9379e87..00000000000 --- a/solr/licenses/hadoop-yarn-client-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6a626356170511385bac2ef4bd60eeea880dec46 diff --git a/solr/licenses/hadoop-yarn-client-LICENSE-ASL.txt b/solr/licenses/hadoop-yarn-client-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-yarn-client-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-client-NOTICE.txt b/solr/licenses/hadoop-yarn-client-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-yarn-client-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-common-2.7.2.jar.sha1 b/solr/licenses/hadoop-yarn-common-2.7.2.jar.sha1 deleted file mode 100644 index ea1109719e4..00000000000 --- a/solr/licenses/hadoop-yarn-common-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ab16767c5c34cfee6ac2780c7fea56207762d2c7 diff --git a/solr/licenses/hadoop-yarn-common-LICENSE-ASL.txt b/solr/licenses/hadoop-yarn-common-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-yarn-common-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-common-NOTICE.txt b/solr/licenses/hadoop-yarn-common-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-yarn-common-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-applicationhistoryservice-2.7.2.jar.sha1 b/solr/licenses/hadoop-yarn-server-applicationhistoryservice-2.7.2.jar.sha1 deleted file mode 100644 index 12bed4aed99..00000000000 --- a/solr/licenses/hadoop-yarn-server-applicationhistoryservice-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5ecdbd3bcc77826e4f4491ce12df722252f04f68 diff --git a/solr/licenses/hadoop-yarn-server-applicationhistoryservice-LICENSE-ASL.txt b/solr/licenses/hadoop-yarn-server-applicationhistoryservice-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-yarn-server-applicationhistoryservice-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-applicationhistoryservice-NOTICE.txt b/solr/licenses/hadoop-yarn-server-applicationhistoryservice-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-yarn-server-applicationhistoryservice-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-common-2.7.2.jar.sha1 b/solr/licenses/hadoop-yarn-server-common-2.7.2.jar.sha1 deleted file mode 100644 index ce515ea91eb..00000000000 --- a/solr/licenses/hadoop-yarn-server-common-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -514ebf83b773352e9e662a7b5011c8d947162c5f diff --git a/solr/licenses/hadoop-yarn-server-common-LICENSE-ASL.txt b/solr/licenses/hadoop-yarn-server-common-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-yarn-server-common-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-common-NOTICE.txt b/solr/licenses/hadoop-yarn-server-common-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-yarn-server-common-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-nodemanager-2.7.2.jar.sha1 b/solr/licenses/hadoop-yarn-server-nodemanager-2.7.2.jar.sha1 deleted file mode 100644 index 06b67d51cbc..00000000000 --- a/solr/licenses/hadoop-yarn-server-nodemanager-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -521e0fb188ef3c6b239eb6dcca69e39cc788690b diff --git a/solr/licenses/hadoop-yarn-server-nodemanager-LICENSE-ASL.txt b/solr/licenses/hadoop-yarn-server-nodemanager-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-yarn-server-nodemanager-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-nodemanager-NOTICE.txt b/solr/licenses/hadoop-yarn-server-nodemanager-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-yarn-server-nodemanager-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-resourcemanager-2.7.2.jar.sha1 b/solr/licenses/hadoop-yarn-server-resourcemanager-2.7.2.jar.sha1 deleted file mode 100644 index 9960f09d21d..00000000000 --- a/solr/licenses/hadoop-yarn-server-resourcemanager-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6444a32767712596c11767943ba10070a82b34da diff --git a/solr/licenses/hadoop-yarn-server-resourcemanager-LICENSE-ASL.txt b/solr/licenses/hadoop-yarn-server-resourcemanager-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-yarn-server-resourcemanager-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-resourcemanager-NOTICE.txt b/solr/licenses/hadoop-yarn-server-resourcemanager-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-yarn-server-resourcemanager-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-tests-2.7.2-tests.jar.sha1 b/solr/licenses/hadoop-yarn-server-tests-2.7.2-tests.jar.sha1 deleted file mode 100644 index 682b4bb1781..00000000000 --- a/solr/licenses/hadoop-yarn-server-tests-2.7.2-tests.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -60de24530087690d0e15444f2c2538bcd384c5cf diff --git a/solr/licenses/hadoop-yarn-server-tests-LICENSE-ASL.txt b/solr/licenses/hadoop-yarn-server-tests-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-yarn-server-tests-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-tests-NOTICE.txt b/solr/licenses/hadoop-yarn-server-tests-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-yarn-server-tests-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-web-proxy-2.7.2.jar.sha1 b/solr/licenses/hadoop-yarn-server-web-proxy-2.7.2.jar.sha1 deleted file mode 100644 index 488685bb97c..00000000000 --- a/solr/licenses/hadoop-yarn-server-web-proxy-2.7.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -430c3b53dcf1901f37a92d703bb4ce343298561b diff --git a/solr/licenses/hadoop-yarn-server-web-proxy-LICENSE-ASL.txt b/solr/licenses/hadoop-yarn-server-web-proxy-LICENSE-ASL.txt deleted file mode 100644 index 9a8e847ee84..00000000000 --- a/solr/licenses/hadoop-yarn-server-web-proxy-LICENSE-ASL.txt +++ /dev/null @@ -1,244 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -APACHE HADOOP SUBCOMPONENTS: - -The Apache Hadoop project contains subcomponents with separate copyright -notices and license terms. Your use of the source code for the these -subcomponents is subject to the terms and conditions of the following -licenses. - -For the org.apache.hadoop.util.bloom.* classes: - -/** - * - * Copyright (c) 2005, European Commission project OneLab under contract - * 034819 (http://www.one-lab.org) - * All rights reserved. - * Redistribution and use in source and binary forms, with or - * without modification, are permitted provided that the following - * conditions are met: - * - Redistributions of source code must retain the above copyright - * notice, this list of conditions and the following disclaimer. - * - Redistributions in binary form must reproduce the above copyright - * notice, this list of conditions and the following disclaimer in - * the documentation and/or other materials provided with the distribution. - * - Neither the name of the University Catholique de Louvain - UCL - * nor the names of its contributors may be used to endorse or - * promote products derived from this software without specific prior - * written permission. - * - * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS - * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT - * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS - * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE - * COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, - * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, - * BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; - * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER - * CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT - * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN - * ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE - * POSSIBILITY OF SUCH DAMAGE. - */ \ No newline at end of file diff --git a/solr/licenses/hadoop-yarn-server-web-proxy-NOTICE.txt b/solr/licenses/hadoop-yarn-server-web-proxy-NOTICE.txt deleted file mode 100644 index c56a5e4eac1..00000000000 --- a/solr/licenses/hadoop-yarn-server-web-proxy-NOTICE.txt +++ /dev/null @@ -1,2 +0,0 @@ -This product includes software developed by The Apache Software -Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/jackson-jaxrs-1.9.13.jar.sha1 b/solr/licenses/jackson-jaxrs-1.9.13.jar.sha1 deleted file mode 100644 index 3bffb8f68d0..00000000000 --- a/solr/licenses/jackson-jaxrs-1.9.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -534d72d2b9d6199dd531dfb27083dd4844082bba diff --git a/solr/licenses/jackson-jaxrs-LICENSE-ASL.txt b/solr/licenses/jackson-jaxrs-LICENSE-ASL.txt deleted file mode 100644 index 49ac3a7d45f..00000000000 --- a/solr/licenses/jackson-jaxrs-LICENSE-ASL.txt +++ /dev/null @@ -1,13 +0,0 @@ -This copy of Jackson JSON processor is licensed under the -Apache (Software) License, version 2.0 ("the License"). -See the License for details about distribution rights, and the -specific rights regarding derivate works. - -You may obtain a copy of the License at: - -http://www.apache.org/licenses/ - -A copy is also included with both the the downloadable source code package -and jar that contains class bytecodes, as file "ASL 2.0". In both cases, -that file should be located next to this file: in source distribution -the location should be "release-notes/asl"; and in jar "META-INF/" \ No newline at end of file diff --git a/solr/licenses/jackson-jaxrs-NOTICE.txt b/solr/licenses/jackson-jaxrs-NOTICE.txt deleted file mode 100644 index e9ebcc69fde..00000000000 --- a/solr/licenses/jackson-jaxrs-NOTICE.txt +++ /dev/null @@ -1,7 +0,0 @@ -This product currently only contains code developed by authors -of specific components, as identified by the source code files; -if such notes are missing files have been created by -Tatu Saloranta. - -For additional credits (generally to people who reported problems) -see CREDITS file. \ No newline at end of file diff --git a/solr/licenses/javax.inject-1.jar.sha1 b/solr/licenses/javax.inject-1.jar.sha1 deleted file mode 100644 index 7ef3c707b3c..00000000000 --- a/solr/licenses/javax.inject-1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6975da39a7040257bd51d21a231b76c915872d38 diff --git a/solr/licenses/javax.inject-LICENSE-ASL.txt b/solr/licenses/javax.inject-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/javax.inject-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/javax.inject-NOTICE.txt b/solr/licenses/javax.inject-NOTICE.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/solr/licenses/jaxb-impl-2.2.3-1.jar.sha1 b/solr/licenses/jaxb-impl-2.2.3-1.jar.sha1 deleted file mode 100644 index 79fe55d7736..00000000000 --- a/solr/licenses/jaxb-impl-2.2.3-1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -56baae106392040a45a06d4a41099173425da1e6 diff --git a/solr/licenses/jaxb-impl-LICENSE-CDDL.txt b/solr/licenses/jaxb-impl-LICENSE-CDDL.txt deleted file mode 100644 index d6e03ec15ce..00000000000 --- a/solr/licenses/jaxb-impl-LICENSE-CDDL.txt +++ /dev/null @@ -1,135 +0,0 @@ -COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.1 - -1. Definitions. - -1.1. "Contributor" means each individual or entity that creates or contributes to the creation of Modifications. - -1.2. "Contributor Version" means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. - -1.3. "Covered Software" means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. - -1.4. "Executable" means the Covered Software in any form other than Source Code. - -1.5. "Initial Developer" means the individual or entity that first makes Original Software available under this License. - -1.6. "Larger Work" means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. - -1.7. "License" means this document. - -1.8. "Licensable" means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. - -1.9. "Modifications" means the Source Code and Executable form of any of the following: - - A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; - - B. Any new file that contains any part of the Original Software or previous Modification; or - - C. Any new file that is contributed or otherwise made available under the terms of this License. - -1.10. "Original Software" means the Source Code and Executable form of computer software code that is originally released under this License. - -1.11. "Patent Claims" means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. - -1.12. "Source Code" means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. - -1.13. "You" (or "Your") means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, "You" includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, "control" means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. - -2. License Grants. - -2.1. The Initial Developer Grant. - -Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: - -(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and - -(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). - -(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. - -(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. - -2.2. Contributor Grant. - -Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: - -(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and - -(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). - -(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. - -(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. - -3. Distribution Obligations. - -3.1. Availability of Source Code. - -Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. - -3.2. Modifications. - -The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. - -3.3. Required Notices. - -You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. - -3.4. Application of Additional Terms. - -You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. - -3.5. Distribution of Executable Versions. - -You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. - -3.6. Larger Works. - -You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. - -4. Versions of the License. - -4.1. New Versions. - -Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. - -4.2. Effect of New Versions. - -You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. - -4.3. Modified Versions. - -When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. - -5. DISCLAIMER OF WARRANTY. - -COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. - -6. TERMINATION. - -6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. - -6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as "Participant") alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. - -6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. - -6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. - -7. LIMITATION OF LIABILITY. - -UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. - -8. U.S. GOVERNMENT END USERS. - -The Covered Software is a "commercial item," as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer software" (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and "commercial computer software documentation" as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. - -9. MISCELLANEOUS. - -This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. - -10. RESPONSIBILITY FOR CLAIMS. - -As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. - -NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) - -The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. \ No newline at end of file diff --git a/solr/licenses/jersey-bundle-1.9.jar.sha1 b/solr/licenses/jersey-bundle-1.9.jar.sha1 deleted file mode 100644 index a5946ca045b..00000000000 --- a/solr/licenses/jersey-bundle-1.9.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4dc55ffd69faa3cea5279f606909efe845a38e8f diff --git a/solr/licenses/jersey-bundle-LICENSE-CDDL.txt b/solr/licenses/jersey-bundle-LICENSE-CDDL.txt deleted file mode 100644 index 64df8d56300..00000000000 --- a/solr/licenses/jersey-bundle-LICENSE-CDDL.txt +++ /dev/null @@ -1,85 +0,0 @@ -COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1 - -1. Definitions. - -1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications. -1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. -1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. -1.4. “Executable” means the Covered Software in any form other than Source Code. -1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License. -1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. -1.7. “License” means this document. -1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. -1.9. “Modifications” means the Source Code and Executable form of any of the following: -A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; -B. Any new file that contains any part of the Original Software or previous Modification; or -C. Any new file that is contributed or otherwise made available under the terms of this License. -1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License. -1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. -1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. -1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. -2. License Grants. - -2.1. The Initial Developer Grant. -Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: -(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and -(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). -(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. -(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. -2.2. Contributor Grant. -Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: -(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and -(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). -(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. -(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. -3. Distribution Obligations. - -3.1. Availability of Source Code. -Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. -3.2. Modifications. -The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. -3.3. Required Notices. -You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. -3.4. Application of Additional Terms. -You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. -3.5. Distribution of Executable Versions. -You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. -3.6. Larger Works. -You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. -4. Versions of the License. - -4.1. New Versions. -Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. -4.2. Effect of New Versions. -You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. -4.3. Modified Versions. -When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. -5. DISCLAIMER OF WARRANTY. - -COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. - -6. TERMINATION. - -6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. -6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. -6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. -6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. -7. LIMITATION OF LIABILITY. - -UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. - -8. U.S. GOVERNMENT END USERS. - -The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. - -9. MISCELLANEOUS. - -This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. - -10. RESPONSIBILITY FOR CLAIMS. - -As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. - -NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) - -The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. \ No newline at end of file diff --git a/solr/licenses/jersey-guice-1.9.jar.sha1 b/solr/licenses/jersey-guice-1.9.jar.sha1 deleted file mode 100644 index b52c97e9f22..00000000000 --- a/solr/licenses/jersey-guice-1.9.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5963c28c47df7e5d6ad34cec80c071c368777f7b diff --git a/solr/licenses/jersey-guice-LICENSE-CDDL.txt b/solr/licenses/jersey-guice-LICENSE-CDDL.txt deleted file mode 100644 index 64df8d56300..00000000000 --- a/solr/licenses/jersey-guice-LICENSE-CDDL.txt +++ /dev/null @@ -1,85 +0,0 @@ -COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1 - -1. Definitions. - -1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications. -1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. -1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. -1.4. “Executable” means the Covered Software in any form other than Source Code. -1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License. -1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. -1.7. “License” means this document. -1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. -1.9. “Modifications” means the Source Code and Executable form of any of the following: -A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; -B. Any new file that contains any part of the Original Software or previous Modification; or -C. Any new file that is contributed or otherwise made available under the terms of this License. -1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License. -1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. -1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. -1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. -2. License Grants. - -2.1. The Initial Developer Grant. -Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: -(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and -(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). -(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. -(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. -2.2. Contributor Grant. -Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: -(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and -(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). -(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. -(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. -3. Distribution Obligations. - -3.1. Availability of Source Code. -Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. -3.2. Modifications. -The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. -3.3. Required Notices. -You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. -3.4. Application of Additional Terms. -You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. -3.5. Distribution of Executable Versions. -You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. -3.6. Larger Works. -You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. -4. Versions of the License. - -4.1. New Versions. -Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. -4.2. Effect of New Versions. -You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. -4.3. Modified Versions. -When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. -5. DISCLAIMER OF WARRANTY. - -COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. - -6. TERMINATION. - -6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. -6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. -6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. -6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. -7. LIMITATION OF LIABILITY. - -UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. - -8. U.S. GOVERNMENT END USERS. - -The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. - -9. MISCELLANEOUS. - -This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. - -10. RESPONSIBILITY FOR CLAIMS. - -As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. - -NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) - -The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. \ No newline at end of file diff --git a/solr/licenses/jersey-json-1.9.jar.sha1 b/solr/licenses/jersey-json-1.9.jar.sha1 deleted file mode 100644 index d6fe94a7091..00000000000 --- a/solr/licenses/jersey-json-1.9.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1aa73e1896bcc7013fed247157d7f676226eb432 diff --git a/solr/licenses/jersey-json-LICENSE-CDDL.txt b/solr/licenses/jersey-json-LICENSE-CDDL.txt deleted file mode 100644 index 64df8d56300..00000000000 --- a/solr/licenses/jersey-json-LICENSE-CDDL.txt +++ /dev/null @@ -1,85 +0,0 @@ -COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL)Version 1.1 - -1. Definitions. - -1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications. -1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. -1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. -1.4. “Executable” means the Covered Software in any form other than Source Code. -1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License. -1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. -1.7. “License” means this document. -1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. -1.9. “Modifications” means the Source Code and Executable form of any of the following: -A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; -B. Any new file that contains any part of the Original Software or previous Modification; or -C. Any new file that is contributed or otherwise made available under the terms of this License. -1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License. -1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. -1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. -1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. -2. License Grants. - -2.1. The Initial Developer Grant. -Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: -(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and -(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). -(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. -(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. -2.2. Contributor Grant. -Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: -(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and -(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). -(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. -(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. -3. Distribution Obligations. - -3.1. Availability of Source Code. -Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. -3.2. Modifications. -The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. -3.3. Required Notices. -You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. -3.4. Application of Additional Terms. -You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. -3.5. Distribution of Executable Versions. -You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. -3.6. Larger Works. -You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. -4. Versions of the License. - -4.1. New Versions. -Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. -4.2. Effect of New Versions. -You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. -4.3. Modified Versions. -When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. -5. DISCLAIMER OF WARRANTY. - -COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. - -6. TERMINATION. - -6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. -6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. -6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. -6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. -7. LIMITATION OF LIABILITY. - -UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. - -8. U.S. GOVERNMENT END USERS. - -The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. - -9. MISCELLANEOUS. - -This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. - -10. RESPONSIBILITY FOR CLAIMS. - -As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. - -NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) - -The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. \ No newline at end of file diff --git a/solr/licenses/kite-morphlines-avro-1.1.0.jar.sha1 b/solr/licenses/kite-morphlines-avro-1.1.0.jar.sha1 deleted file mode 100644 index fb596dddd0e..00000000000 --- a/solr/licenses/kite-morphlines-avro-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e097a18ce6ccf3fc208fc6194dbe64ee38bbcb4c diff --git a/solr/licenses/kite-morphlines-avro-LICENSE-ASL.txt b/solr/licenses/kite-morphlines-avro-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/kite-morphlines-avro-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/kite-morphlines-avro-NOTICE.txt b/solr/licenses/kite-morphlines-avro-NOTICE.txt deleted file mode 100644 index e66f59741cd..00000000000 --- a/solr/licenses/kite-morphlines-avro-NOTICE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This product includes software developed by Cloudera, Inc. -(http://www.cloudera.com/). - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by -Saxonica (http://www.saxonica.com/). \ No newline at end of file diff --git a/solr/licenses/kite-morphlines-core-1.1.0-tests.jar.sha1 b/solr/licenses/kite-morphlines-core-1.1.0-tests.jar.sha1 deleted file mode 100644 index 3c7baae875b..00000000000 --- a/solr/licenses/kite-morphlines-core-1.1.0-tests.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4110e16503a3bde44406ca9969055320b169c041 diff --git a/solr/licenses/kite-morphlines-core-1.1.0.jar.sha1 b/solr/licenses/kite-morphlines-core-1.1.0.jar.sha1 deleted file mode 100644 index b98e853ed11..00000000000 --- a/solr/licenses/kite-morphlines-core-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0a73f6cbbb0cde8dfd2c1924c7b66224f1b0ab64 diff --git a/solr/licenses/kite-morphlines-core-LICENSE-ASL.txt b/solr/licenses/kite-morphlines-core-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/kite-morphlines-core-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/kite-morphlines-core-NOTICE.txt b/solr/licenses/kite-morphlines-core-NOTICE.txt deleted file mode 100644 index e66f59741cd..00000000000 --- a/solr/licenses/kite-morphlines-core-NOTICE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This product includes software developed by Cloudera, Inc. -(http://www.cloudera.com/). - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by -Saxonica (http://www.saxonica.com/). \ No newline at end of file diff --git a/solr/licenses/kite-morphlines-hadoop-sequencefile-1.1.0.jar.sha1 b/solr/licenses/kite-morphlines-hadoop-sequencefile-1.1.0.jar.sha1 deleted file mode 100644 index d228894c0ad..00000000000 --- a/solr/licenses/kite-morphlines-hadoop-sequencefile-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -6f6d7c45138d571f2bf23be8c80c552f8d8a4851 diff --git a/solr/licenses/kite-morphlines-hadoop-sequencefile-LICENSE-ASL.txt b/solr/licenses/kite-morphlines-hadoop-sequencefile-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/kite-morphlines-hadoop-sequencefile-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/kite-morphlines-hadoop-sequencefile-NOTICE.txt b/solr/licenses/kite-morphlines-hadoop-sequencefile-NOTICE.txt deleted file mode 100644 index e66f59741cd..00000000000 --- a/solr/licenses/kite-morphlines-hadoop-sequencefile-NOTICE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This product includes software developed by Cloudera, Inc. -(http://www.cloudera.com/). - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by -Saxonica (http://www.saxonica.com/). \ No newline at end of file diff --git a/solr/licenses/kite-morphlines-json-1.1.0.jar.sha1 b/solr/licenses/kite-morphlines-json-1.1.0.jar.sha1 deleted file mode 100644 index 82db075ae89..00000000000 --- a/solr/licenses/kite-morphlines-json-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -33009c21ad90f18a8ddceca9680c83dc2006555d diff --git a/solr/licenses/kite-morphlines-json-LICENSE-ASL.txt b/solr/licenses/kite-morphlines-json-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/kite-morphlines-json-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/kite-morphlines-json-NOTICE.txt b/solr/licenses/kite-morphlines-json-NOTICE.txt deleted file mode 100644 index e66f59741cd..00000000000 --- a/solr/licenses/kite-morphlines-json-NOTICE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This product includes software developed by Cloudera, Inc. -(http://www.cloudera.com/). - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by -Saxonica (http://www.saxonica.com/). \ No newline at end of file diff --git a/solr/licenses/kite-morphlines-saxon-1.1.0.jar.sha1 b/solr/licenses/kite-morphlines-saxon-1.1.0.jar.sha1 deleted file mode 100644 index ec131b2dc81..00000000000 --- a/solr/licenses/kite-morphlines-saxon-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7cdb3f9d756906b60778f4eef7fcb57ddd3ef0fa diff --git a/solr/licenses/kite-morphlines-saxon-LICENSE-ASL.txt b/solr/licenses/kite-morphlines-saxon-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/kite-morphlines-saxon-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/kite-morphlines-saxon-NOTICE.txt b/solr/licenses/kite-morphlines-saxon-NOTICE.txt deleted file mode 100644 index e66f59741cd..00000000000 --- a/solr/licenses/kite-morphlines-saxon-NOTICE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This product includes software developed by Cloudera, Inc. -(http://www.cloudera.com/). - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by -Saxonica (http://www.saxonica.com/). \ No newline at end of file diff --git a/solr/licenses/kite-morphlines-tika-core-1.1.0.jar.sha1 b/solr/licenses/kite-morphlines-tika-core-1.1.0.jar.sha1 deleted file mode 100644 index ace76dde665..00000000000 --- a/solr/licenses/kite-morphlines-tika-core-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c9fb579f66f669529e4aa359270cc84ae720d4f8 diff --git a/solr/licenses/kite-morphlines-tika-core-LICENSE-ASL.txt b/solr/licenses/kite-morphlines-tika-core-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/kite-morphlines-tika-core-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/kite-morphlines-tika-core-NOTICE.txt b/solr/licenses/kite-morphlines-tika-core-NOTICE.txt deleted file mode 100644 index e66f59741cd..00000000000 --- a/solr/licenses/kite-morphlines-tika-core-NOTICE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This product includes software developed by Cloudera, Inc. -(http://www.cloudera.com/). - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by -Saxonica (http://www.saxonica.com/). \ No newline at end of file diff --git a/solr/licenses/kite-morphlines-tika-decompress-1.1.0.jar.sha1 b/solr/licenses/kite-morphlines-tika-decompress-1.1.0.jar.sha1 deleted file mode 100644 index 10872c115fe..00000000000 --- a/solr/licenses/kite-morphlines-tika-decompress-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3ae30cdeeef3978520e64289a49426aad815ef4b diff --git a/solr/licenses/kite-morphlines-tika-decompress-LICENSE-ASL.txt b/solr/licenses/kite-morphlines-tika-decompress-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/kite-morphlines-tika-decompress-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/kite-morphlines-tika-decompress-NOTICE.txt b/solr/licenses/kite-morphlines-tika-decompress-NOTICE.txt deleted file mode 100644 index e66f59741cd..00000000000 --- a/solr/licenses/kite-morphlines-tika-decompress-NOTICE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This product includes software developed by Cloudera, Inc. -(http://www.cloudera.com/). - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by -Saxonica (http://www.saxonica.com/). \ No newline at end of file diff --git a/solr/licenses/kite-morphlines-twitter-1.1.0.jar.sha1 b/solr/licenses/kite-morphlines-twitter-1.1.0.jar.sha1 deleted file mode 100644 index 421739d4df6..00000000000 --- a/solr/licenses/kite-morphlines-twitter-1.1.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1a26addfbb112f3426c4fce2e3f7c1f338a04f25 diff --git a/solr/licenses/kite-morphlines-twitter-LICENSE-ASL.txt b/solr/licenses/kite-morphlines-twitter-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/kite-morphlines-twitter-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/kite-morphlines-twitter-NOTICE.txt b/solr/licenses/kite-morphlines-twitter-NOTICE.txt deleted file mode 100644 index e66f59741cd..00000000000 --- a/solr/licenses/kite-morphlines-twitter-NOTICE.txt +++ /dev/null @@ -1,8 +0,0 @@ -This product includes software developed by Cloudera, Inc. -(http://www.cloudera.com/). - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This product includes software developed by -Saxonica (http://www.saxonica.com/). \ No newline at end of file diff --git a/solr/licenses/leveldb-0.7.jar.sha1 b/solr/licenses/leveldb-0.7.jar.sha1 deleted file mode 100644 index 1260de3da7a..00000000000 --- a/solr/licenses/leveldb-0.7.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -77df3093388129dea3b2b8758ddd3c7bf36bc22c diff --git a/solr/licenses/leveldb-LICENSE-BSD.txt b/solr/licenses/leveldb-LICENSE-BSD.txt deleted file mode 100644 index e6ffec62600..00000000000 --- a/solr/licenses/leveldb-LICENSE-BSD.txt +++ /dev/null @@ -1,27 +0,0 @@ -Copyright (c) 2011 The LevelDB Authors. All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are -met: - - * Redistributions of source code must retain the above copyright -notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above -copyright notice, this list of conditions and the following disclaimer -in the documentation and/or other materials provided with the -distribution. - * Neither the name of Google Inc. nor the names of its -contributors may be used to endorse or promote products derived from -this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/solr/licenses/leveldb-NOTICE.txt b/solr/licenses/leveldb-NOTICE.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/solr/licenses/leveldb-api-0.7.jar.sha1 b/solr/licenses/leveldb-api-0.7.jar.sha1 deleted file mode 100644 index 5d04d63dc3e..00000000000 --- a/solr/licenses/leveldb-api-0.7.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e620acc87a8162f668a5ea62cbe0561560c50044 diff --git a/solr/licenses/leveldb-api-LICENSE-BSD.txt b/solr/licenses/leveldb-api-LICENSE-BSD.txt deleted file mode 100644 index e6ffec62600..00000000000 --- a/solr/licenses/leveldb-api-LICENSE-BSD.txt +++ /dev/null @@ -1,27 +0,0 @@ -Copyright (c) 2011 The LevelDB Authors. All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are -met: - - * Redistributions of source code must retain the above copyright -notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above -copyright notice, this list of conditions and the following disclaimer -in the documentation and/or other materials provided with the -distribution. - * Neither the name of Google Inc. nor the names of its -contributors may be used to endorse or promote products derived from -this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/solr/licenses/leveldb-api-NOTICE.txt b/solr/licenses/leveldb-api-NOTICE.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/solr/licenses/leveldbjni-1.8.jar.sha1 b/solr/licenses/leveldbjni-1.8.jar.sha1 deleted file mode 100644 index 374f17b7c37..00000000000 --- a/solr/licenses/leveldbjni-1.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -80abdcbefef2d6ad8975b5f909e9412ab71b84c9 diff --git a/solr/licenses/leveldbjni-LICENSE-BSD.txt b/solr/licenses/leveldbjni-LICENSE-BSD.txt deleted file mode 100644 index e6ffec62600..00000000000 --- a/solr/licenses/leveldbjni-LICENSE-BSD.txt +++ /dev/null @@ -1,27 +0,0 @@ -Copyright (c) 2011 The LevelDB Authors. All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are -met: - - * Redistributions of source code must retain the above copyright -notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above -copyright notice, this list of conditions and the following disclaimer -in the documentation and/or other materials provided with the -distribution. - * Neither the name of Google Inc. nor the names of its -contributors may be used to endorse or promote products derived from -this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/solr/licenses/leveldbjni-NOTICE.txt b/solr/licenses/leveldbjni-NOTICE.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/solr/licenses/metrics-healthchecks-3.1.2.jar.sha1 b/solr/licenses/metrics-healthchecks-3.1.2.jar.sha1 deleted file mode 100644 index 946c8d8ed02..00000000000 --- a/solr/licenses/metrics-healthchecks-3.1.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e32a01aa7ca4070676e22e707272422baa0f7ecd diff --git a/solr/licenses/metrics-healthchecks-LICENSE-ASL.txt b/solr/licenses/metrics-healthchecks-LICENSE-ASL.txt deleted file mode 100644 index e4ba40426da..00000000000 --- a/solr/licenses/metrics-healthchecks-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright 2010-2012 Coda Hale and Yammer, Inc. - - Licensed 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. diff --git a/solr/licenses/metrics-healthchecks-NOTICE.txt b/solr/licenses/metrics-healthchecks-NOTICE.txt deleted file mode 100644 index 4fe83de38a5..00000000000 --- a/solr/licenses/metrics-healthchecks-NOTICE.txt +++ /dev/null @@ -1,11 +0,0 @@ -Metrics -Copyright 2010-2013 Coda Hale and Yammer, Inc. - -This product includes software developed by Coda Hale and Yammer, Inc. - -This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64, -LongAdder), which was released with the following comments: - - Written by Doug Lea with assistance from members of JCP JSR-166 - Expert Group and released to the public domain, as explained at - http://creativecommons.org/publicdomain/zero/1.0/ diff --git a/solr/licenses/mrunit-1.0.0-hadoop2.jar.sha1 b/solr/licenses/mrunit-1.0.0-hadoop2.jar.sha1 deleted file mode 100644 index 6146ee05008..00000000000 --- a/solr/licenses/mrunit-1.0.0-hadoop2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d6e4cce578b705508bfd7fd3fafbccc3adb33e83 diff --git a/solr/licenses/mrunit-LICENSE-ASL.txt b/solr/licenses/mrunit-LICENSE-ASL.txt deleted file mode 100644 index 75f307ab0ad..00000000000 --- a/solr/licenses/mrunit-LICENSE-ASL.txt +++ /dev/null @@ -1,479 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. - - -========================================================================== -The Apache License, Version 2.0 applies to the following libraries: -commons-logging - -========================================================================== -The following license applies to the junit library --------------------------------------------------------------------------- - -Common Public License Version 1.0 - -THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS COMMON PUBLIC -LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM -CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. - -1. DEFINITIONS - -"Contribution" means: - -a) in the case of the initial Contributor, the initial code and -documentation distributed under this Agreement, and - -b) in the case of each subsequent Contributor: - -i) changes to the Program, and - -ii) additions to the Program; - -where such changes and/or additions to the Program originate from and are -distributed by that particular Contributor. A Contribution 'originates' from a -Contributor if it was added to the Program by such Contributor itself or anyone -acting on such Contributor's behalf. Contributions do not include additions to -the Program which: (i) are separate modules of software distributed in -conjunction with the Program under their own license agreement, and (ii) are not -derivative works of the Program. - -"Contributor" means any person or entity that distributes the Program. - -"Licensed Patents " mean patent claims licensable by a Contributor which are -necessarily infringed by the use or sale of its Contribution alone or when -combined with the Program. - -"Program" means the Contributions distributed in accordance with this Agreement. - -"Recipient" means anyone who receives the Program under this Agreement, -including all Contributors. - -2. GRANT OF RIGHTS - -a) Subject to the terms of this Agreement, each Contributor hereby grants -Recipient a non-exclusive, worldwide, royalty-free copyright license to -reproduce, prepare derivative works of, publicly display, publicly perform, -distribute and sublicense the Contribution of such Contributor, if any, and such -derivative works, in source code and object code form. - -b) Subject to the terms of this Agreement, each Contributor hereby grants -Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed -Patents to make, use, sell, offer to sell, import and otherwise transfer the -Contribution of such Contributor, if any, in source code and object code form. -This patent license shall apply to the combination of the Contribution and the -Program if, at the time the Contribution is added by the Contributor, such -addition of the Contribution causes such combination to be covered by the -Licensed Patents. The patent license shall not apply to any other combinations -which include the Contribution. No hardware per se is licensed hereunder. - -c) Recipient understands that although each Contributor grants the licenses -to its Contributions set forth herein, no assurances are provided by any -Contributor that the Program does not infringe the patent or other intellectual -property rights of any other entity. Each Contributor disclaims any liability to -Recipient for claims brought by any other entity based on infringement of -intellectual property rights or otherwise. As a condition to exercising the -rights and licenses granted hereunder, each Recipient hereby assumes sole -responsibility to secure any other intellectual property rights needed, if any. -For example, if a third party patent license is required to allow Recipient to -distribute the Program, it is Recipient's responsibility to acquire that license -before distributing the Program. - -d) Each Contributor represents that to its knowledge it has sufficient -copyright rights in its Contribution, if any, to grant the copyright license set -forth in this Agreement. - -3. REQUIREMENTS - -A Contributor may choose to distribute the Program in object code form under its -own license agreement, provided that: - -a) it complies with the terms and conditions of this Agreement; and - -b) its license agreement: - -i) effectively disclaims on behalf of all Contributors all warranties and -conditions, express and implied, including warranties or conditions of title and -non-infringement, and implied warranties or conditions of merchantability and -fitness for a particular purpose; - -ii) effectively excludes on behalf of all Contributors all liability for -damages, including direct, indirect, special, incidental and consequential -damages, such as lost profits; - -iii) states that any provisions which differ from this Agreement are offered -by that Contributor alone and not by any other party; and - -iv) states that source code for the Program is available from such -Contributor, and informs licensees how to obtain it in a reasonable manner on or -through a medium customarily used for software exchange. - -When the Program is made available in source code form: - -a) it must be made available under this Agreement; and - -b) a copy of this Agreement must be included with each copy of the Program. - -Contributors may not remove or alter any copyright notices contained within the -Program. - -Each Contributor must identify itself as the originator of its Contribution, if -any, in a manner that reasonably allows subsequent Recipients to identify the -originator of the Contribution. - -4. COMMERCIAL DISTRIBUTION - -Commercial distributors of software may accept certain responsibilities with -respect to end users, business partners and the like. While this license is -intended to facilitate the commercial use of the Program, the Contributor who -includes the Program in a commercial product offering should do so in a manner -which does not create potential liability for other Contributors. Therefore, if -a Contributor includes the Program in a commercial product offering, such -Contributor ("Commercial Contributor") hereby agrees to defend and indemnify -every other Contributor ("Indemnified Contributor") against any losses, damages -and costs (collectively "Losses") arising from claims, lawsuits and other legal -actions brought by a third party against the Indemnified Contributor to the -extent caused by the acts or omissions of such Commercial Contributor in -connection with its distribution of the Program in a commercial product -offering. The obligations in this section do not apply to any claims or Losses -relating to any actual or alleged intellectual property infringement. In order -to qualify, an Indemnified Contributor must: a) promptly notify the Commercial -Contributor in writing of such claim, and b) allow the Commercial Contributor to -control, and cooperate with the Commercial Contributor in, the defense and any -related settlement negotiations. The Indemnified Contributor may participate in -any such claim at its own expense. - -For example, a Contributor might include the Program in a commercial product -offering, Product X. That Contributor is then a Commercial Contributor. If that -Commercial Contributor then makes performance claims, or offers warranties -related to Product X, those performance claims and warranties are such -Commercial Contributor's responsibility alone. Under this section, the -Commercial Contributor would have to defend claims against the other -Contributors related to those performance claims and warranties, and if a court -requires any other Contributor to pay any damages as a result, the Commercial -Contributor must pay those damages. - -5. NO WARRANTY - -EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN -"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR -IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, -NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each -Recipient is solely responsible for determining the appropriateness of using and -distributing the Program and assumes all risks associated with its exercise of -rights under this Agreement, including but not limited to the risks and costs of -program errors, compliance with applicable laws, damage to or loss of data, -programs or equipment, and unavailability or interruption of operations. - -6. DISCLAIMER OF LIABILITY - -EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY -CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST -PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, -STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY -OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS -GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. - -7. GENERAL - -If any provision of this Agreement is invalid or unenforceable under applicable -law, it shall not affect the validity or enforceability of the remainder of the -terms of this Agreement, and without further action by the parties hereto, such -provision shall be reformed to the minimum extent necessary to make such -provision valid and enforceable. - -If Recipient institutes patent litigation against a Contributor with respect to -a patent applicable to software (including a cross-claim or counterclaim in a -lawsuit), then any patent licenses granted by that Contributor to such Recipient -under this Agreement shall terminate as of the date such litigation is filed. In -addition, if Recipient institutes patent litigation against any entity -(including a cross-claim or counterclaim in a lawsuit) alleging that the Program -itself (excluding combinations of the Program with other software or hardware) -infringes such Recipient's patent(s), then such Recipient's rights granted under -Section 2(b) shall terminate as of the date such litigation is filed. - -All Recipient's rights under this Agreement shall terminate if it fails to -comply with any of the material terms or conditions of this Agreement and does -not cure such failure in a reasonable period of time after becoming aware of -such noncompliance. If all Recipient's rights under this Agreement terminate, -Recipient agrees to cease use and distribution of the Program as soon as -reasonably practicable. However, Recipient's obligations under this Agreement -and any licenses granted by Recipient relating to the Program shall continue and -survive. - -Everyone is permitted to copy and distribute copies of this Agreement, but in -order to avoid inconsistency the Agreement is copyrighted and may only be -modified in the following manner. The Agreement Steward reserves the right to -publish new versions (including revisions) of this Agreement from time to time. -No one other than the Agreement Steward has the right to modify this Agreement. -IBM is the initial Agreement Steward. IBM may assign the responsibility to serve -as the Agreement Steward to a suitable separate entity. Each new version of the -Agreement will be given a distinguishing version number. The Program (including -Contributions) may always be distributed subject to the version of the Agreement -under which it was received. In addition, after a new version of the Agreement -is published, Contributor may elect to distribute the Program (including its -Contributions) under the new version. Except as expressly stated in Sections -2(a) and 2(b) above, Recipient receives no rights or licenses to the -intellectual property of any Contributor under this Agreement, whether -expressly, by implication, estoppel or otherwise. All rights in the Program not -expressly granted under this Agreement are reserved. - -This Agreement is governed by the laws of the State of New York and the -intellectual property laws of the United States of America. No party to this -Agreement will bring a legal action under this Agreement more than one year -after the cause of action arose. Each party waives its rights to a jury trial in -any resulting litigation. - -========================================================================== -The following license applies to the mockito library --------------------------------------------------------------------------- - -The MIT License - -Copyright (c) 2007 Mockito contributors - -Permission is hereby granted, free of charge, to any person obtaining a copy of -this software and associated documentation files (the "Software"), to deal in -the Software without restriction, including without limitation the rights to -use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of -the Software, and to permit persons to whom the Software is furnished to do so, -subject to the following conditions: - -The above copyright notice and this permission notice shall be included in all -copies or substantial portions of the Software. - -THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS -FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR -COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER -IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN -CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - -========================================================================== -The following license applies to the hamcrest library --------------------------------------------------------------------------- - -The BSD 2-Clause License - -Copyright (c) 2000-2006, www.hamcrest.org -All rights reserved. - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - -- Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. -- Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON -ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/solr/licenses/mrunit-NOTICE.txt b/solr/licenses/mrunit-NOTICE.txt deleted file mode 100644 index 2dfba62fbfc..00000000000 --- a/solr/licenses/mrunit-NOTICE.txt +++ /dev/null @@ -1,5 +0,0 @@ -Apache MRUnit -Copyright 2011-2012 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). \ No newline at end of file diff --git a/solr/licenses/netty-3.2.4.Final.jar.sha1 b/solr/licenses/netty-3.2.4.Final.jar.sha1 deleted file mode 100644 index 7546dd38c4e..00000000000 --- a/solr/licenses/netty-3.2.4.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d0112ab5206185ca5701f88d25c84927bf53dde1 diff --git a/solr/licenses/netty-LICENSE-ASL.txt b/solr/licenses/netty-LICENSE-ASL.txt deleted file mode 100644 index d6456956733..00000000000 --- a/solr/licenses/netty-LICENSE-ASL.txt +++ /dev/null @@ -1,202 +0,0 @@ - - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/netty-NOTICE.txt b/solr/licenses/netty-NOTICE.txt deleted file mode 100644 index dba84b13911..00000000000 --- a/solr/licenses/netty-NOTICE.txt +++ /dev/null @@ -1,98 +0,0 @@ - - The Netty Project - ================= - -Please visit the Netty web site for more information: - - * http://www.jboss.org/netty/ - -Copyright 2009 Red Hat, Inc. - -Red Hat licenses this product to you under the Apache License, version 2.0 (the -"License"); you may not use this product 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. - -Also, please refer to each LICENSE..txt file, which is located in -the 'license' directory of the distribution file, for the license terms of the -components that this product depends on. - -------------------------------------------------------------------------------- -This product contains the extensions to Java Collections Framework which has -been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: - - * LICENSE: - * license/LICENSE.jsr166y.txt (Public Domain) - * HOMEPAGE: - * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ - * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ - -This product contains a modified version of Robert Harder's Public Domain -Base64 Encoder and Decoder, which can be obtained at: - - * LICENSE: - * license/LICENSE.base64.txt (Public Domain) - * HOMEPAGE: - * http://iharder.sourceforge.net/current/java/base64/ - -This product contains a modified version of 'JZlib', a re-implementation of -zlib in pure Java, which can be obtained at: - - * LICENSE: - * license/LICENSE.jzlib.txt (BSD Style License) - * HOMEPAGE: - * http://www.jcraft.com/jzlib/ - -This product optionally depends on 'Protocol Buffers', Google's data -interchange format, which can be obtained at: - - * LICENSE: - * license/LICENSE.protobuf.txt (New BSD License) - * HOMEPAGE: - * http://code.google.com/p/protobuf/ - -This product optionally depends on 'SLF4J', a simple logging facade for Java, -which can be obtained at: - - * LICENSE: - * license/LICENSE.slf4j.txt (MIT License) - * HOMEPAGE: - * http://www.slf4j.org/ - -This product optionally depends on 'Apache Commons Logging', a logging -framework, which can be obtained at: - - * LICENSE: - * license/LICENSE.commons-logging.txt (Apache License 2.0) - * HOMEPAGE: - * http://commons.apache.org/logging/ - -This product optionally depends on 'Apache Log4J', a logging framework, -which can be obtained at: - - * LICENSE: - * license/LICENSE.log4j.txt (Apache License 2.0) - * HOMEPAGE: - * http://logging.apache.org/log4j/ - -This product optionally depends on 'JBoss Logging', a logging framework, -which can be obtained at: - - * LICENSE: - * license/LICENSE.jboss-logging.txt (GNU LGPL 2.1) - * HOMEPAGE: - * http://anonsvn.jboss.org/repos/common/common-logging-spi/ - -This product optionally depends on 'Apache Felix', an open source OSGi -framework implementation, which can be obtained at: - - * LICENSE: - * license/LICENSE.felix.txt (Apache License 2.0) - * HOMEPAGE: - * http://felix.apache.org/ \ No newline at end of file diff --git a/solr/licenses/paranamer-2.3.jar.sha1 b/solr/licenses/paranamer-2.3.jar.sha1 deleted file mode 100644 index 21c0b2636d3..00000000000 --- a/solr/licenses/paranamer-2.3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4a85963a752c0a2f715c3924bfc686865e7e1bc6 diff --git a/solr/licenses/paranamer-LICENSE-BSD.txt b/solr/licenses/paranamer-LICENSE-BSD.txt deleted file mode 100644 index fca18473ba0..00000000000 --- a/solr/licenses/paranamer-LICENSE-BSD.txt +++ /dev/null @@ -1,28 +0,0 @@ -[ ParaNamer used to be 'Pubic Domain', but since it includes a small piece of ASM it is now the same license as that: BSD ] - - Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions - are met: - 1. Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - 2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - 3. Neither the name of the copyright holders nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF - THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/solr/licenses/paranamer-NOTICE.txt b/solr/licenses/paranamer-NOTICE.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/solr/licenses/rome-1.6.1.jar.sha1 b/solr/licenses/rome-1.6.1.jar.sha1 deleted file mode 100644 index 4b54e7c9f04..00000000000 --- a/solr/licenses/rome-1.6.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -fb844f8d9b7c1e324d150ca6ebb791cfd9b33243 diff --git a/solr/licenses/snappy-java-1.0.5.jar.sha1 b/solr/licenses/snappy-java-1.0.5.jar.sha1 deleted file mode 100644 index 68bfb2e5026..00000000000 --- a/solr/licenses/snappy-java-1.0.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -10cb4550360a0ec6b80f09a5209d00b6058e82bf diff --git a/solr/licenses/snappy-java-LICENSE-ASL.txt b/solr/licenses/snappy-java-LICENSE-ASL.txt deleted file mode 100644 index 261eeb9e9f8..00000000000 --- a/solr/licenses/snappy-java-LICENSE-ASL.txt +++ /dev/null @@ -1,201 +0,0 @@ - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed 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. diff --git a/solr/licenses/snappy-java-NOTICE.txt b/solr/licenses/snappy-java-NOTICE.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/solr/server/scripts/map-reduce/set-map-reduce-classpath.sh b/solr/server/scripts/map-reduce/set-map-reduce-classpath.sh deleted file mode 100755 index 0d22f63f801..00000000000 --- a/solr/server/scripts/map-reduce/set-map-reduce-classpath.sh +++ /dev/null @@ -1,52 +0,0 @@ -#!/usr/bin/env bash - -###################################################################### -# -# Running this script will set two environment variables: -# HADOOP_CLASSPATH -# HADOOP_LIBJAR: pass this to the -libjar MapReduceIndexBuilder option -# -###################################################################### - -# return absolute path -function absPath { - echo $(cd $(dirname "$1"); pwd)/$(basename "$1") -} - - -# Find location of this script - -sdir="`cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd`" - -solr_distrib="$sdir/../../.." - -echo `absPath $solr_distrib` - -# Setup env variables for MapReduceIndexerTool - -# Setup HADOOP_CLASSPATH - -dir1=`absPath "$solr_distrib/dist"` -dir2=`absPath "$solr_distrib/dist/solrj-lib"` -dir3=`absPath "$solr_distrib/contrib/map-reduce/lib"` -dir4=`absPath "$solr_distrib/contrib/morphlines-core/lib"` -dir5=`absPath "$solr_distrib/contrib/morphlines-cell/lib"` -dir6=`absPath "$solr_distrib/contrib/extraction/lib"` -dir7=`absPath "$solr_distrib/server/solr-webapp/webapp/WEB-INF/lib"` - -# Setup -libjar - -lib1=`ls -m $dir1/*.jar | tr -d ' \n'` -lib2=`ls -m $dir2/*.jar | tr -d ' \n' | sed 's/\,[^\,]*\(log4j\|slf4j\)[^\,]*//g'` -lib3=`ls -m $dir3/*.jar | tr -d ' \n'` -lib4=`ls -m $dir4/*.jar | tr -d ' \n'` -lib5=`ls -m $dir5/*.jar | tr -d ' \n'` -lib6=`ls -m $dir6/*.jar | tr -d ' \n'` -lib7=`ls -m $dir7/*.jar | tr -d ' \n'` - -export HADOOP_CLASSPATH="$dir1/*:$dir2/*:$dir3/*:$dir4/*:$dir5/*:$dir6/*:$dir7/*" -export HADOOP_LIBJAR="$lib1,$lib2,$lib3,$lib4,$lib5,$lib6,$lib7" - -#echo $HADOOP_CLASSPATH -#echo $HADOOP_LIBJAR - From fec87fceb5de0c167969ee03fcc09d58d5d479c9 Mon Sep 17 00:00:00 2001 From: yonik Date: Fri, 24 Mar 2017 20:43:44 -0400 Subject: [PATCH 057/563] SOLR-7452: add support for _m buckets, missing and has sub-facets in need of refinement --- .../search/facet/FacetFieldProcessor.java | 11 ++++ .../search/facet/TestJsonFacetRefinement.java | 58 ++++++++++++++++--- 2 files changed, 60 insertions(+), 9 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java index e8b234d6ed0..97d8607aaf4 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java @@ -545,6 +545,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { for (Object bucketVal : leaves) { bucketList.add( refineBucket(bucketVal, false, null) ); } + for (List bucketAndFacetInfo : skip) { assert bucketAndFacetInfo.size() == 2; Object bucketVal = bucketAndFacetInfo.get(0); @@ -553,6 +554,16 @@ abstract class FacetFieldProcessor extends FacetProcessor { bucketList.add( refineBucket(bucketVal, true, facetInfo ) ); } + // The only difference between skip and missing is the value of "skip" passed to refineBucket + for (List bucketAndFacetInfo : missing) { + assert bucketAndFacetInfo.size() == 2; + Object bucketVal = bucketAndFacetInfo.get(0); + Map facetInfo = (Map) bucketAndFacetInfo.get(1); + + bucketList.add( refineBucket(bucketVal, false, facetInfo ) ); + } + + // If there are just a couple of leaves, and if the domain is large, then // going by term is likely the most efficient? // If the domain is small, or if the number of leaves is large, then doing diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java index 869c90bf1da..5c863471d21 100644 --- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java +++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java @@ -209,6 +209,17 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "}" ); + // for testing missing _m, we need a partial facet within a partial facet + doTestRefine("{top:{type:terms, field:Afield, refine:true, limit:1, facet:{x : {type:terms, field:X, limit:1, refine:true} } } }", + "{top: {buckets:[{val:'A', count:2, x:{buckets:[{val:x1, count:5},{val:x2, count:3}]} } ] } }", + "{top: {buckets:[{val:'B', count:1, x:{buckets:[{val:x2, count:4},{val:x3, count:2}]} } ] } }", + null, + "=={top: {" + + "_m:[ ['A' , {x:{_l:[x1]}} ] ]" + + " } " + + "}" + ); + } @@ -223,20 +234,21 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { client.deleteByQuery("*:*", null); - ModifiableSolrParams p = params("cat_s", "cat_s", "num_d", "num_d"); + ModifiableSolrParams p = params("cat_s", "cat_s", "xy_s", "xy_s", "num_d", "num_d"); String cat_s = p.get("cat_s"); + String xy_s = p.get("xy_s"); String num_d = p.get("num_d"); - clients.get(0).add( sdoc("id", "01", "all_s","all", cat_s, "A", num_d, -1) ); // A wins count tie - clients.get(0).add( sdoc("id", "02", "all_s","all", cat_s, "B", num_d, 3) ); + clients.get(0).add( sdoc("id", "01", "all_s","all", cat_s, "A", xy_s, "X" ,num_d, -1) ); // A wins count tie + clients.get(0).add( sdoc("id", "02", "all_s","all", cat_s, "B", xy_s, "Y", num_d, 3) ); - clients.get(1).add( sdoc("id", "11", "all_s","all", cat_s, "B", num_d, -5) ); // B highest count - clients.get(1).add( sdoc("id", "12", "all_s","all", cat_s, "B", num_d, -11) ); - clients.get(1).add( sdoc("id", "13", "all_s","all", cat_s, "A", num_d, 7) ); + clients.get(1).add( sdoc("id", "11", "all_s","all", cat_s, "B", xy_s, "X", num_d, -5) ); // B highest count + clients.get(1).add( sdoc("id", "12", "all_s","all", cat_s, "B", xy_s, "Y", num_d, -11) ); + clients.get(1).add( sdoc("id", "13", "all_s","all", cat_s, "A", xy_s, "X", num_d, 7) ); - clients.get(2).add( sdoc("id", "21", "all_s","all", cat_s, "A", num_d, 17) ); // A highest count - clients.get(2).add( sdoc("id", "22", "all_s","all", cat_s, "A", num_d, -19) ); - clients.get(2).add( sdoc("id", "23", "all_s","all", cat_s, "B", num_d, 11) ); + clients.get(2).add( sdoc("id", "21", "all_s","all", cat_s, "A", xy_s, "X", num_d, 17) ); // A highest count + clients.get(2).add( sdoc("id", "22", "all_s","all", cat_s, "A", xy_s, "Y", num_d, -19) ); + clients.get(2).add( sdoc("id", "23", "all_s","all", cat_s, "B", xy_s, "X", num_d, 11) ); client.commit(); @@ -255,6 +267,7 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { ); ****/ + client.testJQ(params(p, "q", "*:*", "json.facet", "{" + "cat0:{type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:false}" + @@ -305,6 +318,33 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "}" ); + // test missing buckets (field facet within field facet) + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "ab:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true, facet:{ xy:{type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true } }}" + + "}" + ) + , "facets=={ count:8" + + ", ab:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} }] }" + // just like the previous response, just nested under a field facet + "}" + ); + + // test that sibling facets and stats are included for _m buckets, but skipped for _s buckets + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + " ab :{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true, facet:{ xy:{type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + + ",ab2:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{ xy:{type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + // top level refine=false shouldn't matter + ",allf :{type:terms, field:all_s, limit:1, overrequest:0, refine:true, facet:{cat:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + + ",allf2:{type:terms, field:all_s, limit:1, overrequest:0, refine:false, facet:{cat:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + // top level refine=false shouldn't matter + "}" + ) + , "facets=={ count:8" + + ", ab:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _m buckets + ", allf:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }] }" + // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets + ", ab2:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _m buckets + ", allf2:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }] }" + // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets + "}" + ); } From 8664f1f38a3741a7aff5988221cb7a1a7dda9e5b Mon Sep 17 00:00:00 2001 From: David Smiley Date: Fri, 24 Mar 2017 22:46:24 -0400 Subject: [PATCH 058/563] SOLR-10249: Refactor IndexFetcher to return detailed result --- solr/CHANGES.txt | 2 + .../apache/solr/cloud/RecoveryStrategy.java | 2 +- .../solr/handler/CdcrRequestHandler.java | 2 +- .../org/apache/solr/handler/IndexFetcher.java | 81 ++++++++++++++++--- .../solr/handler/ReplicationHandler.java | 10 +-- 5 files changed, 77 insertions(+), 20 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 428bde46afa..f2937bfcced 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -138,6 +138,8 @@ Other Changes * SOLR-9221: Remove Solr contribs: map-reduce, morphlines-core and morphlines-cell. (Steve Rowe) +* SOLR-10249: Refactor IndexFetcher.doFetch() to return a more detailed result. (Jeff Miller via David Smiley) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. 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 cb6c69c8aa9..2cbc3945607 100644 --- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java +++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java @@ -219,7 +219,7 @@ public class RecoveryStrategy extends Thread implements Closeable { solrParams.set(ReplicationHandler.MASTER_URL, leaderUrl); if (isClosed()) return; // we check closed on return - boolean success = replicationHandler.doFetch(solrParams, false); + boolean success = replicationHandler.doFetch(solrParams, false).getSuccessful(); if (!success) { throw new SolrException(ErrorCode.SERVER_ERROR, "Replication for recovery failed."); diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java index fcc4bbef4e0..ba174f95256 100644 --- a/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/CdcrRequestHandler.java @@ -754,7 +754,7 @@ public class CdcrRequestHandler extends RequestHandlerBase implements SolrCoreAw // we do not want the raw tlog files from the source solrParams.set(ReplicationHandler.TLOG_FILES, false); - success = replicationHandler.doFetch(solrParams, false); + success = replicationHandler.doFetch(solrParams, false).getSuccessful(); // this is required because this callable can race with HttpSolrCall#destroy // which clears the request info. 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 33e80913e68..5efb6c5fe58 100644 --- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java +++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java @@ -104,6 +104,8 @@ import static org.apache.solr.common.params.CommonParams.JAVABIN; import static org.apache.solr.common.params.CommonParams.NAME; import static org.apache.solr.handler.ReplicationHandler.*; +import com.google.common.base.Strings; + /** *

    Provides functionality of downloading changed index files as well as config files and a timer for scheduling fetches from the * master.

    @@ -161,6 +163,52 @@ public class IndexFetcher { private Integer soTimeout; + private static final String INTERRUPT_RESPONSE_MESSAGE = "Interrupted while waiting for modify lock"; + + public static class IndexFetchResult { + private final String message; + private final boolean successful; + private final Throwable exception; + + public static final String FAILED_BY_INTERRUPT_MESSAGE = "Fetching index failed by interrupt"; + public static final String FAILED_BY_EXCEPTION_MESSAGE = "Fetching index failed by exception"; + + /** pre-defined results */ + public static final IndexFetchResult ALREADY_IN_SYNC = new IndexFetchResult("Local index commit is already in sync with peer", true, null); + public static final IndexFetchResult INDEX_FETCH_FAILURE = new IndexFetchResult("Fetching lastest index is failed", false, null); + public static final IndexFetchResult INDEX_FETCH_SUCCESS = new IndexFetchResult("Fetching latest index is successful", true, null); + public static final IndexFetchResult LOCK_OBTAIN_FAILED = new IndexFetchResult("Obtaining SnapPuller lock failed", false, null); + public static final IndexFetchResult MASTER_VERSION_ZERO = new IndexFetchResult("Index in peer is empty and never committed yet", true, null); + public static final IndexFetchResult NO_INDEX_COMMIT_EXIST = new IndexFetchResult("No IndexCommit in local index", false, null); + public static final IndexFetchResult PEER_INDEX_COMMIT_DELETED = new IndexFetchResult("No files to download because IndexCommit in peer was deleted", false, null); + public static final IndexFetchResult LOCAL_ACTIVITY_DURING_REPLICATION = new IndexFetchResult("Local index modification during replication", false, null); + public static final IndexFetchResult EXPECTING_NON_LEADER = new IndexFetchResult("Replicating from leader but I'm the shard leader", false, null); + + IndexFetchResult(String message, boolean successful, Throwable exception) { + this.message = message; + this.successful = successful; + this.exception = exception; + } + + /* + * @return exception thrown if failed by exception or interrupt, otherwise null + */ + public Throwable getException() { + return this.exception; + } + + /* + * @return true if index fetch was successful, false otherwise + */ + public boolean getSuccessful() { + return this.successful; + } + + public String getMessage() { + return this.message; + } + } + private static HttpClient createHttpClient(SolrCore core, String httpBasicAuthUser, String httpBasicAuthPassword, boolean useCompression) { final ModifiableSolrParams httpClientParams = new ModifiableSolrParams(); httpClientParams.set(HttpClientUtil.PROP_BASIC_AUTH_USER, httpBasicAuthUser); @@ -274,7 +322,7 @@ public class IndexFetcher { } } - boolean fetchLatestIndex(boolean forceReplication) throws IOException, InterruptedException { + IndexFetchResult fetchLatestIndex(boolean forceReplication) throws IOException, InterruptedException { return fetchLatestIndex(forceReplication, false); } @@ -287,7 +335,7 @@ public class IndexFetcher { * @return true on success, false if slave is already in sync * @throws IOException if an exception occurs */ - boolean fetchLatestIndex(boolean forceReplication, boolean forceCoreReload) throws IOException, InterruptedException { + IndexFetchResult fetchLatestIndex(boolean forceReplication, boolean forceCoreReload) throws IOException, InterruptedException { boolean cleanupDone = false; boolean successfulInstall = false; @@ -311,7 +359,7 @@ public class IndexFetcher { Replica replica = getLeaderReplica(); CloudDescriptor cd = solrCore.getCoreDescriptor().getCloudDescriptor(); if (cd.getCoreNodeName().equals(replica.getName())) { - return false; + return IndexFetchResult.EXPECTING_NON_LEADER; } masterUrl = replica.getCoreUrl(); LOG.info("Updated masterUrl to " + masterUrl); @@ -321,9 +369,16 @@ public class IndexFetcher { try { response = getLatestVersion(); } catch (Exception e) { - LOG.error("Master at: " + masterUrl + " is not available. Index fetch failed. Exception: " + e.getMessage()); - return false; - } + final String errorMsg = e.toString(); + if (!Strings.isNullOrEmpty(errorMsg) && errorMsg.contains(INTERRUPT_RESPONSE_MESSAGE)) { + LOG.warn("Master at: " + masterUrl + " is not available. Index fetch failed by interrupt. Exception: " + errorMsg); + return new IndexFetchResult(IndexFetchResult.FAILED_BY_INTERRUPT_MESSAGE, false, e); + } else { + LOG.warn("Master at: " + masterUrl + " is not available. Index fetch failed by exception: " + errorMsg); + return new IndexFetchResult(IndexFetchResult.FAILED_BY_EXCEPTION_MESSAGE, false, e); + } + } + long latestVersion = (Long) response.get(CMD_INDEX_VERSION); long latestGeneration = (Long) response.get(GENERATION); @@ -339,7 +394,7 @@ public class IndexFetcher { searcherRefCounted = solrCore.getNewestSearcher(false); if (searcherRefCounted == null) { LOG.warn("No open searcher found - fetch aborted"); - return false; + return IndexFetchResult.NO_INDEX_COMMIT_EXIST; } commit = searcherRefCounted.get().getIndexReader().getIndexCommit(); } finally { @@ -367,7 +422,7 @@ public class IndexFetcher { //there is nothing to be replicated successfulInstall = true; - return true; + return IndexFetchResult.MASTER_VERSION_ZERO; } // TODO: Should we be comparing timestamps (across machines) here? @@ -375,14 +430,14 @@ public class IndexFetcher { //master and slave are already in sync just return LOG.info("Slave in sync with master."); successfulInstall = true; - return true; + return IndexFetchResult.ALREADY_IN_SYNC; } LOG.info("Starting replication process"); // get the list of files first fetchFileList(latestGeneration); // this can happen if the commit point is deleted before we fetch the file list. if (filesToDownload.isEmpty()) { - return false; + return IndexFetchResult.PEER_INDEX_COMMIT_DELETED; } LOG.info("Number of files in latest index in master: " + filesToDownload.size()); if (tlogFilesToDownload != null) { @@ -561,14 +616,14 @@ public class IndexFetcher { LOG.warn( "Replication attempt was not successful - trying a full index replication reloadCore={}", reloadCore); - successfulInstall = fetchLatestIndex(true, reloadCore); + successfulInstall = fetchLatestIndex(true, reloadCore).getSuccessful(); } markReplicationStop(); - return successfulInstall; + return successfulInstall ? IndexFetchResult.INDEX_FETCH_SUCCESS : IndexFetchResult.INDEX_FETCH_FAILURE; } catch (ReplicationHandlerException e) { LOG.error("User aborted Replication"); - return false; + return new IndexFetchResult(IndexFetchResult.FAILED_BY_EXCEPTION_MESSAGE, false, e); } catch (SolrException e) { throw e; } catch (InterruptedException e) { 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 4f6a408d533..98bf11ab376 100644 --- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java @@ -90,6 +90,7 @@ import org.apache.solr.core.SolrEventListener; import org.apache.solr.core.backup.repository.BackupRepository; import org.apache.solr.core.backup.repository.LocalFileSystemRepository; import org.apache.solr.core.snapshots.SolrSnapshotMetaDataManager; +import org.apache.solr.handler.IndexFetcher.IndexFetchResult; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.search.SolrIndexSearcher; @@ -392,10 +393,10 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw private volatile IndexFetcher currentIndexFetcher; - public boolean doFetch(SolrParams solrParams, boolean forceReplication) { + public IndexFetchResult doFetch(SolrParams solrParams, boolean forceReplication) { String masterUrl = solrParams == null ? null : solrParams.get(MASTER_URL); if (!indexFetchLock.tryLock()) - return false; + return IndexFetchResult.LOCK_OBTAIN_FAILED; try { if (masterUrl != null) { if (currentIndexFetcher != null && currentIndexFetcher != pollingIndexFetcher) { @@ -411,17 +412,16 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw if (currentIndexFetcher != pollingIndexFetcher) { currentIndexFetcher.destroy(); } + return new IndexFetchResult(IndexFetchResult.FAILED_BY_EXCEPTION_MESSAGE, false, e); } finally { if (pollingIndexFetcher != null) { if( currentIndexFetcher != pollingIndexFetcher) { currentIndexFetcher.destroy(); } - currentIndexFetcher = pollingIndexFetcher; } indexFetchLock.unlock(); } - return false; } boolean isReplicating() { @@ -1151,7 +1151,7 @@ public class ReplicationHandler extends RequestHandlerBase implements SolrCoreAw try { LOG.debug("Polling for index modifications"); markScheduledExecutionStart(); - boolean pollSuccess = doFetch(null, false); + boolean pollSuccess = doFetch(null, false).getSuccessful(); if (pollListener != null) pollListener.onComplete(core, pollSuccess); } catch (Exception e) { LOG.error("Exception in fetching index", e); From f1aef3d12be1300a93a57570e576d94c59ac969e Mon Sep 17 00:00:00 2001 From: David Smiley Date: Fri, 24 Mar 2017 23:01:32 -0400 Subject: [PATCH 059/563] SOLR-10304: Refactor new SolrDocumentFetcher out of SolrIndexSearcher --- solr/CHANGES.txt | 3 + .../clustering/ClusteringComponent.java | 64 +- .../solr/collection1/conf/schema.xml | 11 +- .../clustering/ClusteringComponentTest.java | 68 ++- .../carrot2/CarrotClusteringEngineTest.java | 3 +- .../component/RealTimeGetComponent.java | 22 +- .../solr/highlight/SolrHighlighter.java | 2 +- .../apache/solr/response/DocsStreamer.java | 104 ++-- .../solr/response/TextResponseWriter.java | 2 +- .../transform/ChildDocTransformerFactory.java | 2 +- .../solr/search/SolrDocumentFetcher.java | 571 +++++++++++++++++ .../apache/solr/search/SolrIndexSearcher.java | 573 ++---------------- .../TopGroupsResultTransformer.java | 6 +- .../org/apache/solr/util/SolrPluginUtils.java | 7 +- .../apache/solr/search/LargeFieldTest.java | 8 +- .../apache/solr/util/SolrPluginUtilsTest.java | 89 +-- 16 files changed, 867 insertions(+), 668 deletions(-) create mode 100644 solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index f2937bfcced..dc3ae9d892a 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -140,6 +140,9 @@ Other Changes * SOLR-10249: Refactor IndexFetcher.doFetch() to return a more detailed result. (Jeff Miller via David Smiley) +* SOLR-10304: Refactor Document handling out of SolrIndexSearcher into a new class "SolrDocumentFetcher". + Deprecated SolrPluginUtils.docListToSolrDocumentList(). (David Smiley) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/ClusteringComponent.java b/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/ClusteringComponent.java index 6275c906f02..7c13e6b6630 100644 --- a/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/ClusteringComponent.java +++ b/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/ClusteringComponent.java @@ -26,6 +26,8 @@ import java.util.Map; import java.util.Set; import org.apache.commons.lang.StringUtils; +import org.apache.lucene.document.Document; +import org.apache.lucene.index.IndexableField; import org.apache.solr.common.SolrDocument; import org.apache.solr.common.SolrDocumentList; import org.apache.solr.common.SolrException; @@ -39,8 +41,12 @@ import org.apache.solr.handler.clustering.carrot2.CarrotClusteringEngine; import org.apache.solr.handler.component.ResponseBuilder; import org.apache.solr.handler.component.SearchComponent; import org.apache.solr.handler.component.ShardRequest; +import org.apache.solr.schema.IndexSchema; +import org.apache.solr.schema.SchemaField; +import org.apache.solr.search.DocIterator; +import org.apache.solr.search.DocList; import org.apache.solr.search.DocListAndSet; -import org.apache.solr.util.SolrPluginUtils; +import org.apache.solr.search.SolrIndexSearcher; import org.apache.solr.util.plugin.SolrCoreAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -87,6 +93,60 @@ public class ClusteringComponent extends SearchComponent implements SolrCoreAwar */ private NamedList initParams; + /** + * Convert a DocList to a SolrDocumentList + * + * The optional param "ids" is populated with the lucene document id + * for each SolrDocument. + * + * @param docs The {@link org.apache.solr.search.DocList} to convert + * @param searcher The {@link org.apache.solr.search.SolrIndexSearcher} to use to load the docs from the Lucene index + * @param fields The names of the Fields to load + * @param ids A map to store the ids of the docs + * @return The new {@link SolrDocumentList} containing all the loaded docs + * @throws IOException if there was a problem loading the docs + * @since solr 1.4 + */ + public static SolrDocumentList docListToSolrDocumentList( + DocList docs, + SolrIndexSearcher searcher, + Set fields, + Map ids ) throws IOException + { + IndexSchema schema = searcher.getSchema(); + + SolrDocumentList list = new SolrDocumentList(); + list.setNumFound(docs.matches()); + list.setMaxScore(docs.maxScore()); + list.setStart(docs.offset()); + + DocIterator dit = docs.iterator(); + + while (dit.hasNext()) { + int docid = dit.nextDoc(); + + Document luceneDoc = searcher.doc(docid, fields); + SolrDocument doc = new SolrDocument(); + + for( IndexableField field : luceneDoc) { + if (null == fields || fields.contains(field.name())) { + SchemaField sf = schema.getField( field.name() ); + doc.addField( field.name(), sf.getType().toObject( field ) ); + } + } + if (docs.hasScores() && (null == fields || fields.contains("score"))) { + doc.addField("score", dit.score()); + } + + list.add( doc ); + + if( ids != null ) { + ids.put( doc, new Integer(docid) ); + } + } + return list; + } + @Override @SuppressWarnings({"rawtypes", "unchecked"}) public void init(NamedList args) { @@ -172,7 +232,7 @@ public class ClusteringComponent extends SearchComponent implements SolrCoreAwar checkAvailable(name, engine); DocListAndSet results = rb.getResults(); Map docIds = new HashMap<>(results.docList.size()); - SolrDocumentList solrDocList = SolrPluginUtils.docListToSolrDocumentList( + SolrDocumentList solrDocList = docListToSolrDocumentList( results.docList, rb.req.getSearcher(), engine.getFieldsToLoad(rb.req), docIds); Object clusters = engine.cluster(rb.getQuery(), solrDocList, docIds, rb.req); rb.rsp.add("clusters", clusters); diff --git a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml index af59a20cf2f..0c06a48adc2 100644 --- a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml +++ b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml @@ -277,7 +277,7 @@ --> - + @@ -305,6 +305,10 @@ + + + + + + + + diff --git a/solr/contrib/clustering/src/test/org/apache/solr/handler/clustering/ClusteringComponentTest.java b/solr/contrib/clustering/src/test/org/apache/solr/handler/clustering/ClusteringComponentTest.java index e8cf8309bb8..bff9f378930 100644 --- a/solr/contrib/clustering/src/test/org/apache/solr/handler/clustering/ClusteringComponentTest.java +++ b/solr/contrib/clustering/src/test/org/apache/solr/handler/clustering/ClusteringComponentTest.java @@ -15,6 +15,13 @@ * limitations under the License. */ package org.apache.solr.handler.clustering; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrDocumentList; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.util.NamedList; @@ -24,8 +31,14 @@ import org.apache.solr.handler.component.QueryComponent; import org.apache.solr.handler.component.SearchComponent; import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.request.SolrQueryRequest; -import org.apache.solr.response.SolrQueryResponse; import org.apache.solr.request.SolrRequestHandler; +import org.apache.solr.response.SolrQueryResponse; +import org.apache.solr.search.DocList; +import org.apache.solr.search.QueryCommand; +import org.apache.solr.search.QueryResult; +import org.apache.solr.search.SolrIndexSearcher; +import org.apache.solr.util.RefCounted; +import org.junit.Before; import org.junit.Test; /** @@ -34,6 +47,11 @@ import org.junit.Test; **/ public class ClusteringComponentTest extends AbstractClusteringTestCase { + @Before + public void doBefore() { + clearIndex(); + } + @Test public void testComponent() throws Exception { SolrCore core = h.getCore(); @@ -79,4 +97,52 @@ public class ClusteringComponentTest extends AbstractClusteringTestCase { req.close(); } + + // tests ClusteringComponent.docListToSolrDocumentList + @Test + public void testDocListConversion() throws Exception { + assertU("", adoc("id", "3234", "url", "ignoreme", "val_i", "1", + "val_dynamic", "quick red fox")); + assertU("", adoc("id", "3235", "url", "ignoreme", "val_i", "1", + "val_dynamic", "quick green fox")); + assertU("", adoc("id", "3236", "url", "ignoreme", "val_i", "1", + "val_dynamic", "quick brown fox")); + assertU("", commit()); + + RefCounted holder = h.getCore().getSearcher(); + try { + SolrIndexSearcher srchr = holder.get(); + QueryResult qr = new QueryResult(); + QueryCommand cmd = new QueryCommand(); + cmd.setQuery(new MatchAllDocsQuery()); + cmd.setLen(10); + qr = srchr.search(qr, cmd); + + DocList docs = qr.getDocList(); + assertEquals("wrong docs size", 3, docs.size()); + Set fields = new HashSet<>(); + fields.add("val_dynamic"); + fields.add("dynamic_val"); + fields.add("range_facet_l"); // copied from id + + SolrDocumentList list = ClusteringComponent.docListToSolrDocumentList(docs, srchr, fields, null); + assertEquals("wrong list Size", docs.size(), list.size()); + for (SolrDocument document : list) { + + assertTrue("unexpected field", ! document.containsKey("val_i")); + assertTrue("unexpected id field", ! document.containsKey("id")); + + assertTrue("original field", document.containsKey("val_dynamic")); + assertTrue("dyn copy field", document.containsKey("dynamic_val")); + assertTrue("copy field", document.containsKey("range_facet_l")); + + assertNotNull("original field null", document.get("val_dynamic")); + assertNotNull("dyn copy field null", document.get("dynamic_val")); + assertNotNull("copy field null", document.get("range_facet_l")); + } + } finally { + if (null != holder) holder.decref(); + } + } + } diff --git a/solr/contrib/clustering/src/test/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngineTest.java b/solr/contrib/clustering/src/test/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngineTest.java index 3d6f3d39e5f..6a5b8a08a76 100644 --- a/solr/contrib/clustering/src/test/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngineTest.java +++ b/solr/contrib/clustering/src/test/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngineTest.java @@ -42,7 +42,6 @@ import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.search.DocList; import org.apache.solr.search.SolrIndexSearcher; import org.apache.solr.util.RefCounted; -import org.apache.solr.util.SolrPluginUtils; import org.carrot2.clustering.lingo.LingoClusteringAlgorithm; import org.carrot2.core.LanguageCode; import org.carrot2.util.attribute.AttributeUtils; @@ -465,7 +464,7 @@ public class CarrotClusteringEngineTest extends AbstractClusteringTestCase { // Perform clustering LocalSolrQueryRequest req = new LocalSolrQueryRequest(h.getCore(), solrParams); Map docIds = new HashMap<>(docList.size()); - SolrDocumentList solrDocList = SolrPluginUtils.docListToSolrDocumentList( docList, searcher, engine.getFieldsToLoad(req), docIds ); + SolrDocumentList solrDocList = ClusteringComponent.docListToSolrDocumentList( docList, searcher, engine.getFieldsToLoad(req), docIds ); @SuppressWarnings("unchecked") List> results = (List>) engine.cluster(query, solrDocList, docIds, req); diff --git a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java index 421e74fc917..882decb1627 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java @@ -69,6 +69,7 @@ import org.apache.solr.schema.FieldType; import org.apache.solr.schema.IndexSchema; import org.apache.solr.schema.SchemaField; import org.apache.solr.search.DocList; +import org.apache.solr.search.SolrDocumentFetcher; import org.apache.solr.search.QParser; import org.apache.solr.search.ReturnFields; import org.apache.solr.search.SolrIndexSearcher; @@ -290,7 +291,8 @@ public class RealTimeGetComponent extends SearchComponent Document luceneDocument = searcherInfo.getSearcher().doc(docid, rsp.getReturnFields().getLuceneFieldNames()); SolrDocument doc = toSolrDoc(luceneDocument, core.getLatestSchema()); - searcherInfo.getSearcher().decorateDocValueFields(doc, docid, searcherInfo.getSearcher().getNonStoredDVs(true)); + SolrDocumentFetcher docFetcher = searcherInfo.getSearcher().getDocFetcher(); + docFetcher.decorateDocValueFields(doc, docid, docFetcher.getNonStoredDVs(true)); if ( null != transformer) { if (null == resultContext) { // either first pass, or we've re-opened searcher - either way now we setContext @@ -423,7 +425,8 @@ public class RealTimeGetComponent extends SearchComponent } Document luceneDocument = searcher.doc(docid, returnFields.getLuceneFieldNames()); SolrDocument doc = toSolrDoc(luceneDocument, core.getLatestSchema()); - searcher.decorateDocValueFields(doc, docid, searcher.getNonStoredDVs(false)); + SolrDocumentFetcher docFetcher = searcher.getDocFetcher(); + docFetcher.decorateDocValueFields(doc, docid, docFetcher.getNonStoredDVs(false)); return doc; } finally { @@ -471,10 +474,10 @@ public class RealTimeGetComponent extends SearchComponent } SolrDocument doc; - Set decorateFields = onlyTheseFields == null ? searcher.getNonStoredDVs(false): onlyTheseFields; + Set decorateFields = onlyTheseFields == null ? searcher.getDocFetcher().getNonStoredDVs(false): onlyTheseFields; Document luceneDocument = searcher.doc(docid, returnFields.getLuceneFieldNames()); doc = toSolrDoc(luceneDocument, core.getLatestSchema()); - searcher.decorateDocValueFields(doc, docid, decorateFields); + searcher.getDocFetcher().decorateDocValueFields(doc, docid, decorateFields); long docVersion = (long) doc.getFirstValue(VERSION_FIELD); Object partialVersionObj = partialDoc.getFieldValue(VERSION_FIELD); @@ -483,7 +486,7 @@ public class RealTimeGetComponent extends SearchComponent if (docVersion > partialDocVersion) { return doc; } - for (String fieldName: (Iterable) partialDoc.getFieldNames()) { + for (String fieldName: partialDoc.getFieldNames()) { doc.setField(fieldName.toString(), partialDoc.getFieldValue(fieldName)); // since partial doc will only contain single valued fields, this is fine } @@ -604,17 +607,18 @@ public class RealTimeGetComponent extends SearchComponent int docid = searcher.getFirstMatch(new Term(idField.getName(), idBytes)); if (docid < 0) return null; - + + SolrDocumentFetcher docFetcher = searcher.getDocFetcher(); if (avoidRetrievingStoredFields) { sid = new SolrInputDocument(); } else { - Document luceneDocument = searcher.doc(docid); + Document luceneDocument = docFetcher.doc(docid); sid = toSolrInputDocument(luceneDocument, core.getLatestSchema()); } if (onlyTheseNonStoredDVs != null) { - searcher.decorateDocValueFields(sid, docid, onlyTheseNonStoredDVs); + docFetcher.decorateDocValueFields(sid, docid, onlyTheseNonStoredDVs); } else { - searcher.decorateDocValueFields(sid, docid, searcher.getNonStoredDVsWithoutCopyTargets()); + docFetcher.decorateDocValueFields(sid, docid, docFetcher.getNonStoredDVsWithoutCopyTargets()); } } } finally { diff --git a/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java index a8ee7346ac9..e526c57e52a 100644 --- a/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java +++ b/solr/core/src/java/org/apache/solr/highlight/SolrHighlighter.java @@ -69,7 +69,7 @@ public abstract class SolrHighlighter if (fields[0].contains("*")) { // create a Java regular expression from the wildcard string String fieldRegex = fields[0].replaceAll("\\*", ".*"); - Collection storedHighlightFieldNames = request.getSearcher().getStoredHighlightFieldNames(); + Collection storedHighlightFieldNames = request.getSearcher().getDocFetcher().getStoredHighlightFieldNames(); List storedFieldsToHighlight = new ArrayList<>(); for (String storedFieldName: storedHighlightFieldNames) { if (storedFieldName.matches(fieldRegex)) { diff --git a/solr/core/src/java/org/apache/solr/response/DocsStreamer.java b/solr/core/src/java/org/apache/solr/response/DocsStreamer.java index bdea9ec94b0..40e0afc7484 100644 --- a/solr/core/src/java/org/apache/solr/response/DocsStreamer.java +++ b/solr/core/src/java/org/apache/solr/response/DocsStreamer.java @@ -49,6 +49,8 @@ import org.apache.solr.schema.TrieIntField; import org.apache.solr.schema.TrieLongField; import org.apache.solr.search.DocIterator; import org.apache.solr.search.DocList; +import org.apache.solr.search.ReturnFields; +import org.apache.solr.search.SolrDocumentFetcher; import org.apache.solr.search.SolrReturnFields; /** @@ -57,15 +59,17 @@ import org.apache.solr.search.SolrReturnFields; public class DocsStreamer implements Iterator { public static final Set KNOWN_TYPES = new HashSet<>(); - private org.apache.solr.response.ResultContext rctx; + private final org.apache.solr.response.ResultContext rctx; + private final SolrDocumentFetcher docFetcher; // a collaborator of SolrIndexSearcher private final DocList docs; - private DocTransformer transformer; - private DocIterator docIterator; + private final DocTransformer transformer; + private final DocIterator docIterator; + + private final Set fnames; // returnFields.getLuceneFieldNames(). Maybe null. Not empty. + private final boolean onlyPseudoFields; + private final Set dvFieldsToReturn; // maybe null. Not empty. - private boolean onlyPseudoFields; - private Set fnames; - private Set dvFieldsToReturn; private int idx = -1; public DocsStreamer(ResultContext rctx) { @@ -74,46 +78,61 @@ public class DocsStreamer implements Iterator { transformer = rctx.getReturnFields().getTransformer(); docIterator = this.docs.iterator(); fnames = rctx.getReturnFields().getLuceneFieldNames(); + //TODO move onlyPseudoFields calc to ReturnFields onlyPseudoFields = (fnames == null && !rctx.getReturnFields().wantsAllFields() && !rctx.getReturnFields().hasPatternMatching()) || (fnames != null && fnames.size() == 1 && SolrReturnFields.SCORE.equals(fnames.iterator().next())); // add non-stored DV fields that may have been requested - if (rctx.getReturnFields().wantsAllFields()) { - // check whether there are no additional fields - Set fieldNames = rctx.getReturnFields().getLuceneFieldNames(true); - if (fieldNames == null) { - dvFieldsToReturn = rctx.getSearcher().getNonStoredDVs(true); - } else { - dvFieldsToReturn = new HashSet<>(rctx.getSearcher().getNonStoredDVs(true)); // copy - // add all requested fields that may be useDocValuesAsStored=false - for (String fl : fieldNames) { - if (rctx.getSearcher().getNonStoredDVs(false).contains(fl)) { - dvFieldsToReturn.add(fl); - } - } - } - } else { - if (rctx.getReturnFields().hasPatternMatching()) { - for (String s : rctx.getSearcher().getNonStoredDVs(true)) { - if (rctx.getReturnFields().wantsField(s)) { - if (null == dvFieldsToReturn) { - dvFieldsToReturn = new HashSet<>(); - } - dvFieldsToReturn.add(s); - } - } - } else if (fnames != null) { - dvFieldsToReturn = new HashSet<>(fnames); // copy - // here we get all non-stored dv fields because even if a user has set - // useDocValuesAsStored=false in schema, he may have requested a field - // explicitly using the fl parameter - dvFieldsToReturn.retainAll(rctx.getSearcher().getNonStoredDVs(false)); - } - } + docFetcher = rctx.getSearcher().getDocFetcher(); + dvFieldsToReturn = calcDocValueFieldsForReturn(docFetcher, rctx.getReturnFields()); if (transformer != null) transformer.setContext(rctx); } + // TODO move to ReturnFields ? Or SolrDocumentFetcher ? + public static Set calcDocValueFieldsForReturn(SolrDocumentFetcher docFetcher, ReturnFields returnFields) { + Set result = null; + if (returnFields.wantsAllFields()) { + // check whether there are no additional fields + Set fieldNames = returnFields.getLuceneFieldNames(true); + if (fieldNames == null) { + result = docFetcher.getNonStoredDVs(true); + } else { + result = new HashSet<>(docFetcher.getNonStoredDVs(true)); // copy + // add all requested fields that may be useDocValuesAsStored=false + for (String fl : fieldNames) { + if (docFetcher.getNonStoredDVs(false).contains(fl)) { + result.add(fl); + } + } + } + } else { + if (returnFields.hasPatternMatching()) { + for (String s : docFetcher.getNonStoredDVs(true)) { + if (returnFields.wantsField(s)) { + if (null == result) { + result = new HashSet<>(); + } + result.add(s); + } + } + } else { + Set fnames = returnFields.getLuceneFieldNames(); + if (fnames == null) { + return null; + } + result = new HashSet<>(fnames); // copy + // here we get all non-stored dv fields because even if a user has set + // useDocValuesAsStored=false in schema, he may have requested a field + // explicitly using the fl parameter + result.retainAll(docFetcher.getNonStoredDVs(false)); + } + } + if (result != null && result.isEmpty()) { + return null; + } + return result; + } public int currentIndex() { return idx; @@ -133,12 +152,12 @@ public class DocsStreamer implements Iterator { sdoc = new SolrDocument(); } else { try { - Document doc = rctx.getSearcher().doc(id, fnames); - sdoc = getDoc(doc, rctx.getSearcher().getSchema()); // make sure to use the schema from the searcher and not the request (cross-core) + Document doc = docFetcher.doc(id, fnames); + sdoc = convertLuceneDocToSolrDoc(doc, rctx.getSearcher().getSchema()); // make sure to use the schema from the searcher and not the request (cross-core) // decorate the document with non-stored docValues fields if (dvFieldsToReturn != null) { - rctx.getSearcher().decorateDocValueFields(sdoc, id, dvFieldsToReturn); + docFetcher.decorateDocValueFields(sdoc, id, dvFieldsToReturn); } } catch (IOException e) { throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading document with docId " + id, e); @@ -157,7 +176,8 @@ public class DocsStreamer implements Iterator { } - public static SolrDocument getDoc(Document doc, final IndexSchema schema) { + // TODO move to SolrDocumentFetcher ? Refactor to also call docFetcher.decorateDocValueFields(...) ? + public static SolrDocument convertLuceneDocToSolrDoc(Document doc, final IndexSchema schema) { SolrDocument out = new SolrDocument(); for (IndexableField f : doc.getFields()) { // Make sure multivalued fields are represented as lists diff --git a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java index c4c29943f90..8bef94a0df1 100644 --- a/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java +++ b/solr/core/src/java/org/apache/solr/response/TextResponseWriter.java @@ -147,7 +147,7 @@ public abstract class TextResponseWriter implements PushWriter { } else if (val instanceof Date) { writeDate(name, (Date) val); } else if (val instanceof Document) { - SolrDocument doc = DocsStreamer.getDoc((Document) val, schema); + SolrDocument doc = DocsStreamer.convertLuceneDocToSolrDoc((Document) val, schema); writeSolrDocument(name, doc, returnFields, 0); } else if (val instanceof SolrDocument) { writeSolrDocument(name, (SolrDocument) val, returnFields, 0); diff --git a/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java b/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java index e829e037c2d..45b0efc6632 100644 --- a/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java +++ b/solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java @@ -139,7 +139,7 @@ class ChildDocTransformer extends DocTransformer { while(i.hasNext()) { Integer childDocNum = i.next(); Document childDoc = context.getSearcher().doc(childDocNum); - SolrDocument solrChildDoc = DocsStreamer.getDoc(childDoc, schema); + SolrDocument solrChildDoc = DocsStreamer.convertLuceneDocToSolrDoc(childDoc, schema); // TODO: future enhancement... // support an fl local param in the transformer, which is used to build diff --git a/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java b/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java new file mode 100644 index 00000000000..267d4ebd560 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/search/SolrDocumentFetcher.java @@ -0,0 +1,571 @@ +/* + * 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.search; + +import java.io.IOException; +import java.io.Reader; +import java.lang.invoke.MethodHandles; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.DocumentStoredFieldVisitor; +import org.apache.lucene.document.LazyDocument; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.IndexableFieldType; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.ReaderUtil; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.StoredFieldVisitor; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.NumericUtils; +import org.apache.solr.common.SolrDocumentBase; +import org.apache.solr.core.SolrConfig; +import org.apache.solr.schema.BoolField; +import org.apache.solr.schema.EnumField; +import org.apache.solr.schema.NumberType; +import org.apache.solr.schema.SchemaField; +import org.apache.solr.schema.TrieDateField; +import org.apache.solr.schema.TrieDoubleField; +import org.apache.solr.schema.TrieFloatField; +import org.apache.solr.schema.TrieIntField; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A helper class of {@link org.apache.solr.search.SolrIndexSearcher} for stored Document related matters + * including DocValue substitutions. + */ +public class SolrDocumentFetcher { + + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private final SolrIndexSearcher searcher; + + private final boolean enableLazyFieldLoading; + + private final SolrCache documentCache; + + /** Contains the names/patterns of all docValues=true,stored=false fields in the schema. */ + private final Set allNonStoredDVs; + + /** Contains the names/patterns of all docValues=true,stored=false,useDocValuesAsStored=true fields in the schema. */ + private final Set nonStoredDVsUsedAsStored; + + /** Contains the names/patterns of all docValues=true,stored=false fields, excluding those that are copyField targets in the schema. */ + private final Set nonStoredDVsWithoutCopyTargets; + + private static int largeValueLengthCacheThreshold = Integer.getInteger("solr.largeField.cacheThreshold", 512 * 1024); // internal setting + + private final Set largeFields; + + private Collection storedHighlightFieldNames; // lazy populated; use getter + + SolrDocumentFetcher(SolrIndexSearcher searcher, SolrConfig solrConfig, boolean cachingEnabled) { + this.searcher = searcher; + this.enableLazyFieldLoading = solrConfig.enableLazyFieldLoading; + if (cachingEnabled) { + documentCache = solrConfig.documentCacheConfig == null ? null : solrConfig.documentCacheConfig.newInstance(); + } else { + documentCache = null; + } + + final Set nonStoredDVsUsedAsStored = new HashSet<>(); + final Set allNonStoredDVs = new HashSet<>(); + final Set nonStoredDVsWithoutCopyTargets = new HashSet<>(); + final Set storedLargeFields = new HashSet<>(); + + for (FieldInfo fieldInfo : searcher.getFieldInfos()) { // can find materialized dynamic fields, unlike using the Solr IndexSchema. + final SchemaField schemaField = searcher.getSchema().getFieldOrNull(fieldInfo.name); + if (schemaField == null) { + continue; + } + if (!schemaField.stored() && schemaField.hasDocValues()) { + if (schemaField.useDocValuesAsStored()) { + nonStoredDVsUsedAsStored.add(fieldInfo.name); + } + allNonStoredDVs.add(fieldInfo.name); + if (!searcher.getSchema().isCopyFieldTarget(schemaField)) { + nonStoredDVsWithoutCopyTargets.add(fieldInfo.name); + } + } + if (schemaField.stored() && schemaField.isLarge()) { + storedLargeFields.add(schemaField.getName()); + } + } + + this.nonStoredDVsUsedAsStored = Collections.unmodifiableSet(nonStoredDVsUsedAsStored); + this.allNonStoredDVs = Collections.unmodifiableSet(allNonStoredDVs); + this.nonStoredDVsWithoutCopyTargets = Collections.unmodifiableSet(nonStoredDVsWithoutCopyTargets); + this.largeFields = Collections.unmodifiableSet(storedLargeFields); + } + + public boolean isLazyFieldLoadingEnabled() { + return enableLazyFieldLoading; + } + + public SolrCache getDocumentCache() { + return documentCache; + } + + /** + * Returns a collection of the names of all stored fields which can be highlighted the index reader knows about. + */ + public Collection getStoredHighlightFieldNames() { + synchronized (this) { + if (storedHighlightFieldNames == null) { + storedHighlightFieldNames = new LinkedList<>(); + for (FieldInfo fieldInfo : searcher.getFieldInfos()) { + final String fieldName = fieldInfo.name; + try { + SchemaField field = searcher.getSchema().getField(fieldName); + if (field.stored() && ((field.getType() instanceof org.apache.solr.schema.TextField) + || (field.getType() instanceof org.apache.solr.schema.StrField))) { + storedHighlightFieldNames.add(fieldName); + } + } catch (RuntimeException e) { // getField() throws a SolrException, but it arrives as a RuntimeException + log.warn("Field [{}] found in index, but not defined in schema.", fieldName); + } + } + } + return storedHighlightFieldNames; + } + } + + /** @see SolrIndexSearcher#doc(int) */ + public Document doc(int docId) throws IOException { + return doc(docId, (Set) null); + } + + /** + * Retrieve the {@link Document} instance corresponding to the document id. + *

    + * NOTE: the document will have all fields accessible, but if a field filter is provided, only the provided + * fields will be loaded (the remainder will be available lazily). + * + * @see SolrIndexSearcher#doc(int, Set) + */ + public Document doc(int i, Set fields) throws IOException { + Document d; + if (documentCache != null) { + d = documentCache.get(i); + if (d != null) return d; + } + + final DirectoryReader reader = searcher.getIndexReader(); + if (documentCache != null && !enableLazyFieldLoading) { + // we do not filter the fields in this case because that would return an incomplete document which would + // be eventually cached. The alternative would be to read the stored fields twice; once with the fields + // and then without for caching leading to a performance hit + // see SOLR-8858 for related discussion + fields = null; + } + final SolrDocumentStoredFieldVisitor visitor = new SolrDocumentStoredFieldVisitor(fields, reader, i); + reader.document(i, visitor); + d = visitor.getDocument(); + + if (documentCache != null) { + documentCache.put(i, d); + } + + return d; + } + + /** {@link StoredFieldVisitor} which loads the specified fields eagerly (or all if null). + * If {@link #enableLazyFieldLoading} then the rest get special lazy field entries. Designated "large" + * fields will always get a special field entry. */ + private class SolrDocumentStoredFieldVisitor extends DocumentStoredFieldVisitor { + private final Document doc; + private final LazyDocument lazyFieldProducer; // arguably a better name than LazyDocument; at least how we use it here + private final int docId; + private final boolean addLargeFieldsLazily; + + SolrDocumentStoredFieldVisitor(Set toLoad, IndexReader reader, int docId) { + super(toLoad); + this.docId = docId; + this.doc = getDocument(); + this.lazyFieldProducer = toLoad != null && enableLazyFieldLoading ? new LazyDocument(reader, docId) : null; + this.addLargeFieldsLazily = (documentCache != null && !largeFields.isEmpty()); + //TODO can we return Status.STOP after a val is loaded and we know there are no other fields of interest? + // When: toLoad is one single-valued field, no lazyFieldProducer + } + + @Override + public Status needsField(FieldInfo fieldInfo) throws IOException { + Status status = super.needsField(fieldInfo); + assert status != Status.STOP : "Status.STOP not supported or expected"; + if (addLargeFieldsLazily && largeFields.contains(fieldInfo.name)) { // load "large" fields using this lazy mechanism + if (lazyFieldProducer != null || status == Status.YES) { + doc.add(new LargeLazyField(fieldInfo.name, docId)); + } + return Status.NO; + } + if (status == Status.NO && lazyFieldProducer != null) { // lazy + doc.add(lazyFieldProducer.getField(fieldInfo)); + } + return status; + } + } + + /** @see SolrIndexSearcher#doc(int, StoredFieldVisitor) */ + public void doc(int docId, StoredFieldVisitor visitor) throws IOException { + if (documentCache != null) { + Document cached = documentCache.get(docId); + if (cached != null) { + visitFromCached(cached, visitor); + return; + } + } + searcher.getIndexReader().document(docId, visitor); + } + + /** Executes a stored field visitor against a hit from the document cache */ + private void visitFromCached(Document document, StoredFieldVisitor visitor) throws IOException { + for (IndexableField f : document) { + final FieldInfo info = searcher.getFieldInfos().fieldInfo(f.name()); + final StoredFieldVisitor.Status needsField = visitor.needsField(info); + if (needsField == StoredFieldVisitor.Status.STOP) return; + if (needsField == StoredFieldVisitor.Status.NO) continue; + BytesRef binaryValue = f.binaryValue(); + if (binaryValue != null) { + visitor.binaryField(info, toByteArrayUnwrapIfPossible(binaryValue)); + continue; + } + Number numericValue = f.numericValue(); + if (numericValue != null) { + if (numericValue instanceof Double) { + visitor.doubleField(info, numericValue.doubleValue()); + } else if (numericValue instanceof Integer) { + visitor.intField(info, numericValue.intValue()); + } else if (numericValue instanceof Float) { + visitor.floatField(info, numericValue.floatValue()); + } else if (numericValue instanceof Long) { + visitor.longField(info, numericValue.longValue()); + } else { + throw new AssertionError(); + } + continue; + } + // must be String + if (f instanceof LargeLazyField) { // optimization to avoid premature string conversion + visitor.stringField(info, toByteArrayUnwrapIfPossible(((LargeLazyField) f).readBytes())); + } else { + visitor.stringField(info, f.stringValue().getBytes(StandardCharsets.UTF_8)); + } + } + } + + private byte[] toByteArrayUnwrapIfPossible(BytesRef bytesRef) { + if (bytesRef.offset == 0 && bytesRef.bytes.length == bytesRef.length) { + return bytesRef.bytes; + } else { + return Arrays.copyOfRange(bytesRef.bytes, bytesRef.offset, bytesRef.offset + bytesRef.length); + } + } + + /** Unlike LazyDocument.LazyField, we (a) don't cache large values, and (b) provide access to the byte[]. */ + class LargeLazyField implements IndexableField { + + final String name; + final int docId; + // synchronize on 'this' to access: + BytesRef cachedBytes; // we only conditionally populate this if it's big enough + + private LargeLazyField(String name, int docId) { + this.name = name; + this.docId = docId; + } + + @Override + public String toString() { + return fieldType().toString() + "<" + name() + ">"; // mimic Field.java + } + + @Override + public String name() { + return name; + } + + @Override + public IndexableFieldType fieldType() { + return searcher.getSchema().getField(name()); + } + + @Override + public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) { + return analyzer.tokenStream(name(), stringValue()); // or we could throw unsupported exception? + } + /** (for tests) */ + synchronized boolean hasBeenLoaded() { + return cachedBytes != null; + } + + @Override + public synchronized String stringValue() { + try { + return readBytes().utf8ToString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + synchronized BytesRef readBytes() throws IOException { + if (cachedBytes != null) { + return cachedBytes; + } else { + BytesRef bytesRef = new BytesRef(); + searcher.getIndexReader().document(docId, new StoredFieldVisitor() { + boolean done = false; + @Override + public Status needsField(FieldInfo fieldInfo) throws IOException { + if (done) { + return Status.STOP; + } + return fieldInfo.name.equals(name()) ? Status.YES : Status.NO; + } + + @Override + public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException { + bytesRef.bytes = value; + bytesRef.length = value.length; + done = true; + } + + @Override + public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { + throw new UnsupportedOperationException("'large' binary fields are not (yet) supported"); + } + }); + if (bytesRef.length < largeValueLengthCacheThreshold) { + return cachedBytes = bytesRef; + } else { + return bytesRef; + } + } + } + + @Override + public BytesRef binaryValue() { + return null; + } + + @Override + public Reader readerValue() { + return null; + } + + @Override + public Number numericValue() { + return null; + } + } + + /** + * This will fetch and add the docValues fields to a given SolrDocument/SolrInputDocument + * + * @param doc + * A SolrDocument or SolrInputDocument instance where docValues will be added + * @param docid + * The lucene docid of the document to be populated + * @param fields + * The list of docValues fields to be decorated + */ + public void decorateDocValueFields(@SuppressWarnings("rawtypes") SolrDocumentBase doc, int docid, Set fields) + throws IOException { + final List leafContexts = searcher.getLeafContexts(); + final int subIndex = ReaderUtil.subIndex(docid, leafContexts); + final int localId = docid - leafContexts.get(subIndex).docBase; + final LeafReader leafReader = leafContexts.get(subIndex).reader(); + for (String fieldName : fields) { + final SchemaField schemaField = searcher.getSchema().getFieldOrNull(fieldName); + if (schemaField == null || !schemaField.hasDocValues() || doc.containsKey(fieldName)) { + log.warn("Couldn't decorate docValues for field: [{}], schemaField: [{}]", fieldName, schemaField); + continue; + } + FieldInfo fi = searcher.getFieldInfos().fieldInfo(fieldName); + if (fi == null) { + continue; // Searcher doesn't have info about this field, hence ignore it. + } + final DocValuesType dvType = fi.getDocValuesType(); + switch (dvType) { + case NUMERIC: + final NumericDocValues ndv = leafReader.getNumericDocValues(fieldName); + if (ndv == null) { + continue; + } + Long val; + if (ndv.advanceExact(localId)) { + val = ndv.longValue(); + } else { + continue; + } + Object newVal = val; + if (schemaField.getType().isPointField()) { + // TODO: Maybe merge PointField with TrieFields here + NumberType type = schemaField.getType().getNumberType(); + switch (type) { + case INTEGER: + newVal = val.intValue(); + break; + case LONG: + newVal = val.longValue(); + break; + case FLOAT: + newVal = Float.intBitsToFloat(val.intValue()); + break; + case DOUBLE: + newVal = Double.longBitsToDouble(val); + break; + case DATE: + newVal = new Date(val); + break; + default: + throw new AssertionError("Unexpected PointType: " + type); + } + } else { + if (schemaField.getType() instanceof TrieIntField) { + newVal = val.intValue(); + } else if (schemaField.getType() instanceof TrieFloatField) { + newVal = Float.intBitsToFloat(val.intValue()); + } else if (schemaField.getType() instanceof TrieDoubleField) { + newVal = Double.longBitsToDouble(val); + } else if (schemaField.getType() instanceof TrieDateField) { + newVal = new Date(val); + } else if (schemaField.getType() instanceof EnumField) { + newVal = ((EnumField) schemaField.getType()).intValueToStringValue(val.intValue()); + } + } + doc.addField(fieldName, newVal); + break; + case BINARY: + BinaryDocValues bdv = leafReader.getBinaryDocValues(fieldName); + if (bdv == null) { + continue; + } + BytesRef value; + if (bdv.advanceExact(localId)) { + value = BytesRef.deepCopyOf(bdv.binaryValue()); + } else { + continue; + } + doc.addField(fieldName, value); + break; + case SORTED: + SortedDocValues sdv = leafReader.getSortedDocValues(fieldName); + if (sdv == null) { + continue; + } + if (sdv.advanceExact(localId)) { + final BytesRef bRef = sdv.binaryValue(); + // Special handling for Boolean fields since they're stored as 'T' and 'F'. + if (schemaField.getType() instanceof BoolField) { + doc.addField(fieldName, schemaField.getType().toObject(schemaField, bRef)); + } else { + doc.addField(fieldName, bRef.utf8ToString()); + } + } + break; + case SORTED_NUMERIC: + final SortedNumericDocValues numericDv = leafReader.getSortedNumericDocValues(fieldName); + NumberType type = schemaField.getType().getNumberType(); + if (numericDv != null) { + if (numericDv.advance(localId) == localId) { + final List outValues = new ArrayList(numericDv.docValueCount()); + for (int i = 0; i < numericDv.docValueCount(); i++) { + long number = numericDv.nextValue(); + switch (type) { + case INTEGER: + outValues.add((int)number); + break; + case LONG: + outValues.add(number); + break; + case FLOAT: + outValues.add(NumericUtils.sortableIntToFloat((int)number)); + break; + case DOUBLE: + outValues.add(NumericUtils.sortableLongToDouble(number)); + break; + case DATE: + outValues.add(new Date(number)); + break; + default: + throw new AssertionError("Unexpected PointType: " + type); + } + } + assert outValues.size() > 0; + doc.addField(fieldName, outValues); + } + } + case SORTED_SET: + final SortedSetDocValues values = leafReader.getSortedSetDocValues(fieldName); + if (values != null && values.getValueCount() > 0) { + if (values.advance(localId) == localId) { + final List outValues = new LinkedList<>(); + for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) { + value = values.lookupOrd(ord); + outValues.add(schemaField.getType().toObject(schemaField, value)); + } + assert outValues.size() > 0; + doc.addField(fieldName, outValues); + } + } + case NONE: + break; + } + } + } + + /** + * Returns an unmodifiable set of non-stored docValues field names. + * + * @param onlyUseDocValuesAsStored + * If false, returns all non-stored docValues. If true, returns only those non-stored docValues which have + * the {@link SchemaField#useDocValuesAsStored()} flag true. + */ + public Set getNonStoredDVs(boolean onlyUseDocValuesAsStored) { + return onlyUseDocValuesAsStored ? nonStoredDVsUsedAsStored : allNonStoredDVs; + } + + /** + * Returns an unmodifiable set of names of non-stored docValues fields, except those that are targets of a copy field. + */ + public Set getNonStoredDVsWithoutCopyTargets() { + return nonStoredDVsWithoutCopyTargets; + } + +} 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 83df60f2a42..4207a9b411b 100644 --- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java +++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java @@ -18,19 +18,14 @@ package org.apache.solr.search; import java.io.Closeable; import java.io.IOException; -import java.io.Reader; import java.lang.invoke.MethodHandles; import java.net.URL; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.Date; import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -40,21 +35,26 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import com.google.common.collect.Iterables; -import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.document.Document; -import org.apache.lucene.document.DocumentStoredFieldVisitor; -import org.apache.lucene.document.LazyDocument; -import org.apache.lucene.index.*; -import org.apache.lucene.index.StoredFieldVisitor.Status; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.ExitableDirectoryReader; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.MultiPostingsEnum; +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.StoredFieldVisitor; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.TermContext; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.*; import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.store.Directory; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.FixedBitSet; -import org.apache.lucene.util.NumericUtils; -import org.apache.solr.common.SolrDocumentBase; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.params.ModifiableSolrParams; @@ -71,15 +71,8 @@ import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrRequestInfo; import org.apache.solr.response.SolrQueryResponse; -import org.apache.solr.schema.BoolField; -import org.apache.solr.schema.EnumField; import org.apache.solr.schema.IndexSchema; -import org.apache.solr.schema.NumberType; import org.apache.solr.schema.SchemaField; -import org.apache.solr.schema.TrieDateField; -import org.apache.solr.schema.TrieDoubleField; -import org.apache.solr.schema.TrieFloatField; -import org.apache.solr.schema.TrieIntField; import org.apache.solr.search.facet.UnInvertedField; import org.apache.solr.search.stats.StatsSource; import org.apache.solr.uninverting.UninvertingReader; @@ -107,6 +100,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI private final SolrCore core; private final IndexSchema schema; + private final SolrDocumentFetcher docFetcher; private final String name; private final Date openTime = new Date(); @@ -119,12 +113,10 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI private final int queryResultWindowSize; private final int queryResultMaxDocsCached; private final boolean useFilterForSortedQuery; - public final boolean enableLazyFieldLoading; private final boolean cachingEnabled; private final SolrCache filterCache; private final SolrCache queryResultCache; - private final SolrCache documentCache; private final SolrCache fieldValueCache; // map of generic caches - not synchronized since it's read-only after the constructor. @@ -135,21 +127,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI private final FieldInfos fieldInfos; - /** Contains the names/patterns of all docValues=true,stored=false fields in the schema. */ - private final Set allNonStoredDVs; - - /** Contains the names/patterns of all docValues=true,stored=false,useDocValuesAsStored=true fields in the schema. */ - private final Set nonStoredDVsUsedAsStored; - - /** Contains the names/patterns of all docValues=true,stored=false fields, excluding those that are copyField targets in the schema. */ - private final Set nonStoredDVsWithoutCopyTargets; - - private static int largeValueLengthCacheThreshold = Integer.getInteger("solr.largeField.cacheThreshold", 512 * 1024); // internal setting - - private final Set largeFields; - - private Collection storedHighlightFieldNames; // lazy populated; use getter - private DirectoryFactory directoryFactory; private final LeafReader leafReader; @@ -161,9 +138,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI private final NamedList readerStats; - private static DirectoryReader getReader(SolrCore core, SolrIndexConfig config, DirectoryFactory directoryFactory, - String path) throws IOException { + String path) throws IOException { final Directory dir = directoryFactory.get(path, DirContext.DEFAULT, config.lockType); try { return core.getIndexReaderFactory().newReader(dir, core); @@ -283,7 +259,9 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI this.queryResultWindowSize = solrConfig.queryResultWindowSize; this.queryResultMaxDocsCached = solrConfig.queryResultMaxDocsCached; this.useFilterForSortedQuery = solrConfig.useFilterForSortedQuery; - this.enableLazyFieldLoading = solrConfig.enableLazyFieldLoading; + + this.fieldInfos = leafReader.getFieldInfos(); + this.docFetcher = new SolrDocumentFetcher(this, solrConfig, enableCache); this.cachingEnabled = enableCache; if (cachingEnabled) { @@ -296,7 +274,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI queryResultCache = solrConfig.queryResultCacheConfig == null ? null : solrConfig.queryResultCacheConfig.newInstance(); if (queryResultCache != null) clist.add(queryResultCache); - documentCache = solrConfig.documentCacheConfig == null ? null : solrConfig.documentCacheConfig.newInstance(); + SolrCache documentCache = docFetcher.getDocumentCache(); if (documentCache != null) clist.add(documentCache); if (solrConfig.userCacheConfigs.isEmpty()) { @@ -316,42 +294,11 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI } else { this.filterCache = null; this.queryResultCache = null; - this.documentCache = null; this.fieldValueCache = null; this.cacheMap = NO_GENERIC_CACHES; this.cacheList = NO_CACHES; } - final Set nonStoredDVsUsedAsStored = new HashSet<>(); - final Set allNonStoredDVs = new HashSet<>(); - final Set nonStoredDVsWithoutCopyTargets = new HashSet<>(); - final Set storedLargeFields = new HashSet<>(); - - this.fieldInfos = leafReader.getFieldInfos(); - for (FieldInfo fieldInfo : fieldInfos) { // can find materialized dynamic fields, unlike using the Solr IndexSchema. - final SchemaField schemaField = schema.getFieldOrNull(fieldInfo.name); - if (schemaField == null) { - continue; - } - if (!schemaField.stored() && schemaField.hasDocValues()) { - if (schemaField.useDocValuesAsStored()) { - nonStoredDVsUsedAsStored.add(fieldInfo.name); - } - allNonStoredDVs.add(fieldInfo.name); - if (!schema.isCopyFieldTarget(schemaField)) { - nonStoredDVsWithoutCopyTargets.add(fieldInfo.name); - } - } - if (schemaField.stored() && schemaField.isLarge()) { - storedLargeFields.add(schemaField.getName()); - } - } - - this.nonStoredDVsUsedAsStored = Collections.unmodifiableSet(nonStoredDVsUsedAsStored); - this.allNonStoredDVs = Collections.unmodifiableSet(allNonStoredDVs); - this.nonStoredDVsWithoutCopyTargets = Collections.unmodifiableSet(nonStoredDVsWithoutCopyTargets); - this.largeFields = Collections.unmodifiableSet(storedLargeFields); - // We already have our own filter cache setQueryCache(null); @@ -361,9 +308,21 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI assert ObjectReleaseTracker.track(this); } + public SolrDocumentFetcher getDocFetcher() { + return docFetcher; + } + + List getLeafContexts() { + return super.leafContexts; + } + + public FieldInfos getFieldInfos() { + return fieldInfos; + } + /* - * Override these two methods to provide a way to use global collection stats. - */ + * Override these two methods to provide a way to use global collection stats. + */ @Override public TermStatistics termStatistics(Term term, TermContext context) throws IOException { final SolrRequestInfo reqInfo = SolrRequestInfo.getRequestInfo(); @@ -526,30 +485,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI return filterCache; } - /** - * Returns a collection of the names of all stored fields which can be highlighted the index reader knows about. - */ - public Collection getStoredHighlightFieldNames() { - synchronized (this) { - if (storedHighlightFieldNames == null) { - storedHighlightFieldNames = new LinkedList<>(); - for (FieldInfo fieldInfo : fieldInfos) { - final String fieldName = fieldInfo.name; - try { - SchemaField field = schema.getField(fieldName); - if (field.stored() && ((field.getType() instanceof org.apache.solr.schema.TextField) - || (field.getType() instanceof org.apache.solr.schema.StrField))) { - storedHighlightFieldNames.add(fieldName); - } - } catch (RuntimeException e) { // getField() throws a SolrException, but it arrives as a RuntimeException - log.warn("Field [{}] found in index, but not defined in schema.", fieldName); - } - } - } - return storedHighlightFieldNames; - } - } - // // Set default regenerators on filter and query caches if they don't have any // @@ -638,119 +573,26 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI // } // } - /* ********************** Document retrieval *************************/ - - /* - * Future optimizations (yonik) - * - * If no cache is present: - use NO_LOAD instead of LAZY_LOAD - use LOAD_AND_BREAK if a single field is being - * retrieved - */ - - /** {@link StoredFieldVisitor} which loads the specified fields eagerly (or all if null). - * If {@link #enableLazyFieldLoading} then the rest get special lazy field entries. Designated "large" - * fields will always get a special field entry. */ - private class SolrDocumentStoredFieldVisitor extends DocumentStoredFieldVisitor { - private final Document doc; - private final LazyDocument lazyFieldProducer; // arguably a better name than LazyDocument; at least how we use it here - private final int docId; - private final boolean addLargeFieldsLazily; - - SolrDocumentStoredFieldVisitor(Set toLoad, IndexReader reader, int docId) { - super(toLoad); - this.docId = docId; - this.doc = getDocument(); - this.lazyFieldProducer = toLoad != null && enableLazyFieldLoading ? new LazyDocument(reader, docId) : null; - this.addLargeFieldsLazily = (documentCache != null && !largeFields.isEmpty()); - //TODO can we return Status.STOP after a val is loaded and we know there are no other fields of interest? - // When: toLoad is one single-valued field, no lazyFieldProducer - } - - @Override - public Status needsField(FieldInfo fieldInfo) throws IOException { - Status status = super.needsField(fieldInfo); - assert status != Status.STOP : "Status.STOP not supported or expected"; - if (addLargeFieldsLazily && largeFields.contains(fieldInfo.name)) { // load "large" fields using this lazy mechanism - if (lazyFieldProducer != null || status == Status.YES) { - doc.add(new LargeLazyField(fieldInfo.name, docId)); - } - return Status.NO; - } - if (status == Status.NO && lazyFieldProducer != null) { // lazy - doc.add(lazyFieldProducer.getField(fieldInfo)); - } - return status; - } - } - /** * Retrieve the {@link Document} instance corresponding to the document id. + * + * @see SolrDocumentFetcher */ @Override - public Document doc(int i) throws IOException { - return doc(i, (Set) null); + public Document doc(int docId) throws IOException { + return doc(docId, (Set) null); } /** - * Visit a document's fields using a {@link StoredFieldVisitor} This method does not currently add to the Solr - * document cache. + * Visit a document's fields using a {@link StoredFieldVisitor}. + * This method does not currently add to the Solr document cache. * * @see IndexReader#document(int, StoredFieldVisitor) + * @see SolrDocumentFetcher */ @Override - public void doc(int docId, StoredFieldVisitor visitor) throws IOException { - if (documentCache != null) { - Document cached = documentCache.get(docId); - if (cached != null) { - visitFromCached(cached, visitor); - return; - } - } - getIndexReader().document(docId, visitor); - } - - /** Executes a stored field visitor against a hit from the document cache */ - private void visitFromCached(Document document, StoredFieldVisitor visitor) throws IOException { - for (IndexableField f : document) { - final FieldInfo info = fieldInfos.fieldInfo(f.name()); - final Status needsField = visitor.needsField(info); - if (needsField == Status.STOP) return; - if (needsField == Status.NO) continue; - BytesRef binaryValue = f.binaryValue(); - if (binaryValue != null) { - visitor.binaryField(info, toByteArrayUnwrapIfPossible(binaryValue)); - continue; - } - Number numericValue = f.numericValue(); - if (numericValue != null) { - if (numericValue instanceof Double) { - visitor.doubleField(info, numericValue.doubleValue()); - } else if (numericValue instanceof Integer) { - visitor.intField(info, numericValue.intValue()); - } else if (numericValue instanceof Float) { - visitor.floatField(info, numericValue.floatValue()); - } else if (numericValue instanceof Long) { - visitor.longField(info, numericValue.longValue()); - } else { - throw new AssertionError(); - } - continue; - } - // must be String - if (f instanceof LargeLazyField) { // optimization to avoid premature string conversion - visitor.stringField(info, toByteArrayUnwrapIfPossible(((LargeLazyField) f).readBytes())); - } else { - visitor.stringField(info, f.stringValue().getBytes(StandardCharsets.UTF_8)); - } - } - } - - private byte[] toByteArrayUnwrapIfPossible(BytesRef bytesRef) { - if (bytesRef.offset == 0 && bytesRef.bytes.length == bytesRef.length) { - return bytesRef.bytes; - } else { - return Arrays.copyOfRange(bytesRef.bytes, bytesRef.offset, bytesRef.offset + bytesRef.length); - } + public final void doc(int docId, StoredFieldVisitor visitor) throws IOException { + getDocFetcher().doc(docId, visitor); } /** @@ -758,328 +600,14 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI *

    * NOTE: the document will have all fields accessible, but if a field filter is provided, only the provided * fields will be loaded (the remainder will be available lazily). + * + * @see SolrDocumentFetcher */ @Override - public Document doc(int i, Set fields) throws IOException { - - Document d; - if (documentCache != null) { - d = documentCache.get(i); - if (d != null) return d; - } - - final DirectoryReader reader = getIndexReader(); - if (documentCache != null && !enableLazyFieldLoading) { - // we do not filter the fields in this case because that would return an incomplete document which would - // be eventually cached. The alternative would be to read the stored fields twice; once with the fields - // and then without for caching leading to a performance hit - // see SOLR-8858 for related discussion - fields = null; - } - final SolrDocumentStoredFieldVisitor visitor = new SolrDocumentStoredFieldVisitor(fields, reader, i); - reader.document(i, visitor); - d = visitor.getDocument(); - - if (documentCache != null) { - documentCache.put(i, d); - } - - return d; + public final Document doc(int i, Set fields) throws IOException { + return getDocFetcher().doc(i, fields); } - /** Unlike LazyDocument.LazyField, we (a) don't cache large values, and (b) provide access to the byte[]. */ - class LargeLazyField implements IndexableField { - - final String name; - final int docId; - // synchronize on 'this' to access: - BytesRef cachedBytes; // we only conditionally populate this if it's big enough - - private LargeLazyField(String name, int docId) { - this.name = name; - this.docId = docId; - } - - @Override - public String toString() { - return fieldType().toString() + "<" + name() + ">"; // mimic Field.java - } - - @Override - public String name() { - return name; - } - - @Override - public IndexableFieldType fieldType() { - return schema.getField(name()); - } - - @Override - public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) { - return analyzer.tokenStream(name(), stringValue()); // or we could throw unsupported exception? - } - /** (for tests) */ - synchronized boolean hasBeenLoaded() { - return cachedBytes != null; - } - - @Override - public synchronized String stringValue() { - try { - return readBytes().utf8ToString(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - synchronized BytesRef readBytes() throws IOException { - if (cachedBytes != null) { - return cachedBytes; - } else { - BytesRef bytesRef = new BytesRef(); - getIndexReader().document(docId, new StoredFieldVisitor() { - boolean done = false; - @Override - public Status needsField(FieldInfo fieldInfo) throws IOException { - if (done) { - return Status.STOP; - } - return fieldInfo.name.equals(name()) ? Status.YES : Status.NO; - } - - @Override - public void stringField(FieldInfo fieldInfo, byte[] value) throws IOException { - bytesRef.bytes = value; - bytesRef.length = value.length; - done = true; - } - - @Override - public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException { - throw new UnsupportedOperationException("'large' binary fields are not (yet) supported"); - } - }); - if (bytesRef.length < largeValueLengthCacheThreshold) { - return cachedBytes = bytesRef; - } else { - return bytesRef; - } - } - } - - @Override - public BytesRef binaryValue() { - return null; - } - - @Override - public Reader readerValue() { - return null; - } - - @Override - public Number numericValue() { - return null; - } - } - - /** - * This will fetch and add the docValues fields to a given SolrDocument/SolrInputDocument - * - * @param doc - * A SolrDocument or SolrInputDocument instance where docValues will be added - * @param docid - * The lucene docid of the document to be populated - * @param fields - * The list of docValues fields to be decorated - */ - public void decorateDocValueFields(@SuppressWarnings("rawtypes") SolrDocumentBase doc, int docid, Set fields) - throws IOException { - final int subIndex = ReaderUtil.subIndex(docid, leafContexts); - final int localId = docid - leafContexts.get(subIndex).docBase; - final LeafReader leafReader = leafContexts.get(subIndex).reader(); - for (String fieldName : fields) { - final SchemaField schemaField = schema.getFieldOrNull(fieldName); - if (schemaField == null || !schemaField.hasDocValues() || doc.containsKey(fieldName)) { - log.warn("Couldn't decorate docValues for field: [{}], schemaField: [{}]", fieldName, schemaField); - continue; - } - FieldInfo fi = fieldInfos.fieldInfo(fieldName); - if (fi == null) { - continue; // Searcher doesn't have info about this field, hence ignore it. - } - final DocValuesType dvType = fi.getDocValuesType(); - switch (dvType) { - case NUMERIC: - final NumericDocValues ndv = leafReader.getNumericDocValues(fieldName); - if (ndv == null) { - continue; - } - Long val; - if (ndv.advanceExact(localId)) { - val = ndv.longValue(); - } else { - continue; - } - Object newVal = val; - if (schemaField.getType().isPointField()) { - // TODO: Maybe merge PointField with TrieFields here - NumberType type = schemaField.getType().getNumberType(); - switch (type) { - case INTEGER: - newVal = val.intValue(); - break; - case LONG: - newVal = val; - break; - case FLOAT: - newVal = Float.intBitsToFloat(val.intValue()); - break; - case DOUBLE: - newVal = Double.longBitsToDouble(val); - break; - case DATE: - newVal = new Date(val); - break; - default: - throw new AssertionError("Unexpected PointType: " + type); - } - } else { - if (schemaField.getType() instanceof TrieIntField) { - newVal = val.intValue(); - } else if (schemaField.getType() instanceof TrieFloatField) { - newVal = Float.intBitsToFloat(val.intValue()); - } else if (schemaField.getType() instanceof TrieDoubleField) { - newVal = Double.longBitsToDouble(val); - } else if (schemaField.getType() instanceof TrieDateField) { - newVal = new Date(val); - } else if (schemaField.getType() instanceof EnumField) { - newVal = ((EnumField) schemaField.getType()).intValueToStringValue(val.intValue()); - } - } - doc.addField(fieldName, newVal); - break; - case BINARY: - BinaryDocValues bdv = leafReader.getBinaryDocValues(fieldName); - if (bdv == null) { - continue; - } - BytesRef value; - if (bdv.advanceExact(localId)) { - value = BytesRef.deepCopyOf(bdv.binaryValue()); - } else { - continue; - } - doc.addField(fieldName, value); - break; - case SORTED: - SortedDocValues sdv = leafReader.getSortedDocValues(fieldName); - if (sdv == null) { - continue; - } - if (sdv.advanceExact(localId)) { - final BytesRef bRef = sdv.binaryValue(); - // Special handling for Boolean fields since they're stored as 'T' and 'F'. - if (schemaField.getType() instanceof BoolField) { - doc.addField(fieldName, schemaField.getType().toObject(schemaField, bRef)); - } else { - doc.addField(fieldName, bRef.utf8ToString()); - } - } - break; - case SORTED_NUMERIC: - final SortedNumericDocValues numericDv = leafReader.getSortedNumericDocValues(fieldName); - NumberType type = schemaField.getType().getNumberType(); - if (numericDv != null) { - if (numericDv.advance(localId) == localId) { - final List outValues = new ArrayList(numericDv.docValueCount()); - for (int i = 0; i < numericDv.docValueCount(); i++) { - long number = numericDv.nextValue(); - switch (type) { - case INTEGER: - outValues.add((int)number); - break; - case LONG: - outValues.add(number); - break; - case FLOAT: - outValues.add(NumericUtils.sortableIntToFloat((int)number)); - break; - case DOUBLE: - outValues.add(NumericUtils.sortableLongToDouble(number)); - break; - case DATE: - outValues.add(new Date(number)); - break; - default: - throw new AssertionError("Unexpected PointType: " + type); - } - } - assert outValues.size() > 0; - doc.addField(fieldName, outValues); - } - } - case SORTED_SET: - final SortedSetDocValues values = leafReader.getSortedSetDocValues(fieldName); - if (values != null && values.getValueCount() > 0) { - if (values.advance(localId) == localId) { - final List outValues = new LinkedList(); - for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) { - value = values.lookupOrd(ord); - outValues.add(schemaField.getType().toObject(schemaField, value)); - } - assert outValues.size() > 0; - doc.addField(fieldName, outValues); - } - } - case NONE: - break; - } - } - } - - /** - * Takes a list of docs (the doc ids actually), and reads them into an array of Documents. - */ - public void readDocs(Document[] docs, DocList ids) throws IOException { - readDocs(docs, ids, null); - } - - /** - * Takes a list of docs (the doc ids actually) and a set of fields to load, and reads them into an array of Documents. - */ - public void readDocs(Document[] docs, DocList ids, Set fields) throws IOException { - final DocIterator iter = ids.iterator(); - for (int i = 0; i < docs.length; i++) { - docs[i] = doc(iter.nextDoc(), fields); - } - } - - /** - * Returns an unmodifiable set of non-stored docValues field names. - * - * @param onlyUseDocValuesAsStored - * If false, returns all non-stored docValues. If true, returns only those non-stored docValues which have - * the {@link SchemaField#useDocValuesAsStored()} flag true. - */ - public Set getNonStoredDVs(boolean onlyUseDocValuesAsStored) { - return onlyUseDocValuesAsStored ? nonStoredDVsUsedAsStored : allNonStoredDVs; - } - - /** - * Returns an unmodifiable set of names of non-stored docValues fields, except those that are targets of a copy field. - */ - public Set getNonStoredDVsWithoutCopyTargets() { - return nonStoredDVsWithoutCopyTargets; - } - - /* ********************** end document retrieval *************************/ - - //////////////////////////////////////////////////////////////////////////////// - //////////////////////////////////////////////////////////////////////////////// - //////////////////////////////////////////////////////////////////////////////// - /** expert: internal API, subject to change */ public SolrCache getFieldValueCache() { return fieldValueCache; @@ -2555,15 +2083,6 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI return a.intersects(getDocSet(deState)); } - /** - * Takes a list of document IDs, and returns an array of Documents containing all of the stored fields. - */ - public Document[] readDocs(DocList ids) throws IOException { - final Document[] docs = new Document[ids.size()]; - readDocs(docs, ids); - return docs; - } - /** * Warm this searcher based on an old one (primarily for auto-cache warming). */ diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java index adb81de2c1d..83c81e5e9fd 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/shardresultserializer/TopGroupsResultTransformer.java @@ -19,12 +19,12 @@ package org.apache.solr.search.grouping.distributed.shardresultserializer; import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import org.apache.lucene.document.Document; -import org.apache.lucene.document.DocumentStoredFieldVisitor; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.Sort; @@ -285,9 +285,7 @@ public class TopGroupsResultTransformer implements ShardResultTransformer fields, Map ids ) throws IOException { + /* DWS deprecation note: + It's only called by ClusteringComponent, and I think the "ids" param aspect is a bit messy and not worth supporting. + If someone wants a similar method they can speak up and we can add a method to SolrDocumentFetcher. + */ IndexSchema schema = searcher.getSchema(); SolrDocumentList list = new SolrDocumentList(); diff --git a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java index 57dc2de27af..44463191838 100644 --- a/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java +++ b/solr/core/src/test/org/apache/solr/search/LargeFieldTest.java @@ -112,8 +112,8 @@ public class LargeFieldTest extends SolrTestCaseJ4 { private void assertLazyNotLoaded(Document d, String fieldName) { IndexableField field = d.getField(fieldName); if (fieldName == BIG_FIELD) { - assertTrue(field instanceof SolrIndexSearcher.LargeLazyField); - assertFalse(((SolrIndexSearcher.LargeLazyField)field).hasBeenLoaded()); + assertTrue(field instanceof SolrDocumentFetcher.LargeLazyField); + assertFalse(((SolrDocumentFetcher.LargeLazyField)field).hasBeenLoaded()); } else { assertTrue(field instanceof LazyDocument.LazyField); assertFalse(((LazyDocument.LazyField)field).hasBeenLoaded()); @@ -123,8 +123,8 @@ public class LargeFieldTest extends SolrTestCaseJ4 { private void assertLazyLoaded(Document d, String fieldName) { IndexableField field = d.getField(fieldName); if (fieldName == BIG_FIELD) { - assertTrue(field instanceof SolrIndexSearcher.LargeLazyField); - assertTrue(((SolrIndexSearcher.LargeLazyField)field).hasBeenLoaded()); + assertTrue(field instanceof SolrDocumentFetcher.LargeLazyField); + assertTrue(((SolrDocumentFetcher.LargeLazyField)field).hasBeenLoaded()); } else { assertTrue(field instanceof LazyDocument.LazyField); assertTrue(((LazyDocument.LazyField)field).hasBeenLoaded()); diff --git a/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java b/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java index fc506807630..e41484fbd32 100644 --- a/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java +++ b/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java @@ -16,37 +16,28 @@ */ package org.apache.solr.util; -import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.request.SolrQueryRequest; -import org.apache.solr.search.QParser; -import org.apache.solr.search.QueryCommand; -import org.apache.solr.search.QueryResult; -import org.apache.solr.util.SolrPluginUtils.DisjunctionMaxQueryParser; -import org.apache.solr.search.SolrIndexSearcher; -import org.apache.solr.search.DocList; -import org.apache.solr.common.SolrDocument; -import org.apache.solr.common.SolrDocumentList; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.PhraseQuery; -import org.apache.lucene.search.DisjunctionMaxQuery; -import org.apache.lucene.search.BooleanQuery; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.MatchAllDocsQuery; -import org.apache.lucene.search.BooleanClause.Occur; -import org.junit.BeforeClass; -import org.junit.Test; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.List; -import java.util.Map; import java.util.HashMap; import java.util.Iterator; -import java.util.Set; -import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.DisjunctionMaxQuery; +import org.apache.lucene.search.PhraseQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermQuery; +import org.apache.solr.SolrTestCaseJ4; +import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.search.QParser; +import org.apache.solr.util.SolrPluginUtils.DisjunctionMaxQueryParser; +import org.junit.BeforeClass; +import org.junit.Test; /** * Tests that the functions in SolrPluginUtils work as advertised. @@ -58,52 +49,6 @@ public class SolrPluginUtilsTest extends SolrTestCaseJ4 { initCore("solrconfig.xml","schema.xml"); } - @Test - public void testDocListConversion() throws Exception { - assertU("", adoc("id", "3234", "val_i", "1", - "val_dynamic", "quick red fox")); - assertU("", adoc("id", "3235", "val_i", "1", - "val_dynamic", "quick green fox")); - assertU("", adoc("id", "3236", "val_i", "1", - "val_dynamic", "quick brown fox")); - assertU("", commit()); - - RefCounted holder = h.getCore().getSearcher(); - try { - SolrIndexSearcher srchr = holder.get(); - QueryResult qr = new QueryResult(); - QueryCommand cmd = new QueryCommand(); - cmd.setQuery(new MatchAllDocsQuery()); - cmd.setLen(10); - qr = srchr.search(qr, cmd); - - DocList docs = qr.getDocList(); - assertEquals("wrong docs size", 3, docs.size()); - Set fields = new HashSet<>(); - fields.add("val_dynamic"); - fields.add("dynamic_val"); - fields.add("range_facet_l"); // copied from id - - SolrDocumentList list = SolrPluginUtils.docListToSolrDocumentList(docs, srchr, fields, null); - assertEquals("wrong list Size", docs.size(), list.size()); - for (SolrDocument document : list) { - - assertTrue("unexpected field", ! document.containsKey("val_i")); - assertTrue("unexpected id field", ! document.containsKey("id")); - - assertTrue("original field", document.containsKey("val_dynamic")); - assertTrue("dyn copy field", document.containsKey("dynamic_val")); - assertTrue("copy field", document.containsKey("range_facet_l")); - - assertNotNull("original field null", document.get("val_dynamic")); - assertNotNull("dyn copy field null", document.get("dynamic_val")); - assertNotNull("copy field null", document.get("range_facet_l")); - } - } finally { - if (null != holder) holder.decref(); - } - } - @Test public void testPartialEscape() { From 40f43248fb9c1af34bd430f18306eebed0dee9ba Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Sat, 25 Mar 2017 06:21:29 -0400 Subject: [PATCH 060/563] fix typo in comment --- lucene/core/src/java/org/apache/lucene/index/IndexWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index aa28d9992a0..899643a0679 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -4219,7 +4219,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } } - // If any error occured, throw it. + // If any error occurred, throw it. if (!suppressExceptions) { IOUtils.reThrow(th); } From 56d455c4333a4bda71811426cc6f6e9258852617 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Sat, 25 Mar 2017 18:37:25 +0100 Subject: [PATCH 061/563] Add 6.5.0 back compat test indexes --- .../index/TestBackwardsCompatibility.java | 4 +++- .../org/apache/lucene/index/index.6.5.0-cfs.zip | Bin 0 -> 15832 bytes .../apache/lucene/index/index.6.5.0-nocfs.zip | Bin 0 -> 15836 bytes 3 files changed, 3 insertions(+), 1 deletion(-) create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-cfs.zip create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-nocfs.zip diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java index 1dda6b6b758..8e87dcccda5 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java @@ -297,7 +297,9 @@ public class TestBackwardsCompatibility extends LuceneTestCase { "6.4.1-cfs", "6.4.1-nocfs", "6.4.2-cfs", - "6.4.2-nocfs" + "6.4.2-nocfs", + "6.5.0-cfs", + "6.5.0-nocfs" }; final String[] unsupportedNames = { diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-cfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-cfs.zip new file mode 100644 index 0000000000000000000000000000000000000000..5bff9b252a5db8d6b884192d41de1061f0191ab3 GIT binary patch literal 15832 zcmdVBWprEFwk0exGutt+>U~Ll{cczH zxPP8B&KXBzNK<<)NlSC?HJ7Xe2q-E50DuIz#=3|*W|d7}zdc$50sz1P?*Q8LwE9Md zN{Ubb5a*&<)t`ssXBYqw=*`;`K;NDSsBTLqrae_iPEAfQGh0v1P>k2r#?|M$rune+gnb;4?ut zLVfUk(hFis;Zb~*LvQ(Bg$fz?3^Q$Gj^0ue3k{h9QWQ-HI`R#;z@2HDq~aSgWS~91 z;*>LbOEhDFn1*_m?+djnWl|C%>jxM`dZb+BL&Q_Xf+O|~r(|z;ZSWghFFb!EWuY5Q zJyIc!0(`Rqx@6CS2Lb0yt5qmFsA<`#LRpC^K7L@d*n2pZLYVz#u7XU1_tS#p2qu!A zKuYZdGdw;Z0NLMiR<^5{vI+$NXp#JqGy8v+vl&cF3)DWx)#JN#-qF2k_9e`6DfuLS z<=qN~@DPhIeO=K=QtF}t`RFYtIP&A>>r+XKT6q~CB4idSLVQ^{st=?!pe)e5;&N&| zB2-yLG;&DtMMQDFrm@@j%)yVvcE(P|cehq|qhvdt4yR6Q!zp(R)~myAOq};~g-RRb zMw`Nu`$r4$$)2<4OF+10G4% zc_^QVxuZjVR2Hz_FmbCMb(_2J#~OTuo1UJXfxr>?y76sSDJp)pZsx?C!@GJ&+&ri& zeeJbA7R?-_dD+V{kK=Ns+%rVgWk;opyzox2OQG=IC%8se67%3-Sd=K8v_%7~UZITj zl|?^9)o5m$Zvj{|{G{dXv6+0+b!B>ZWh&4M`@?DaONaa9I=am05LK1u<&?T&wGA{z zXir5*1F_DuRTdm&y13C-gZMS;T)5Q!^Qpp`4&IFZ({j14Isn=Xo}c39uY-dgl>xlI zVDreUzSG}2Ut38;c{_BAln+qIPACk(%CXx`NiMGJ?7|CnSnsu50HUIkxUE!cuv$x z80Ix4kmz}edVvv$-8uK zO1nP@#y`!Lw$QO!Xn6mz>vN<3nGv_E*&6uZqLrJ2bG$!;M{HTkunBbxuEb$4HOm@t zKnYb<5*{(aBjz%ru{vFmqLIF>S|uq>f=a{03W`G#0VBuwDe2_nxXd zg1i>%QoI%;+4yy*9=sO&+`Jasa41W!2JWXK=jc!9WgX<__;o5-6WorAS#-2=r!zat z9jS*m!n6>WU&Mr(;U#pjTz$K9PU*i9e)R{8z}rQ^quvu0a7)G{#P^#pyO>hvm@@C0 zFcV}sIEP{d5Sc)wrshqcI-yz%e#Px;lKU(h05>-wAQIckI;leEaWgST_IQ|IK61@s zgR6-hAvYmgqt^-(s%z%d&6+}+$Ss_TneoXA=ynq9-gs&6U3ys6ugh@U2E*hqMw zEXXi}dC&sJafX!rQbyHY!K`ht+*8D)dC*j%VaI>|kj<1Z%1b0$4P zks)6Q&``33POlLnR9}O;qBzsP$u~B8sTMACPnIj5;^E0XHG{>*91siy*&BhQh%d5d z<tfyKIlQo+f@j9@%Ot+ zkB-7WUz}W_PVyISI=L)W3Z~`QK8icCI&eF3Wm_kR*abptn|z*he=a7xK95kUyn^f~ z^VUF<3P6i2*5mgyv=eRwwlj-zz9ws=z|*`NzhJoKU3~Uc!06dp(mp>S;RbzlRO9wc zla9!b32V=bbD%B#=w$GorzI*;BpVLKf_a{lk$HLo5`+K5xe{x|gsa+bX{+GHg=D5a z+8Dle?v5bq1+7MCaI8)eVmSniV9Ogg1_G{+|u%|Jf8L+Xd%AF28U z&TR1|!mNAaUum>=9orJ7=Cs_L!r8qnMR<|{5dKtn>{D{g z{n2sdNt6YGT|zBgy~51J0MGkU(CDTr(RknGiaK^IMV;9TCd*i?{G(gaaf^OGM;5}_ zyDVXyxDTXLflk_F&)ZeFj>UVT)zyd+DIgJG)nC>PbRvE*8xG?5|?}kBPoNg6K7-PDsQB>bg)fL}?C!!GS zfQ20F>~E*uPU~X0VZ{K9CY8tCWkGw@kv%Y6o#YwBf~EAqyedh1EIF(&kYzzggr;KI zePP>0Sf=Dht!h)3avh3YylPXoR@{o*?(LcMvRyvJ5tC@1pQ`R^>z;dNvLjFZFo(8} z5h%pm!Fp3_GWpT-#!&vz2*Q!mcP6@X{Q{aemjGtq=A8M>T813@qrSa&y4~WWSosj_eUL}yM+d2*wKcpQ*y0TlLH=l7oFDM z9I~({^DH=Y!$_<3SiVy#Y{=wB|Hw5H#kle7vj61gPi$jQUYV4tpIUytTICRa4KCqo z?T=9}$WpUquaqlU{u_|3?fDlJd@HHd%veng31p+Xd;vfvwsYiC`$ z1xNgRJ+JwqqWaE)ep8T=S2Ivb@gbT_Q^Fib)@07L zR0VPaA4k8BSdOa@&XEJJr==I*+1}~28b#BZ_0Gr-?;M66miD;&P*!DPs;eWVXzYg^ z7Kb)$tF0_&hn2S^j;&zL%*EQAN|ERvu=g&=amhLxE6#<8m}M|Na`=?5Vi_r5tV7VP zctoZht3UOxSj`?fy;qwEBRv)^ zu>yByg;jJGgEsuUCOwui6So34$nBF}OLz7*Y#lUA^=Bp&q>9_@$?ws_8V%xF4F?4J z^C4Ffq0tJTrtHK+nHszovx?^7;JjH6u%1KjwM;FX0mNDgH1zL*sv| zSm3bQy&GS=d$%!QW!sUL6`=n1?gC!qv&V1m{9mGp;Xf>z6fpl1P0+~ie~KpYA65Ht zv3Y##Gl(HyVm{I#ko8Xj$JH23NW2Lt=+nT-V1hVVerd3RLE1On#Fe;C1v%n~oXd^U zi9{JtBq?O8DelRr_yX;3^b|;>4P32tj6Ch?+3+1oly`VU9wF<|Is1_Z->iI?y5LDJ zQ$m4?;9cYAf*gTy*e_~S^;iN-?|qr$_}D^Z6BwUBzYtH8;Yr0sRmco#h#jf5t@RIyyPf&F8&krPFIBmyO`|D)N~q!3GXp{RhY2^E?eWhpjg zHy{bx~DxMjDh<%#pC)FI@ff?qg`6QAD+TQLa zCkdU8e6Mp>%|thl0{;m{=XeI$nYQ|3cfHuSmsS;;``v^ZFaSs%QH-4*hRPR}8I-aB zQs(KoY}NC`dEMa2V=2ej&`zcN?c|m4cCv2kOQQCU7JV;frjZ>7F3@8yR`419ONcR% zyr+}(!guY@CT-*PmJ&yuUxpz1$M8Q9h}ILw6Rd=*`Z{pxK}}lfxV=DnCmWNKyRAMh zyhoiB*PY|3wBF^U0WozQs`Xb#NDChN(fu+xkfuk)n!Wmh7jD58s!5X%>4bJXp9fvy zi`zW-sA6QCTC9^#Q#Ge$=-l}2Ln$v)zvq-Cd05quqtP0a2vHW|zR)y!1#J)I?T@gU?KrMH;M(6YCSKvZ+;QNwBYrT`Nx;Jpa}{fOMK?ahEN5E}?@u-4qgRAu5mSO6iw2q*}Ig~YDq#}B9|k;K3& zp%t!)4>J8Q{^1OgNtSj(NF#u*M2xbFjt2Z$i1kGR3+N6Z%sS!qM~Y?D!gx`gXnhh9 zvDF}kCP*6BnNOfC;}FmE06%)rE_&dc@e-(bAv}4W=p3OQV^%atxLbj8gvY0Gd~Y8D z*Z>_0*e)c@;7$|6nM58X@+r=UVb>MX-AW}z8UHXWCefL+9crxUt{(*^5~>aXS`czO^1#I2|~dkDXRHozt50p4BYqHdDC1wNX-*j5L7vce5tbx3>F;9lnl904#$8!BUS;Jz_`uiT2kio;ZpN&={Osmb z^`?l$<)RZpX@y;(5AwuDDK1=11?X*Pz{_SRCQPuH=eDOVI!x7Kg76N(O{Q^&!?&Ts zXH@r$fKsIQN#kX=BBNEnZsJnVsp`WE_@)!|hAPp>goPPnDsVHyV3D3SZ=ffIYz-iM zl?oVTSnOxUsjJeZFO(*vwaPh_BQK71^C+R(TyXvSXDyu4T~ z%ofn<9UouGg}K?AgvgtK>iOi-SLGgXTk)-BTxOP&7$iyT<87oSKN>99NC3RJiv}(e zl93Tx0v%P=A^{x~oKgj9l`0X$(kZu5XbqJtu0*!`cis=S z28vM1#v08#TOc{?g_uR*uKYaG*PsQBBaA8A0%Dj00L;kI#q!RAMC^$*n4ePoXn&g69*T zmfU`$T^4T)87EADlu%sQ8mBHA4$3+etr0N-@PIPfeJvz6`b_aGmq;I-pkvrVp4 z5-~`SQ&h#b4ITcqx{K>fI74VIZnLd}Pbmm}4q%w~#yYgK{-2uYCgw9Xa4I-w;cTQH zoVBr+dHEU4W~V3%jJ>$HpRNpUws@7=O{lem@CizjSym_(!lT7vC9}3!V~_a;q+^%E zqctfSK}el9tM<&LW4M_V5e{7l02nqPNT0E_e9F46yRpsNi-9u0pRgfK2nu{6!8@^o zObDpn<>1(Qrl$Xp5m=|gA)5)~Tm;AbkSe$eIxT|D8rSlzYE1DqWe`Y#-GnMoa;Kv3 z9Yb7>o}0k?M%QT+ch4Zjmh443I)xLZ2nqMNA3S{lsGAcsWuvI0@iMj;MkZxB6Ob%m04{l;Pq0Z9i|9;%*xR?V2iji~K%M8JE- z)14q34`*wc9++;NE}Tx{KenG%-Re7A4UdM+QJOCSP4_?g+eSlx1W@+lbh|s5!>I+g#4Jbb+z0G{y|0CC5nZpf^`2QqKX#9 zaE|Ow4{!Tb%KP%FY&)2tQXhEJofl@#i)F(dmcwr06`HP(mjY2UwR*aIRHzWuRRX*( z?Cu9&m=jNku$JrGX*(f!InD7fcd%L}D@#n3A%=c9eX}F)?@~6U`v2hF-E~e<|0&GZJpJ+F%}Qlor%V2F)r4 z%?rYem@n^nQqJMhjA~xFCwi~bM>_imQzUqLspeKekC7CR2ZNbK4rW)A< zbS&7#o$BS!E|+fTz!ljNbTlOovbMLx!F}<=BcJG~kec4*W83+tMZQ5{ z_8@%HE4*9}p{h1_o zo=+t-Sdx%mB^0X8UvgZ<&A0WAzqM4_W2vtLnHfD8s>vu-*n?n^YYJRXn{+pNu0z8< zWVLQW9iqi?a>>c{D7zl_y6%eOek4-JQJt40q4j0BO_Eb$h*$g+&4j}w^C57l#L8Jw zRKZd1NLpadp}Z9I1HHR<1(bL9kb2N!K|3@CZsmuxiqm3g)v+aM)iW4r)wOIC*7GbB z*3Et#FU=k&?&x$^>{d{1zqxhGd;B^QKRNq3w2b@fum_ioYl!q$;NLYE-OI@q>8~1$ zf%ZSB!NC59X)tUx#y1UiNSco(SfHp(pp-g5fdECV;I|(Als%1FKRlfpt5^S&z1)r5 zviuaQNJ~Dq9(lhGP*K4M0KtyY=IaU}RR3fO@i!4f#tRX8{;LRL{7;LZpPi$uL8;BR z*3qCm<{z!2Q-cW!2}&QkbU{+DpNdF{Z(b51)Az-MY)8O9=_ad&NMjX1p&h9Wi)0Mr zjqQnQfHEx$FISL7NRj;3f6BT{l9hUu-Iid?inq%XG(q=degF<))5cP4V+qy58yDRkw20u))Z3Akb(^g zTJjYkFNz=VlM1P$fyFUX%2Tp00uCeiV(a9KLp-ZFwa#4c!ijr$Ic|?X{1~}Qv~IOd zZ@yh&h%g@#5|AD`iMmO;FlIzKFrZG4AQnB~)qI{AIlxJo#j{_ra@iPTRD+qe5?(Gs98&~*sNP{Jr2odsQQ6=7d(7Vw=iAa%L)@HGlQBt>+VM$ii zW~od!qen_(L`o|rQc}hqRZ2>HRV|r1_R?lXYO$WQsUO{rZjKhHwMxEO5Cb1${G8r_ z1y09w&+2!^@HwV~GmmT~p`@hoY_}q%fu^WpOw$TPOT~%;&q1{kel|r_3cg!Gdb36^ z-7Yy%o8^G}qq+GF$oAb@aOwm?WWOv~f=r(CUTdV(&JOi=xuQmJQBY+l8e261H6k^t z4bZT?2X!&Z1B!vIT9O)j)$Z)^Td{PF_R)tSFUYI@*oqjP!Y7eK7kQPJE#8r6) zeo`2TS)EX3FR#P2pad_6WDW+lIkSt3r0T%Y)Ot9Si=iP6^*t!qL=k*sOpv+BNrbb z+$S{5&4iLTAH0epAFq+q7R^r3QmDUAKmGP@aaUY-JS@&=Ks$I*(wXk~P$&Omwj8pi zRJJ9_qI5Pc$)Z@cCdndA9M_qrPfk|~0en0FXhBDvf=FZ?w(bP=AUhkl9y(cZUCu`Z zj0f#r2~n268rrv$B8~jPSjTRUjG>5@GQh}{#*tVI`W@jYBCT+;go@MApj1iH&u3Zo- zozFOtP>^IPIQ~x7Sjud=!7Wx_YPG96Im%RZ)5<^T78{jTu;8oe;!vBmSX0iAmIfDE zZ*1LT+*7rFeeHTg`*7HN>RgC>CVT2SOIn;UI3Da!Hr15wxW+SvH+Wdd1~cOFLi`1> zJM+rFL3|!Ry6z62=QGzsH@rU3*EYHJ@LV28kMpmBTYa!!c$yX{2uNR$zNR74P(V@? zKvGaZ2Hq(fdr3Z9FCDcQx;hSPuPmG;wsFiJB(^p2?pBnc%kt-b^+(5ujiQZh!SJtg zmZNZkh@Wis=(!x}{5B%6HiH&QhsW6r_c8;A0c2X=4LoTT43@is3?tecaN_3|9Ic&{ zdi-PLfC+U{Sgh{r=A+-539jh4$hTV0i*Ui!ZD9uG`6U`*P2$0=oW}sewHS-iA3)6j zyMb@?)ZQ9Zaob}uqZAl?(lm_qT7>lr@LZMG;{-qOA$jxTEb9buGOA1bvAvb!%)Tn+ zDeD*cqwm)U?D&xuGypL7yb@V;t&E0vG;b8xn|7yM7D9dRf*7Tyc3K)5todrkh}=r? zFHxaqg(GEuRN1-z*oY(=`AkXlsz0Ln%#zL#IPC1_Q*+I7vdvPMMod<_HR7MSp~kYT zS8BjPyV~Y8pLGl~6Yn4t%hapm+WnXs6pwl*;4n#nCnIJfI!YDR^Sl9=M;R-BC+Dr2 zdb&$ABo!8xOJ)lirHG93biV_A5&R^I(Sc^dC_`!=7x^aK!!OaI;0d|hPDd(pWuXpk z-C0QyzFpOce^$uttT&i!K8cI8k~lrc7#yk))sec*CW>)dGNJr#Da|3}+Cs&R;m2-h zK)^ytj$7dHe1nNLbBJacVVpNuV5(ylz_h^Ck_qrjrDsv(Q%K`SQB_pJ;x5ZhcV|ss ze;A`<#Ku%Df&;VYm0Y@9P5(RKo_g60Gh| zFGdcrR-AjogY;RUvHq9$1_ z`*Y8Sddx?FG+;qgY9Z4GakV65X>a-w=;2dhbaXc?&#(BWaLsS^XyM;JRPuU*j5M$Z z`Rz#QU!o>So*MdTxcRVK0ga$A5IZr5_DI=-?~P=#IyFv@>OlCV!8$R_kLV!y$uKqq zbZCJ~eNEI|t1+JcU~1AGm>MU=v4nmq9TT;Y{d4ooa~OwMIamd|gjwD+aS}5RNIWMd zfb6RTb}U)oRAlZ3q`)f?rkng+0)u{n75(k8ccmY%`SIS@-cvm0+eb3}J!{rSozMw1 z^%1-*+;8k8Mh?TUIS@;pM(^>BVQ>(@W)PUi;X@x?zU`4~FBW^fKCOidzrg%1e*)#A z9Db8OjQ>gbgZ4j6{v=~ER`=o@pcz*I4Q$saLMjdaU(27I%NMu5$sgLh>(0|(>vs~EWLqT;MXnpE;%{_7761ZU@#@CKcRgXgBy~i4S;GO+eHGGMe72%#ASY$ z2m0o^Ga*Cc+d;`>{gl&)Y8G>CVY08mmC5h`HccJS>OaNww`^tXXZ54LS?1jF{QaP1`(w7O5^5i0WJpMm8L%R=kX0UPmgqMs zc@jP&quZ`#XE&#iD*JlQ>a&(LHzsSN)sfbt*C_*u{dP&U0m`K~dk9fmWF?P1HR7Ha9pOQ!^RDlE|Wjhl3 z!8{Q#n$w>#_YEEJc;5^#`y6iYyD$npMZ2Zr%QBwKf?1Zr&H+~+SyDKapX>;WslYC` zb33g?Bl-ww-qA?nXMd@s_n$p$f)u1@Pe=cr<_C;wQ+}0pbZ&RD2*J0y&k9ShKgUzx zBXYAC_{uf5R_u|8T0KC`Umfc>a~Nvf`da%W+#0#ddhtaG>->3_S!q_ULL{zpIx(88 z+HyFo3pPQ>n}0oe{A&jCvwq>;s7CA=`zF4GZ{Y{=<Kckih^)2X@|mtTTz3rK*b{e#I_6~Cm8MMc+n>1Z zT1>cPC!r&jg2obfZqBRwY3*6w&Op*kg0u&3X-VsK=@q--&asPF^yEWo(e^Y{UZz2) zg29iB{vwppOnWp#6U=@w1#uqdHbqI}YB&CK&~xym0Tu`jKU;6VH8Arw9HXL48tCs= zv*l;s-j^+)gW)MFij|>54PQDM)EKc>8=lr-Sxjb=(HPauibAQm*;O|@GM|X$%*H@$ zQa8cvFpf@uT3j9C5vrq?9Dq5(zXty8Lhl6{9E|iI z!?6eR(mRS`=9TCAFdp!W=0RAr`|FQhm8vWcTeBT*}SX8#H!U$4NQLV>`bCOpw1g zi=lAcxlc03%7{c;_Fy6imV!^f4<`Q5GQK~^Y`GGJ@wfnt(#=A^P`}7=!L5mgv53Q& z8tU@G=V^FCjC6mr>2|OWg8V94?JV4t8lQY3w*z;_pl`}#azET@^Ff#LIg#i?WV>cuj6@O`@j%CUolcJTgx0lL{%T2w`*fj#U zl1Ie8u;XG?I!_Nu_Gh=`_fqP+#eT0!(rTMBPK3}yTyZ&>^*eUPb8+Y8?ytj!_{6G# zD-{!l ztQG;Rv#?GEWQ<&5NQaLdnh=R(S=x3;lsQ1(0U4VE&?S-ynL`g=Dyxe8S+sFG1Zmj1 z6cZOCygdlAha#+3r)0S--0-tzUmG0mYV_%3Q5{%8sQ*p!rdO~7mjpLI~a|vun}Z^pw|j*ufmxYdp^hSIRlGSM>q9 zYPr_lNTr&v+c?j@;B6&>x+cBFUH67~;5b9#3!BN^(5CJb@9R3vd#juo`AK(v<3sKF zCR@B`dykYG&H6dm@foMDQQr=5Bf2cbh%#4v>e5YYPzF9~TkEITT19AU9==w7O4D~7 zYHX|f-pCIg%4SMD*WI@z8kTLzb^94CgI88T9iznB!~&Yd+SJt2{A_+vtd3kNgM9@M zcpWiB6i!crr`*U+Sfce~u%FTBAY|Hr=Xu_QU(Tegj4e{C*nh}!oi^mq=9WdFGPqu1 zydDO)LlkXh!T#g&;KCI!_@c;X0}P`tSGLh4Q2;;aKy&Abx+ty+P7)C@YU9ZVs&by_E9F@mOuImIm7ChbpaM@Zx1mXLxgGt+@uKs3`R(KEUh?e8Mulc_xnz%(mP z>s!~etf$hdYZtW%l&-e&kl8ODd@!iHYK?px%380yM;wU|gXJb{EX+Ycc}g+PvWFCg z(LXkEee@U!aY!X{1g3DPt<0+TWicrekD{G)sexerIO^atPN-MrhD9xmd_5~@-7>Vk z*M+R4E7R>6oA_L~qi8JgZ2e}3qkDGVnYY}q*k!(iaKUHn>{VG!ySGb0o;`^?kEx1`X<^}rm;Dg{i+1onfm6}JpFf|H@o zt9LWN!0%IK2_4EhJBM!2$~>`{3Rj93w@%g7(k2%uag&i}r8KWOZ!YziAX4P2QI1A- zLt0zZ8NBb@rN{5EsWLxM1voN!152-`_}J0=?>J>8*jc`oNWMN_(i~O48}Yd2{hj5f zEuYf-#`2l|lPn+ZzgYg@{|KX5@>`Rm7w70tmahjHH~4?c^0TbQ7CisPTalAp>*C&6 zKFnJ)>5uUk=KnCuXToIr$@0bjVEKJjZ!CXDfPF=l1Gs<@f$Sbr{B=7bZ~@&qQziNn z0Lkb(NL)EsQY^48(?4}e9~Q;UzpQhA&HO~GM)&{!2Y#3^k9~>^b&C8X zE<@!NmKhc7$t_k_?Ng-haU`jL$`H+$PtzN0axar=(z{tgfi4w70Rd zy*j_VfPsXD3JnSgmJk&e8|oh%AYO8!++tz_{UQspO3KveA7qozdn#Ok1gf#n-aiaA zpekt#D=JB5DE!TBI4mrT00~i9*Wgf(e^U^5HS{>t0^@xwW~qnAccj048;~#Hm47QE z%>T(U0`gyFr2l_J8BxQeukK0t=~opf(EEStmFgOug=_sS$27Oj>dtRv#QW`62mNMO zwKp`jG_-QC*Jk=Z8CFN-{$&W3AAWNK$N$|896FKsH4X^+-UqW=7Yv%A2ZuhpP-^#_ zQc8M6f`!Sq$QX3v)dWD@g=!sC#tBx2^NAFGMR>nSL2SY@tc>$3>I&I@6I0OKhz+0q zC5);K+0qp)=pRde>r=knehvu!aq;Kr=hB_+OdSkqEv)s;|J~AMC4hkOK>oWiL(KmI z&Gq?t`Pa}P*uRDj|2A;=cdXyX?|#K%eoOBkSZ~GbpJDxO*8UYs9_A0MpSFR22KBoo z`B$iV*q=~;8I=DS+V2+QU(s0L-q8Lu9RD+{-z~VmV(G&FiS>^Y_PevnF!20JQ{?3qog~DfggZh&%{j(5%CnmpQ zUH^skk3;0Vg7d|^;bMXHo$KM_P@ge1OHhz Rk>Bo`U;u#O+p7S8{{>a{!j%93 literal 0 HcmV?d00001 diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-nocfs.zip b/lucene/backward-codecs/src/test/org/apache/lucene/index/index.6.5.0-nocfs.zip new file mode 100644 index 0000000000000000000000000000000000000000..72e7dc2849d3023bae16d811f8c70be1b06654e5 GIT binary patch literal 15836 zcmdUWWpG>Rwxx_QGcz+Y$IQ&k%*-(}#mvlX$85*U?3f|8V`hjs=5hLVze;*$?(3Pl zf1Xt5l%$f@`SvbpYprjuEiVNEiUI%tAOL=e9+KcV{$kf}&%OZx0N?-|fF1*#k*SG_ zG9&=Rw|qhU=i%ZG4FCeYd3yus+ZzG(UD?ERXG-ar=_!_$8<{y;v3lt#Tj^?wiD{bS z0MIAGS+cH|66FUbmwmEUnK*I`&>!0dqua38ikAtP1pM%Vj4wiQ^dsZJVa$j8=4hrU z_W@6aVH_EJ%C8D&9qn}}5TVb|bM{tf9Sw<45E&rl@r0lg#lWTB%yT5Q#YhmLPWZ~R zZfG6xOr;XqT6qC4G@g`cX$WlW(8>&mg-AyTXUe6=oSUxce%^ZEH@LodL8hw0H#&x- z!rGS0-hFsQ9?y}!c?lWgHHcgS0!4#Eo zT1Qn7!ELK^1Pvphp{mT=uU2oOJjSeNh104Wu@;3XM@tG2TFfew0HLHq1uM%~ln?Hg z=HpXY5I>-n69GqSDvJaZsuW39ao2NyZua`@#zuKp;I#gM*=~KP-GljjqLTg9$I;_Q z&tYNq!lPHSX)&B}@$8B{Q4W70OH2;OOY^I<+_yR3ACg1VpM68vkdIU&J!ulgN@gA% zb4H6VJzY5xu8kCEk3zj18lY>oz!c(9S>@HfeZjDXVhzHB{i>b!ug`sFZo(j&2i&nZqAh8ZOl_~yzS{q#hxj-Cqb_^aGF># z;X8prgr>m?CeH>GkL+dl{{B))qRPPWVWB>>Gi%O{_i;DDBJ1*Y5xmr?wksC2W!|y^sy^+ht3V1QfVvl z&=&+?7KHGx`@k{$q8Un)pVlr>;RyLL9Kytak>Y}k3mZB>xA4+ryCBt}GyIr=?0|bf z^C3x)hsorFo_UIR{2&)$av$OinOeeXYzxb6rZ)IdDsO^uTd443OwVy0*s&`uYOQ1&K4N*CxQd<5zDja;9PHt zc#B{>7KSrCt&(mIg|QGtT6iEdn60Z}EeVQPpL-d|R}R&%UzaAQ zl-U(}K52&KW-{BUPhI>F?q?Kq>V20*xSFF)qn5n61cu`1FO4bK5<8a#7BV?` zdtUvqDk5yNl0KDStqzCWsVY2NDT5TMcpCFqWFHNf32I|o?oV&NLE8J zX0Cwcl+QwtXDwHO7(!$Ybdrf$M0ogQlF`hwFJcJ?sN3 zzfQK9#SIf@0~!I5K&t`mmtH6Q5>qjn`ony~n#LTTux3m68I0tNT zd8wkSR>V4LqSxUB5qQ196e9%w%$Ho7_R4Fl*E`d*D@SW*Q0bnH-Mk48!%5R>A7!!}Rkm1QP-5dc z5Tc(IG#L>J4T;1D_$lJBL4^ADzWvA>hLuw)TjymcR6fVUlP`CJhoyd51Ev~wjy3KF zBMyW#ISr6sNDi)JGoWlNYP4#ChPOuFva34v*`6EQ-l^fc1{aX*R1Zk>{sc;SZwRSa z_vu29*qBFdYP{d~A}fWWFi>&8=5nDx?+8m`MyX9uPLw0$&K%MC0j#3}Pu)YdQNhW%_>_^W4$4Oxukq%D zYlMSEi`ySAE4S8b#wt_6C&yIFw4wdjZ8m%X#W-d6fjGLn})h2y^ zz&o=McK1gmbCg%1Jq+h=2s8jEhN(7Y7}alvtePE16VxlViL4qY!m&|GU`?0$b_oa7aIGH+FldUHyjA2ARHS*JlNkEytEeFG=m_WLgy#1tQOAtGquzB$UH4DL9b-NhER+VMWMOX2qlK zE{5nTh_B&8Dwe;>daN}53a{M5hza+e@>HB`wf>5nU0$~n!Q~DTWI2Qj63Z}xGJ&!L zjR(qQkTPn{yA*9F*>LdnL6I7xt2W`55a7M6c`b3`Zzz{@X zNa8!m6@H3*4e?^pMMAqTGo)%%OLTh9?sdCZ$`r6yDEqGZJu{r|XlhV3_=I5U8V2jI zq%3JlL}a>|G)108MES3MyYJJs8LSSoub0feju70Oylem1;gnb}&Mv;H&t*X~Z)Kch zFsuKupiH<8CcGb8u!vSaDWA!0ESkT%tid#O!i4c4eVi!c&`Ui*;ii@>K`Ou~!uQv1&Ur-S>h+$k!kmLdL9|CGjVhJCQ48CN=00ri?=jJ>

    j!%1*xs(Sp!ZhU- z(b=^{(r%sfc&~x#%tckqPwXB4b1y=k^#F7 z=-Y?GYG3%tSMSgD&-3?pbWiBNlf(_WV}8;%&TR8`fc=YDJ6ZfYj`#!;~xjgojETUI(vyK0X;sH!C*bUxa34_o*Fg%c`H@=yXTDgH= ziO`dkq@G-=mYkjvuSc?@mzbKFP`;6=ryQ%7s2-n|GX#=EM+NT40eeGp4YLIs4RcS} z+sc4RN36t~TWcGi$EXLlq{D!z%FL_83v&Bp*wLTA1JeWrUGL6}j97b{KhqvRuH7Q3 z+j2-?v=nhA6&bIjG3zK9$=u?%l2^V23+u;ri0Ks2fR;kChxQ#ST^%bm6%#u{6AL4a zseSv-O^`Ok1|Phw!2Y zsJfaC<{b-JmtP2K-@}+z`QM26Q^T4l&zmF>dOINg$gv(HowbGQf1BowzvN4TM)Fa7 zj`sgix4>d{`?bFK4ep@B$TN)4UJ?H7>MU6r2d8gW{x9jo_@9cO-@0bh0cT%B*_cNf|ZWay{RXj)D0?#2^XY7 zUgTax%8+ttVFw*aA137&s36m)P$E6xdfgMGIZvOKc1ZE};}eSTc@J*+4}AERHRCj; zPYStGN>l_mt?s1-f?r^QHL0301(~S>Sd#cTBIHw;oo!2rJIv8OO^^)e18z@ZU!;YO|$u7BFpTVea4TUQniCP{D&T!?I4Ua)aD+5K( zuz$K2m=Dbt6{+HvRK^kDSD`K?x{<@?`j=v2Bb#_WIGOON|K?=cnPz8ZdVbLrwK$ZW zIZnY@RAQy&vnB7tL=G=O3K@%n^Spl8^@2O5uj6_(45OOM9H=`m7| z%2-O0kXD8>TtDmau=;*-wi!rjf1eAyw9Ye&ryPaMl{!taQ{H$XZ-R@8900z~$w-+g zy|ETJHPKF0^VNvfFT8nlHSJ{OnAk2W-F-muU_Weu2>=lrAU+DIDUVo~ODHZN;=YA{ z4F_k@HH~}$Z~ZZQccdfm?IhIkb_x{2GphrP5>=6uIi#s#L8wfb^@Q4-xH@m&YXskiZ>G{6RgFk2RLIdy}Sp`-CHgYd-T04V5cZd z>a9NAYD%kVgf%;${f1`y&54T64n1^~yeXn=Db@4QjN@|nYH8jzUSF*{MyI?RYUxu& zzORStU@Nfe3X#{Tiff9M=UTcF@d^8y$125V?UE*ZHxeW$e_7$^EnyZ4Hli~B5h}v< zbHGfe*HqT4k0@SF`(tl7zG>oTdhcgzS&d9yY6(TM9BRte8K9zE<4END07nunE`)az zZqw?jm?=&(LGe=`RH;nsmdW|QL)gE=mdFO4=%FJ3?Sei_xOUW>2hZa~7`DsS6$24S z1q+bjz6uWCiU%5|Y5QIh$X;fyjl3oetjMND1&0i#XdwZcDvU-<3uxIOs(+yY=#Wsv z89;XP04xB*2y4*YdO#eY-8uleE0g)X1}sif00a;NG(S!$VI1}$O$0iki)T4nBpc<|>89_S8zzeiGyti@QG8~kQQ`9kWrxBxo z(|o#OlIS3q?;sQiBZv?q1Q8?HRHl~ZGWIeKhm?x1sZ8Ip7Txp1m;KNe01T)B1E`Pz zMohOjifBc};B^^S+B%r=!Z985$EsXUBIEH?f$}qt$*o3@T5!w$Mi^J7mh2u_7+d4o z$R3hFN^o{Y^NRi_tZ0U=@OkipubE02(l%PH+&0lshtVdnZFlG1^YSwMRfNwZB@*jW zmxS?SBytHnLYFiRtRb)?8jXnOkPF!?Hzv)EQtVn@u91y__>H$$Mhj;z?Tw3%Rax)# zZ+dd)QNNzx<>8ug1gR*wjq_}i+k8GK&o-mZfH#?dch3Ew#qutd{`II4ZdAuCTt<$P zR6>`k4dW1FMoNscp#Qi?qNlqNv&FSQxWMF#cEVusG|6Gcp)F)K(p}{&a19W~3Orjr$$EbxPGfQZoR`Mp< zmV1_Y8pga?TUPm5?YijnPtMwCmU(MBRK`hh+9N8^b`i?=3HPf$ZnO=bclrXANQ2v> z^71~5pks}ixk_y(MAToNk>10c8pwD#gjKo?hSZCpE(^JWu`-~rV6V`k+S`l^r>5!L z3rfVzgL_Drm--jQ4j@^?)v&o5wq+>_jGn!hSNytD<8aQXRjA^>qK>rNCv-pZVOz46 zZF_|l9}jOzsz}~u4*JpnNv5|^LDrNfnk;U`oSZi&ndu!Wr)GWjHy6M0aCMtGuW(P7 z!Ba&^7h?mZ4Edt8B~H#fl)5w(00b*T;0T`V+A!&^XI;ZIEnpZsF%gGd3E>>OATV8m zRlP{MA#v3_=_)1xXrMox#?UZ0F{(0-Jp5v6P+Ct2jhW0BmqYrG8wiEknh~9JC z7w0r7?;xyf7goH`Dth92f2j}6w)9St5+PX$V6&8@k_~xqy7sLi@V5_7{`wR+;e$u2 z^F!EJ8LI5f4T7?A)U-fw%hj)TwA4+e)>#b$iS;iwk3)(lSZN{7k&y#A=!Ko!p{)2`n*EhXl7Fp80INIlK3jgCi^7fgBUhs z1qZ%~;a}Te%fYY6Hx17guw`dA-rGy7`T6^T?-|ji7KI{3KVx>nJ{o=iEUNgx7xg@Y zpcI($@&<}q68K-xFzC@Y%yPXEHpxwD4lN%MarIM%r1GGyJMPoA~Hft z(Tb|Zqjv|h%q(=1oBa;h9*7;mzvK}1U`0x<@)LMF)_05U8h=0V04xX#f=d4|@cdq& zHpG4SX{Xl$gdfx)60F-XO+bS(X0hx*H*p~{CN*Uz%t_D1j@`;|>7#^7oYqug(z~Tg zNna$ZeQrQ=>eBeWkjqy==6d>buOqy*oDZVQg4!7Tq_ZSK@JXkHi?V zH$FBkgp&2MPs>S2j);!nTNf9EeaC|;jF|z z)dF!!J{7fgk_D+0=i^1YES|7sTgqw=)gphu4d(HgzYq8<%$dn|gg_JU63k2k`zFOAucb29 zyCV+u&?6l2+PvyESo_?zJp2A_5-y~s@D&`0lJ?QvF)~cLnlWh;5qE#*vV3 z=UP(ZfX)@<>kfFREZC#3d~jz%MscaRR)ZD|u)wg33k0l`rq|e<|(e`Z~BXo?mwu%kp741 zuhewrH~n=)T7oK6s;o+&k~u;F4@sjGxDo%9KZnvhK9`wj*!+~g+K<$+`jn_lN4~fb z`(p#3riLB@f|a1h-xnc}mu5x&H&Raz^oi*EuL_9iKdgXCem0NtMrF3%x<{jm7=Lt+ z&Wxv}q$vFzvxP`~f65^h{v~Pn+<+HzvR%QTw41yJBJFhmgBpf zHwbGuav_ps`**cBL6w_vl^Z~GBmY^h?x4t65SQW{NF^=o=&@h2^hvU`on9Pyj^y4> zo|_r&UyY;hM^j3?f6G@(T@Mh=n+US~O9cJQ*S{A*m|E5-i#O}s%%AJ)*?dk{9>1?x z%{GL5G1qv4Q;pKV!NF?`5{aBErr|bfkM77gs}84aoS`CBohhdx%$E^HN@K~3Ev4NF zj29&u#1Gp6L4<6iOd{GU1Wo)15Ihpgm~k>Yo18p}ik4XZxW2}{KiRtPFvYviw;$IP z!|vWBOO}{8gSecU?asuT7H@Q+jU!Nwb$*YkRmSf80l*79rz^UKCq?c$ zI7FUwVz{H|e;JC{?l`-??^g&Q^dRseTokhPI|)1q#qq-lT7}@eb~$`i>-5^CZ|1#| ze7|_x%i2Fv{LXGdX^0hVwVgO+lHyWd5i_Me@WMLg9$srTYsv`NGwT4Ua~oBHMG<#} z%VRs~xrQteyM~83L$Q~l{ri?RN9X=t9&(q&aHr``rw*=G>nD32ynTh@iT9F8k6&;zB}+PSr^w<*?5X(jc%<0!+x9?ghOcntn{ScQtzu?| zxE{<)KEgd56n><8I4J&@Ep>2hpE)|XWd7sq?nPGy_6`*WUx5jkI|(#32Ul`rG_h(L z6a=kNNc7biYk+sX!vX|#zZu5QMv{*1+XnzV8CFU|boOCiLM0 zJ7D<6H$1@~S%#UXu?ch~8*F0Ldz1*UH2qzOvgwuK=dW7Ud!LP<@0^Qp@0_!OZ^D9a z+^d2QT7nOZ!OCi9?dy7bpPB`;Y+uRqgAb4+LyqUsQnXW)%VFy)r(zi01A1m*CK=tc zdm>?c`ZrL)Yfv+smFnvEic|V@w;IcOeeB9$-G$2X+=oquxRi|y8aWW;K8Jse@mR4j z;4_@|%3M&zd^ZfQ``o8C2!wA>1w-HLs4$1 zHHU4Ikh&|G4)6Hd>7B;K*sx^j(41DEGMSl^LALcNC)>ju5$x1R6xAW6bi_yvU)0dRttG5q75 zgyoht&Ixf?Pzz?I^6fM_`PZKB&RRn88rim1?rm9Ru{>ZvNaZ|!v)N?fMP7zCPhef4 zk31j5FVSA_NP6y>F60}l;RMz+FPzzTXV}%K0&OcB_+JDXOg$g<)!uQ_B1Oj^a3R@{ zPs1Udl1$Sfosv!Cl^uATPjPa!LvHCqBE*6;1rs@fed~=`Bhpc3*_h|=0LNn4nDx^G z0fGf({Zi^5%OJWh@gkAB+DaKk6&aD2#;3I=9v=`I&K(6Ex5t>+5#rd9x2jL^2xHQf zhr^H<7~)uvhl50HP_=VV0#4wAk*+&)Vid{Zwd_Edwe2(VDVB)mKGn)9K|j5S1tOu~ zjWaUjrMbn9^?vt+>#@5SZiy>N`wTeb>4H470!pET`+FR7PV@0k@DJ3? zH8PKCqu4Vw$2}=oCl5_w^sVC7iHCEWxj{jzkPWLO@J>tZH3sg4>&jVPUGS{4QBPxS(li%vF7#le_Xb0C(Xz9krf2eJSPP`bO(loIZA%tw!)*`sk!qAyffE|$(^vQ-p@iUT zVgrr6(&{Um_6ObAm*qNp!08-G`*7r49v%mCJ>iQo!9H!v!^EsFh|yULw51Rf<`5L6 z5Gm)g)&8ls_{-;ybC*Y&ZEaTFjm|EWV~x%?{NpBiw7Ko#*pRf76~sCe9#hby=lSxN za5MvM`|0yqK|@Ut&-m3AS-#Kx5Kq)r`#^^c1HdyDkzj?hXwc#%F{hz{;puvr=|`^{ z6U`Kb(eXGL1JUn%7MN0VKRQdv99IYOdDLT!ou4BXWGmb_R21L(r;vz^!RV(5&~Yk< zn8ZrOlyq3wvg-g%F(#AsO{A|V$#e&;?;fA_e^!xxokQF0^tnp9iTqubWx&Xuo2FIWAM>5tuCZi1*}$Df5A+=+ z<8(^hpIz`#RWMOcqs$5~OUq0wT??5xJq3WkTMaBJ0*fsZ$ZAXTc|3W*cw41;THR<$ zR+KgN){WY_%TafyNk$(H4@Y>nB~?7ORw&Q0apZ=a=64b4hadIX$|yhDV_Z11mC=55 zKDgg>-+~mWrq-G|#|UKHr2Od49;PO?3P-vh4C3lhOeHgLI|50XAU)T2)(K--X#8f8 z2pQ5$^J$|t=TBX<%CDPJ7y1#QfHfC2X1B*QC$j68HpL4EG_Vvv#`Z}aH3?fr+pwq- zPGJz8=?as&z)=ZuKiilpJIyq6VlM>i?S-Y)nO?er0&4R9iQfaS+nxE2g!9quogEzZNo9r=YPj2C($E7Vh>SE0ZXZgQH1 z7~b}3YyQSv2E=DLl-LIccH@@d%({P#a8-*6je{rlD?up1og~sdFWZPnxio$C5bRRu zGlySb3u0)k0@7q8Pe)3g-j)!-7N#OqK=_6Gr)IoG_6>0S+xt)N{Xhh8tox%0aj0yt z5=usm9j7vOxUR+>#>}(BFh7L~P}wFkbK1c*){hmzfpcS?7}jR=;DLK#dZ+f>z}zyL zTlC{R7V{2uN(d2m?wnXNEj4+&X+~L7V~i2FoZGD9e6nnx(JzXV>`NS^fR)F}Vqvk+ zp2=6y8Ic^E*H~qu( zpVU84|HJf;ItEkyAod}uSsl>GPJ=Rp%J~1Q{@D~yWTXEZpH~-)csl!4|1kfj^$#Yf z`t^(=)lcOk8CrX=oAlQFk&$0Xe~)Ks-|nv~;KQiwO&?^qBakFl85EHdRDNYLP3@04 zN&;RK$jiBuZcuYi*tp96!4ei$;EkvE*uJ&w)`!P*5tF_!Nmfz6}y09@j-;1q$rIq%HL zQ2BR}bJ;WsT2U+$uC2`vw0UwF@4@D1LLRJA(*Bk$5t&#l+sFAgu|sNOpL0lvXXOsP;ZKjdn8o>E(8_ZpYpx5 zX#dW>TeV1BgYScWjdTbnhOz%q<)Vu#youB7efeJ17d_s~+Z>99frCkVwwhvQu`9X#neupIIa#WwkZX7J z!r+47L!pLRw2&_kI;aK^WJ9w7%?|Q@z-JDfPByyLDvxhFyK5mppwBDs=C4X+AHV19 z-w!pwCm6D&k(wvhRcn55xLkQrTeu8n`5Z%EIoip0pMvDg9Bt%TtZ}y#Ix9&W+>z_n80ttKMg{cj3pV{VoJlNj@zd2u?+#hcmVs1X?6eHV= zZHuC;wg#Sw>+1S`bf9+ zPQ;%aRVBVc1h_s^Skx+nS!>UhT0a4e*}|c|QBd;BO2W$bHSPv-SWP*+bHiyH2Ifs1 znQpS0GsHTRo(Ns}q1t#Oej{wS*bDI^baHi4jK}hH^-c!HxNLtNK#_^K;J(meWKjUu z1u3Y`0h1i7ldUvV&jt#*P3)G>Ojo-?LliO<@+A%z0yV4Q@nfk zZ z8%IfcFxhrHI`po%E?#p##*+r0e7bNQch`7m)_i6o#&zr7fbu4ah`nv^K2D;JT=1+E z9^uA%D4N0&8PLw1C&|pHoT}=0I4A1p?a~wVK1(6Aey2&rmP`!057qe#hSq&DJk}=4Mm=+sh0AP%aVhtsbEYZ;m1N|MO#%d z(No+;CgeKv^D>9gB{bq1>*7WHTg<%+4q+qFa%ryWror@SZD9hkeeKa@19P=kMGqJA^>gffuKl07HNl zxtKjb;&rp2qy75{UFa#`WUGw_xNBs%j|}iN$o%01l0iB|-cbgEYQu;W9sHnYVZAy; z(u4YXIJYSk1Vo6Arvz%5+RqpTN?*PJVemnTU<5jt2lwW83yqShW$ODF4IH$*_4!07 z2K8y+ws&1!X!?%1UThA&qq(H0Hq!FzkmN6-yh;^rZxStmv3l0%lHa%y$#^&S&@Q&M zNEy+bEm-EefF<3G&U|L&(VFhV6_vY5_j~}EO!cw3JU^bDtcR@m&YofGe{;mTj#FXM zAapD6-UR=#b{?$HRSu`z83H%(@FU@8Jv8{Vm;0dR%9ch|DTOPrtTJE2EJ1cl6an65thF^W$)E_Bie98@5io{hlB5MKAan^PH;9t1pCY4Wi(Y z92T^T&e3j!j3I0nOWUY#0$%1cL|^BFkWs0HkyA@iV^v3>wOl%%QyUQ6A%2Poj)5jm zrK6MJ(hMqo;+}e2z8zJDrzuMb1&dnBxm<~I@9u&s}eVj_(L)q)!Jf@+zfAjaToNtD$2St}-*6xgo+cpF^6~4f4%9P$PA&9K$ zlzLGg$>yHf&aDM5JAOGQ>b2Xrv-cb6Q$NtGU7`#ZBiNt~r)qysY4px}??U0;OTY`@R2h|I1Jt z_fwcybE`ZAF+?eh-vLz-F&FCf7ViPtS(G{G-hApiohu|LGsP`Dau)x~_*Q0CP!v6XN}Abal0mJQc&sLlgz-_X*(aSFT1 zdNp?Om+uWlia2gvfzY==s-=U@w6?_rvMq-5lZcA+ecGgeSgk;2pWUS_z9Q1e&hIo8 z;gXHx0@~ryTKdwb5B8=F3L~#+RzR={_PGQOC=G3oWoj{GzG$&`+^mAy!AU6x^CCX3 z-QT;8l}#o;=$(Jd!h1NM3~g6gLw_*)46;Y$nZ|EFnCQ0B`G@eK;heh=8hSCVk>OG?n7j^b z)!7&xyV^V3oy|e0>rJR@_bO!1_D{g+HGY$9$`BK(5WHG}mYFm>T1r5Cb}t(Ge)LeN zg6PyV+T|7YH>Bf*uJl$PKc8E<E;=6&cbF9YjRqDN2XSPDwdRJUVQxjq#g0Ic6} z3Vf@n@$*q;to#gwVh{WcHZhQprWv{;(dyA`SMTxLQ{xTi!{fR2EB^0Ij`4Y}&EH5q z^M8`$WBnJ&AN?O;GQ;|V|sH1Au7S4w6&@mX-)? zz$_vNRwdx7$`4k;4;H8(Ejc0=GWtGrbU$wy)CJOsLiuG0D0+lG*E&SvVeIc*Tt%Ye zX7Vi{YS2Hq_|KsJEf@d8p1AdwZ|<+LpLq57;s1T+#|-ntzuZK>JW%Q~Qc-D@Ny$ku zVht%$Dz^=T(V3Nnfr*iU@d78cr#~|_wFMx}#Dw=|918$~LW1qzdG_FSI*JcK(b>h96xs3dTHQFDSnWGk;u4Y&{ z-aY&sA1c)tL@~0P3*<6Vis@=5a?Nj2ie_30L-&W{2)kHMfyAZ$wDQ=->{!i%BL1FeOgJ+_(h@jfb-NWOMBkIx)Fyhi= zCZfe&<55vj1c(T#2F6E*0$W13>yanWAQN+{#+}mxF=l`JiqjnX=>1kkSpJh`1ns}d z$ngJ+GJ?riKala$t14J(`2W`{eQ196`dg0SHF`VU-^z&J+piAl&8zBUVrFAv>+Gb* z{NK4%Cl&r>238z@a{>?k-3gpMof@441oguoqu&4wig5s&A-{~bXY)@VF{sKEMDOb~ zKs|(V9YN<#qQ%xYiNhH?@0d>{!q&)ntwdtlCZPI}=AsSrkG>`Jq#Se(Ht-*7e`{2} z-9Eex3I6fm&(qJfyE$4oo6uR?8Cm_YcAvK)!Jo(91_kA%fPnEp{%@m(82<%Y=>PNK zUxSBW{~A2}Z?JwJ!TS}9C)Phs*zeYgU$K_op1iH=pC{~h2f?pU zyjX8gfBFgjS%|-P*MG&b!TuBLABXt+?&+^s%cOr`{qqoi-z5AMikJKi>d!61e-`5J zTWG&xSyB9n^^ZgRojd-D)yen=);|yNcZ&2Y6cO_q)SraupN054Gx-(k?k}u=9OCc) zYrkTFvHh{Ge;(rR<@;AC3wFS7#r)re`F|^^zv7W{0Ddd5Z+QR5;tB@-vuq;0-F3hK L0E@Rz0RaCC&U3Kv literal 0 HcmV?d00001 From 013601f05396523ad900a409e67cdbea19571447 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Sat, 25 Mar 2017 18:44:39 +0100 Subject: [PATCH 062/563] Update project doap files with 6.5.0 release --- dev-tools/doap/lucene.rdf | 7 +++++++ dev-tools/doap/solr.rdf | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/dev-tools/doap/lucene.rdf b/dev-tools/doap/lucene.rdf index c1950a0b3a3..64be1107089 100644 --- a/dev-tools/doap/lucene.rdf +++ b/dev-tools/doap/lucene.rdf @@ -66,6 +66,13 @@ + + + lucene-6.5.0 + 2017-03-27 + 6.5.0 + + lucene-6.4.2 diff --git a/dev-tools/doap/solr.rdf b/dev-tools/doap/solr.rdf index 45efd08fa5c..b351d89371e 100644 --- a/dev-tools/doap/solr.rdf +++ b/dev-tools/doap/solr.rdf @@ -66,6 +66,13 @@ + + + solr-6.5.0 + 2017-03-27 + 6.5.0 + + solr-6.4.2 From d156aafd1d5cfda2d7b56e4c73c6ddee43d48e91 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Mon, 27 Mar 2017 16:44:22 +0700 Subject: [PATCH 063/563] SOLR-9835: TestInjection.waitForInSyncWithLeader() should rely on commit point of searcher --- .../org/apache/solr/util/TestInjection.java | 25 +++++++++++++------ 1 file changed, 17 insertions(+), 8 deletions(-) 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 97291a1f6a7..15f5b3d8665 100644 --- a/solr/core/src/java/org/apache/solr/util/TestInjection.java +++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java @@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.lucene.search.IndexSearcher; import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.request.QueryRequest; import org.apache.solr.cloud.ZkController; @@ -43,6 +44,7 @@ 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.search.SolrIndexSearcher; import org.apache.solr.update.SolrIndexWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -363,7 +365,8 @@ public class TestInjection { @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; - + log.info("Start waiting for replica in sync with leader"); + long currentTime = System.currentTimeMillis(); Pair pair = parseValue(waitForReplicasInSync); boolean enabled = pair.first(); if (!enabled) return true; @@ -380,14 +383,20 @@ public class TestInjection { 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); + RefCounted searcher = core.getSearcher(); + try { + String localVersion = searcher.get().getIndexReader().getIndexCommit().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)) { + log.info("Waiting time for replica in sync with leader: {}", System.currentTimeMillis()-currentTime); + return true; + } else { + Thread.sleep(500); + } + } finally { + searcher.decref(); } + } } From cd66a5ff51b7a9ff55edaa9fb5d7df5af42707e4 Mon Sep 17 00:00:00 2001 From: Shalin Shekhar Mangar Date: Mon, 27 Mar 2017 20:06:01 +0530 Subject: [PATCH 064/563] SOLR-9835: Fixed precommit failure. --- solr/core/src/java/org/apache/solr/util/TestInjection.java | 1 - 1 file changed, 1 deletion(-) 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 15f5b3d8665..12992731ac7 100644 --- a/solr/core/src/java/org/apache/solr/util/TestInjection.java +++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java @@ -28,7 +28,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.lucene.search.IndexSearcher; import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.request.QueryRequest; import org.apache.solr.cloud.ZkController; From e11c86f6e4f85fc4ea561283cf6d2fa8c8df2208 Mon Sep 17 00:00:00 2001 From: markrmiller Date: Mon, 27 Mar 2017 10:44:27 -0400 Subject: [PATCH 065/563] SOLR-10076: Move changes entry to 6.6 release. --- solr/CHANGES.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index dc3ae9d892a..e102b27e3e4 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -121,6 +121,8 @@ New Features * SOLR-9994: Add support for CollapseQParser with PointFields. (Varun Thacker, Cao Manh Dat) +* SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs. (Mano Kovacs via Mark Miller) + Optimizations ---------------------- @@ -242,8 +244,6 @@ 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) - * SOLR-10224: Add disk total and disk free metrics. (ab) * SOLR-10085: SQL result set fields should be ordered by the field list (Joel Bernstein) From bd01064f69c2eeb1f7a710e0fcf8e57b8da3150f Mon Sep 17 00:00:00 2001 From: Erick Erickson Date: Mon, 27 Mar 2017 09:31:15 -0700 Subject: [PATCH 066/563] SOLR-10371: There is some spelling mistakes in the Java source code Thanks hu xiaodong" --- .../apache/lucene/search/suggest/FileDictionary.java | 8 ++++---- .../lucene/search/suggest/document/SuggestField.java | 2 +- .../apache/solr/analytics/request/AnalyticsStats.java | 2 +- .../solr/handler/dataimport/XPathRecordReader.java | 10 +++++----- .../org/apache/solr/internal/csv/writer/CSVConfig.java | 2 +- .../solr/internal/csv/writer/CSVConfigGuesser.java | 8 ++++---- .../org/apache/solr/cloud/TestRandomFlRTGCloud.java | 4 ++-- .../org/apache/solr/common/cloud/ZkStateReader.java | 2 +- .../org/apache/solr/common/util/JsonRecordReader.java | 2 +- .../src/java/org/apache/solr/common/util/StrUtils.java | 2 +- 10 files changed, 21 insertions(+), 21 deletions(-) diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java index 7d495182fb8..b0660a9e763 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/FileDictionary.java @@ -67,7 +67,7 @@ public class FileDictionary implements Dictionary { /** * Creates a dictionary based on an inputstream. * Using {@link #DEFAULT_FIELD_DELIMITER} as the - * field seperator in a line. + * field separator in a line. *

    * NOTE: content is treated as UTF-8 */ @@ -78,7 +78,7 @@ public class FileDictionary implements Dictionary { /** * Creates a dictionary based on a reader. * Using {@link #DEFAULT_FIELD_DELIMITER} as the - * field seperator in a line. + * field separator in a line. */ public FileDictionary(Reader reader) { this(reader, DEFAULT_FIELD_DELIMITER); @@ -86,7 +86,7 @@ public class FileDictionary implements Dictionary { /** * Creates a dictionary based on a reader. - * Using fieldDelimiter to seperate out the + * Using fieldDelimiter to separate out the * fields in a line. */ public FileDictionary(Reader reader, String fieldDelimiter) { @@ -96,7 +96,7 @@ public class FileDictionary implements Dictionary { /** * Creates a dictionary based on an inputstream. - * Using fieldDelimiter to seperate out the + * Using fieldDelimiter to separate out the * fields in a line. *

    * NOTE: content is treated as UTF-8 diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java index e5bdda93a6c..7f06328ee1b 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/SuggestField.java @@ -38,7 +38,7 @@ import org.apache.lucene.util.BytesRef; * Besides the usual {@link org.apache.lucene.analysis.Analyzer}s, * {@link CompletionAnalyzer} * can be used to tune suggest field only parameters - * (e.g. preserving token seperators, preserving position increments + * (e.g. preserving token separators, preserving position increments * when converting the token stream to an automaton) *

    *

    diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsStats.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsStats.java index a9cdef908a1..771aff7db5e 100644 --- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsStats.java +++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsStats.java @@ -76,7 +76,7 @@ public class AnalyticsStats { // Get filter to all docs Filter filter = docs.getTopFilter(); - // Computing each Analytics Request Seperately + // Computing each Analytics Request Separately for( AnalyticsRequest areq : requests ){ // The Accumulator which will control the statistics generation // for the entire analytics request diff --git a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java index 69c6c57c7c0..311215cb295 100644 --- a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java +++ b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathRecordReader.java @@ -145,7 +145,7 @@ public class XPathRecordReader { if (!xpath.startsWith("/")) throw new RuntimeException("xpath must start with '/' : " + xpath); List paths = splitEscapeQuote(xpath); - // deal with how split behaves when seperator starts a string! + // deal with how split behaves when separator starts a string! if ("".equals(paths.get(0).trim())) paths.remove(0); rootNode.build(paths, name, multiValued, isRecord, flags); @@ -600,19 +600,19 @@ public class XPathRecordReader { /** - * The Xpath is split into segments using the '/' as a seperator. However + * The Xpath is split into segments using the '/' as a separator. However * this method deals with special cases where there is a slash '/' character * inside the attribute value e.g. x/@html='text/html'. We split by '/' but * then reassemble things were the '/' appears within a quoted sub-string. * - * We have already enforced that the string must begin with a seperator. This + * We have already enforced that the string must begin with a separator. This * method depends heavily on how split behaves if the string starts with the - * seperator or if a sequence of multiple seperator's appear. + * separator or if a sequence of multiple separator's appear. */ private static List splitEscapeQuote(String str) { List result = new LinkedList<>(); String[] ss = str.split("/"); - for (int i=0; i values) { 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 51b4b59d806..80531cb60ee 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 @@ -931,7 +931,7 @@ public class ZkStateReader implements Closeable { public static String getBaseUrlForNodeName(final String nodeName, String urlScheme) { final int _offset = nodeName.indexOf("_"); if (_offset < 0) { - throw new IllegalArgumentException("nodeName does not contain expected '_' seperator: " + nodeName); + throw new IllegalArgumentException("nodeName does not contain expected '_' separator: " + nodeName); } final String hostAndPort = nodeName.substring(0,_offset); try { diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JsonRecordReader.java b/solr/solrj/src/java/org/apache/solr/common/util/JsonRecordReader.java index b9766fbb31d..4dafd39e869 100644 --- a/solr/solrj/src/java/org/apache/solr/common/util/JsonRecordReader.java +++ b/solr/solrj/src/java/org/apache/solr/common/util/JsonRecordReader.java @@ -501,7 +501,7 @@ public class JsonRecordReader { *

    * We have already enforced that the string must begin with a separator. This * method depends heavily on how split behaves if the string starts with the - * seperator or if a sequence of multiple separators appear. + * separator or if a sequence of multiple separators appear. */ private static List splitEscapeQuote(String str) { List result = new LinkedList<>(); diff --git a/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java b/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java index 73135974b1e..a20c6e2af35 100644 --- a/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java +++ b/solr/solrj/src/java/org/apache/solr/common/util/StrUtils.java @@ -297,7 +297,7 @@ public class StrUtils { /** * writes chars from item to out, backslash escaping as needed based on separator -- - * but does not append the seperator itself + * but does not append the separator itself */ public static void appendEscapedTextToBuilder(StringBuilder out, String item, From 2ba54a36babd4cb6f2fb97e0f550d4980dbbced5 Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Mon, 27 Mar 2017 23:56:23 +0530 Subject: [PATCH 067/563] SOLR-10352: bin/solr script now prints warning when available system entropy is lower than 300 --- solr/CHANGES.txt | 3 +++ solr/bin/solr | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index e102b27e3e4..2cebb9fdea8 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -145,6 +145,9 @@ Other Changes * SOLR-10304: Refactor Document handling out of SolrIndexSearcher into a new class "SolrDocumentFetcher". Deprecated SolrPluginUtils.docListToSolrDocumentList(). (David Smiley) +* SOLR-10352: bin/solr script now prints warning when available system entropy is lower than 300 (Esther Quansah via + Ishan Chattopadhyaya) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/bin/solr b/solr/bin/solr index 9563bb2df93..58fdcba6da4 100755 --- a/solr/bin/solr +++ b/solr/bin/solr @@ -1760,6 +1760,12 @@ function launch_solr() { -jar start.jar "${SOLR_JETTY_CONFIG[@]}" \ 1>"$SOLR_LOGS_DIR/solr-$SOLR_PORT-console.log" 2>&1 & echo $! > "$SOLR_PID_DIR/solr-$SOLR_PORT.pid" + # check if /proc/sys/kernel/random/entropy_avail exists then check output of cat /proc/sys/kernel/random/entropy_avail to see if less than 300 + if [[ -f /proc/sys/kernel/random/entropy_avail ]] && (( `cat /proc/sys/kernel/random/entropy_avail` < 30000)); then + echo "Warning: Available entropy is low. As a result, use of the UUIDField, SSL, or any other features that require" + echo "RNG might not work properly. To check for the amount of available entropy, use 'cat /proc/sys/kernel/random/entropy_avail'." + echo "" + fi # no lsof on cygwin though if hash lsof 2>/dev/null ; then # hash returns true if lsof is on the path echo -n "Waiting up to $SOLR_STOP_WAIT seconds to see Solr running on port $SOLR_PORT" From 0b3ca1bb61cb6768ef99a4ee7f4ac05a71d19f56 Mon Sep 17 00:00:00 2001 From: Erick Erickson Date: Mon, 27 Mar 2017 12:15:05 -0700 Subject: [PATCH 068/563] SLR-10108: bin/solr script recursive copy broken --- solr/CHANGES.txt | 2 + solr/bin/solr | 24 +++- solr/bin/solr.cmd | 27 ++-- .../apache/solr/cloud/SolrCLIZkUtilsTest.java | 87 ++++++++++++- .../solr/common/cloud/ZkMaintenanceUtils.java | 120 ++++++++++++++---- 5 files changed, 213 insertions(+), 47 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 2cebb9fdea8..0b6faf83e92 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -135,6 +135,8 @@ Bug Fixes * SOLR-10281: ADMIN_PATHS is duplicated in two places and inconsistent. This can cause automatic retries to /admin/metrics handler by the CloudSolrClient. (shalin) +* SOLR-10108: bin/solr script recursive copy broken (Erick Erickson) + Other Changes ---------------------- diff --git a/solr/bin/solr b/solr/bin/solr index 58fdcba6da4..27ec0542ab7 100755 --- a/solr/bin/solr +++ b/solr/bin/solr @@ -496,17 +496,27 @@ function print_usage() { echo " NOTE: and may both be Zookeeper resources prefixed by 'zk:'" echo " When is a zk resource, may be '.'" echo " If ends with '/', then will be a local folder or parent znode and the last" - echo " element of the path will be appended." + echo " element of the path will be appended unless also ends in a slash. " + echo " may be zk:, which may be useful when using the cp -r form to backup/restore " + echo " the entire zk state." + echo " You must enclose local paths that end in a wildcard in quotes or just" + echo " end the local path in a slash. That is," + echo " 'bin/solr zk cp -r /some/dir/ zk:/ -z localhost:2181' is equivalent to" + echo " 'bin/solr zk cp -r \"/some/dir/*\" zk:/ -z localhost:2181'" + echo " but 'bin/solr zk cp -r /some/dir/* zk:/ -z localhost:2181' will throw an error" echo "" - echo " The 'file:' prefix is stripped, thus 'file:/' specifies an absolute local path and" - echo " 'file:somewhere' specifies a relative local path. All paths on Zookeeper are absolute" - echo " so the slash is required." + echo " here's an example of backup/restore for a ZK configuration:" + echo " to copy to local: 'bin/solr zk cp -r zk:/ /some/dir -z localhost:2181'" + echo " to restore to ZK: 'bin/solr zk cp -r /some/dir/ zk:/ -z localhost:2181'" + echo "" + echo " The 'file:' prefix is stripped, thus 'file:/wherever' specifies an absolute local path and" + echo " 'file:somewhere' specifies a relative local path. All paths on Zookeeper are absolute." echo "" echo " Zookeeper nodes CAN have data, so moving a single file to a parent znode" echo " will overlay the data on the parent Znode so specifying the trailing slash" - echo " is important." + echo " can be important." echo "" - echo " Wildcards are not supported" + echo " Wildcards are supported when copying from local, trailing only and must be quoted." echo "" echo " rm deletes files or folders on Zookeeper" echo " -r     Recursively delete if is a directory. Command will fail if " @@ -1093,7 +1103,7 @@ if [[ "$SCRIPT_CMD" == "zk" ]]; then if [ -z "$ZK_DST" ]; then ZK_DST=$1 else - print_short_zk_usage "Unrecognized or misplaced command $1" + print_short_zk_usage "Unrecognized or misplaced command $1. 'cp' with trailing asterisk requires quoting, see help text." fi fi shift diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd index 29422f33da8..ee76a218e73 100644 --- a/solr/bin/solr.cmd +++ b/solr/bin/solr.cmd @@ -479,23 +479,32 @@ echo. echo. ^, ^ : [file:][/]path/to/local/file or zk:/path/to/zk/node echo NOTE: ^ and ^ may both be Zookeeper resources prefixed by 'zk:' echo When ^ is a zk resource, ^ may be '.' -echo If ^ ends with '/', then ^ will be a local folder or parent znode and the last -echo element of the ^ path will be appended. +echo element of the ^ path will be appended unless ^ also ends in a slash. +echo ^ may be zk:, which may be useful when using the cp -r form to backup/restore +echo the entire zk state. +echo You must enclose local paths that end in a wildcard in quotes or just +echo end the local path in a slash. That is, +echo 'bin/solr zk cp -r /some/dir/ zk:/ -z localhost:2181' is equivalent to +echo 'bin/solr zk cp -r ^"/some/dir/*^" zk:/ -z localhost:2181' +echo but 'bin/solr zk cp -r /some/dir/* zk:/ -z localhost:2181' will throw an error +echo . +echo here's an example of backup/restore for a ZK configuration: +echo to copy to local: 'bin/solr zk cp -r zk:/ /some/dir -z localhost:2181' +echo to restore to ZK: 'bin/solr zk cp -r /some/dir/ zk:/ -z localhost:2181' echo. -echo The 'file:' prefix is stripped, thus 'file:/' specifies an absolute local path and -echo 'file:somewhere' specifies a relative local path. All paths on Zookeeper are absolute -echo so the slash is required. +echo The 'file:' prefix is stripped, thus 'file:/wherever' specifies an absolute local path and +echo 'file:somewhere' specifies a relative local path. All paths on Zookeeper are absolute. echo. echo Zookeeper nodes CAN have data, so moving a single file to a parent znode echo will overlay the data on the parent Znode so specifying the trailing slash -echo is important. +echo can be important. echo. -echo Wildcards are not supported +echo Wildcards are supported when copying from local, trailing only and must be quoted. echo. echo rm deletes files or folders on Zookeeper echo -r Recursively delete if ^ is a directory. Command will fail if ^ echo has children and -r is not specified. Optional -echo ^ : [zk:]/path/to/zk/node. ^ may not be the root ('/')" +echo ^ : [zk:]/path/to/zk/node. ^ may not be the root ('/') echo. echo mv moves (renames) znodes on Zookeeper echo ^, ^ : Zookeeper nodes, the 'zk:' prefix is optional. @@ -512,7 +521,7 @@ echo. echo Only the node names are listed, not data echo. echo mkroot makes a znode in Zookeeper with no data. Can be used to make a path of arbitrary -echo depth but primarily intended to create a 'chroot'." +echo depth but primarily intended to create a 'chroot'. echo. echo ^: The Zookeeper path to create. Leading slash is assumed if not present. echo Intermediate nodes are created as needed if not present. diff --git a/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java b/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java index 78be30bfdc2..35ba1d4584c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java @@ -28,11 +28,14 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.nio.file.SimpleFileVisitor; import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.List; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkMaintenanceUtils; import org.apache.solr.util.SolrCLI; import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -131,7 +134,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase { Path configSet = TEST_PATH().resolve("configsets"); Path srcPathCheck = configSet.resolve("cloud-subdirs").resolve("conf"); - + copyConfigUp(configSet, "cloud-subdirs", "cp1"); // Now copy it somewhere else on ZK. @@ -201,7 +204,6 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase { assertEquals("Copy should have succeeded.", 0, res); verifyZkLocalPathsMatch(srcPathCheck, "/cp4"); - // try with recurse not specified args = new String[]{ "-src", "file:" + srcPathCheck.toAbsolutePath().toString(), @@ -306,6 +308,70 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase { assertEquals("Copy from somewhere in ZK to ZK root should have succeeded.", 0, res); assertTrue("Should have found znode /solrconfig.xml: ", zkClient.exists("/solrconfig.xml", true)); + // Check that the form path/ works for copying files up. Should append the last bit of the source path to the dst + args = new String[]{ + "-src", "file:" + srcPathCheck.toAbsolutePath().toString(), + "-dst", "zk:/cp7/", + "-recurse", "true", + "-zkHost", zkAddr, + }; + + res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args)); + assertEquals("Copy should have succeeded.", 0, res); + verifyZkLocalPathsMatch(srcPathCheck, "/cp7/" + srcPathCheck.getFileName().toString()); + + // Check for an intermediate ZNODE having content. You know cp7/stopwords is a parent node. + tmp = createTempDir("dirdata"); + Path file = Paths.get(tmp.toAbsolutePath().toString(), "zknode.data"); + List lines = new ArrayList<>(); + lines.add("{Some Arbitrary Data}"); + Files.write(file, lines, Charset.forName("UTF-8")); + // First, just copy the data up the cp7 since it's a directory. + args = new String[]{ + "-src", "file:" + file.toAbsolutePath().toString(), + "-dst", "zk:/cp7/conf/stopwords/", + "-recurse", "false", + "-zkHost", zkAddr, + }; + + res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args)); + assertEquals("Copy should have succeeded.", 0, res); + + String content = new String(zkClient.getData("/cp7/conf/stopwords", null, null, true), StandardCharsets.UTF_8); + assertTrue("There should be content in the node! ", content.contains("{Some Arbitrary Data}")); + + + res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args)); + assertEquals("Copy should have succeeded.", 0, res); + + tmp = createTempDir("cp8"); + args = new String[]{ + "-src", "zk:/cp7", + "-dst", "file:" + tmp.toAbsolutePath().toString(), + "-recurse", "true", + "-zkHost", zkAddr, + }; + res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args)); + assertEquals("Copy should have succeeded.", 0, res); + + // Next, copy cp7 down and verify that zknode.data exists for cp7 + Path zData = Paths.get(tmp.toAbsolutePath().toString(), "conf/stopwords/zknode.data"); + assertTrue("znode.data should have been copied down", zData.toFile().exists()); + + // Finally, copy up to cp8 and verify that the data is up there. + args = new String[]{ + "-src", "file:" + tmp.toAbsolutePath().toString(), + "-dst", "zk:/cp9", + "-recurse", "true", + "-zkHost", zkAddr, + }; + + res = cpTool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(cpTool.getOptions()), args)); + assertEquals("Copy should have succeeded.", 0, res); + + content = new String(zkClient.getData("/cp9/conf/stopwords", null, null, true), StandardCharsets.UTF_8); + assertTrue("There should be content in the node! ", content.contains("{Some Arbitrary Data}")); + } @Test @@ -577,13 +643,22 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase { verifyAllZNodesAreFiles(fileRoot, zkRoot); } + private static boolean isEphemeral(String zkPath) throws KeeperException, InterruptedException { + Stat znodeStat = zkClient.exists(zkPath, null, true); + return znodeStat.getEphemeralOwner() != 0; + } + void verifyAllZNodesAreFiles(Path fileRoot, String zkRoot) throws KeeperException, InterruptedException { - for (String node : zkClient.getChildren(zkRoot, null, true)) { - Path thisPath = Paths.get(fileRoot.toAbsolutePath().toString(), node); - assertTrue("Znode " + node + " should have been found on disk at " + fileRoot.toAbsolutePath().toString(), + for (String child : zkClient.getChildren(zkRoot, null, true)) { + // Skip ephemeral nodes + if (zkRoot.endsWith("/") == false) zkRoot += "/"; + if (isEphemeral(zkRoot + child)) continue; + + Path thisPath = Paths.get(fileRoot.toAbsolutePath().toString(), child); + assertTrue("Znode " + child + " should have been found on disk at " + fileRoot.toAbsolutePath().toString(), Files.exists(thisPath)); - verifyAllZNodesAreFiles(thisPath, zkRoot + "/" + node); + verifyAllZNodesAreFiles(thisPath, zkRoot + child); } } diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java index b7aa3d27589..f569ae3bfd4 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java @@ -17,6 +17,7 @@ package org.apache.solr.common.cloud; +import java.io.File; import java.io.IOException; import java.lang.invoke.MethodHandles; import java.nio.file.FileVisitResult; @@ -31,6 +32,7 @@ import java.util.regex.Pattern; import org.apache.solr.client.solrj.SolrServerException; import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +42,7 @@ import org.slf4j.LoggerFactory; */ public class ZkMaintenanceUtils { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private static final String ZKNODE_DATA_FILE = "zknode.data"; private ZkMaintenanceUtils() {} // don't let it be instantiated, all methods are static. /** @@ -119,6 +122,9 @@ public class ZkMaintenanceUtils { if (srcIsZk == false && dstIsZk == false) { throw new SolrServerException("At least one of the source and dest parameters must be prefixed with 'zk:' "); } + if (dstIsZk && dst.length() == 0) { + dst = "/"; // for consistency, one can copy from zk: and send to zk:/ + } dst = normalizeDest(src, dst); if (srcIsZk && dstIsZk) { @@ -148,18 +154,26 @@ public class ZkMaintenanceUtils { Files.write(filename, data); } + private static String normalizeDest(String srcName, String dstName) { - // Pull the last element of the src path and add it to the dst. - if (dstName.endsWith("/")) { + // Special handling for "." + if (dstName.equals(".")) { + return Paths.get(".").normalize().toAbsolutePath().toString(); + } + // Pull the last element of the src path and add it to the dst if the src does NOT end in a slash + + // If the source ends in a slash, do not append the last segment to the dest + + if (dstName.endsWith("/")) { // Dest is a directory. int pos = srcName.lastIndexOf("/"); if (pos < 0) { dstName += srcName; } else { dstName += srcName.substring(pos + 1); } - } else if (dstName.equals(".")) { - dstName = Paths.get(".").normalize().toAbsolutePath().toString(); } + + log.info("copying from '{}' to '{}'", srcName, dstName); return dstName; } @@ -226,10 +240,17 @@ public class ZkMaintenanceUtils { } }); } + + public static void uploadToZK(SolrZkClient zkClient, final Path fromPath, final String zkPath, + final Pattern filenameExclusions) throws IOException { - public static void uploadToZK(SolrZkClient zkClient, final Path rootPath, final String zkPath, - final Pattern filenameExclusions) throws IOException { + String path = fromPath.toString(); + if (path.endsWith("*")) { + path = path.substring(0, path.length() - 1); + } + final Path rootPath = Paths.get(path); + if (!Files.exists(rootPath)) throw new IOException("Path " + rootPath + " does not exist"); @@ -243,7 +264,12 @@ public class ZkMaintenanceUtils { } String zkNode = createZkNodeName(zkPath, rootPath, file); try { - zkClient.makePath(zkNode, file.toFile(), false, true); + // if the path exists (and presumably we're uploading data to it) just set its data + if (file.toFile().getName().equals(ZKNODE_DATA_FILE) && zkClient.exists(zkNode, true)) { + zkClient.setData(zkNode, file.toFile(), true); + } else { + zkClient.makePath(zkNode, file.toFile(), false, true); + } } catch (KeeperException | InterruptedException e) { throw new IOException("Error uploading file " + file.toString() + " to zookeeper path " + zkNode, SolrZkClient.checkInterrupted(e)); @@ -253,28 +279,58 @@ public class ZkMaintenanceUtils { @Override public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException { - return (dir.getFileName().toString().startsWith(".")) ? FileVisitResult.SKIP_SUBTREE : FileVisitResult.CONTINUE; + if (dir.getFileName().toString().startsWith(".")) return FileVisitResult.SKIP_SUBTREE; + + return FileVisitResult.CONTINUE; } }); } - public static void downloadFromZK(SolrZkClient zkClient, String zkPath, Path dir) throws IOException { + private static boolean isEphemeral(SolrZkClient zkClient, String zkPath) throws KeeperException, InterruptedException { + Stat znodeStat = zkClient.exists(zkPath, null, true); + return znodeStat.getEphemeralOwner() != 0; + } + + private static int copyDataDown(SolrZkClient zkClient, String zkPath, File file) throws IOException, KeeperException, InterruptedException { + byte[] data = zkClient.getData(zkPath, null, null, true); + if (data != null && data.length > 1) { // There are apparently basically empty ZNodes. + log.info("Writing file {}", file.toString()); + Files.write(file.toPath(), data); + return data.length; + } + return 0; + } + + public static void downloadFromZK(SolrZkClient zkClient, String zkPath, Path file) throws IOException { try { - List files = zkClient.getChildren(zkPath, null, true); - Files.createDirectories(dir); - for (String file : files) { - List children = zkClient.getChildren(zkPath + "/" + file, null, true); - if (children.size() == 0) { - byte[] data = zkClient.getData(zkPath + "/" + file, null, null, true); - Path filename = dir.resolve(file); - log.info("Writing file {}", filename); - Files.write(filename, data); - } else { - downloadFromZK(zkClient, zkPath + "/" + file, dir.resolve(file)); + List children = zkClient.getChildren(zkPath, null, true); + // If it has no children, it's a leaf node, write the assoicated data from the ZNode. + // Otherwise, continue recursing, but write the associated data to a special file if any + if (children.size() == 0) { + // If we didn't copy data down, then we also didn't create the file. But we still need a marker on the local + // disk so create a dir. + if (copyDataDown(zkClient, zkPath, file.toFile()) == 0) { + Files.createDirectories(file); + } + } else { + Files.createDirectories(file); // Make parent dir. + // ZK nodes, whether leaf or not can have data. If it's a non-leaf node and + // has associated data write it into the special file. + copyDataDown(zkClient, zkPath, new File(file.toFile(), ZKNODE_DATA_FILE)); + + for (String child : children) { + String zkChild = zkPath; + if (zkChild.endsWith("/") == false) zkChild += "/"; + zkChild += child; + if (isEphemeral(zkClient, zkChild)) { // Don't copy ephemeral nodes + continue; + } + // Go deeper into the tree now + downloadFromZK(zkClient, zkChild, file.resolve(child)); } } } catch (KeeperException | InterruptedException e) { - throw new IOException("Error downloading files from zookeeper path " + zkPath + " to " + dir.toString(), + throw new IOException("Error downloading files from zookeeper path " + zkPath + " to " + file.toString(), SolrZkClient.checkInterrupted(e)); } } @@ -336,10 +392,24 @@ public class ZkMaintenanceUtils { if ("\\".equals(separator)) relativePath = relativePath.replaceAll("\\\\", "/"); // It's possible that the relative path and file are the same, in which case - // adding the bare slash is A Bad Idea - if (relativePath.length() == 0) return zkRoot; - - return zkRoot + "/" + relativePath; + // adding the bare slash is A Bad Idea unless it's a non-leaf data node + boolean isNonLeafData = file.toFile().getName().equals(ZKNODE_DATA_FILE); + if (relativePath.length() == 0 && isNonLeafData == false) return zkRoot; + + // Important to have this check if the source is file:whatever/ and the destination is just zk:/ + if (zkRoot.endsWith("/") == false) zkRoot += "/"; + + String ret = zkRoot + relativePath; + + // Special handling for data associated with non-leaf node. + if (isNonLeafData) { + // special handling since what we need to do is add the data to the parent. + ret = ret.substring(0, ret.indexOf(ZKNODE_DATA_FILE)); + if (ret.endsWith("/")) { + ret = ret.substring(0, ret.length() - 1); + } + } + return ret; } } From cb20eae1789442286f680f8dcfaf914394aed7a3 Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Mon, 27 Mar 2017 22:17:34 +0200 Subject: [PATCH 069/563] SOLR-10362: "Memory Pool not found" error when reporting JVM metrics. --- solr/CHANGES.txt | 2 ++ .../java/org/apache/solr/util/stats/MetricUtils.java | 11 ++++++++++- .../org/apache/solr/util/stats/MetricUtilsTest.java | 6 ++++++ 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 0b6faf83e92..b380e6b23da 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -137,6 +137,8 @@ Bug Fixes * SOLR-10108: bin/solr script recursive copy broken (Erick Erickson) +* SOLR-10362: "Memory Pool not found" error when reporting JVM metrics. (ab) + Other Changes ---------------------- 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 ba1284e3b72..9809070de6a 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 @@ -16,6 +16,7 @@ */ package org.apache.solr.util.stats; +import java.lang.invoke.MethodHandles; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; @@ -40,11 +41,14 @@ import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.metrics.AggregateMetric; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Metrics specific utility functions. */ public class MetricUtils { + private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); public static final String METRIC_NAME = "metric"; public static final String VALUE = "value"; @@ -208,7 +212,12 @@ public class MetricUtils { consumer.accept(n, convertCounter(counter, compact)); } else if (metric instanceof Gauge) { Gauge gauge = (Gauge) metric; - consumer.accept(n, convertGauge(gauge, compact)); + try { + consumer.accept(n, convertGauge(gauge, compact)); + } catch (InternalError ie) { + LOG.warn("Error converting gauge '" + n + "', possible JDK bug: SOLR-10362", ie); + consumer.accept(n, null); + } } else if (metric instanceof Meter) { Meter meter = (Meter) metric; consumer.accept(n, convertMeter(meter)); 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 35caef8af5d..fe9815711df 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 @@ -81,6 +81,8 @@ public class MetricUtilsTest extends SolrTestCaseJ4 { am.set("bar", 2); Gauge gauge = () -> "foobar"; registry.register("gauge", gauge); + Gauge error = () -> {throw new InternalError("expected error");}; + registry.register("expected.error", error); MetricUtils.toMaps(registry, Collections.singletonList(MetricFilter.ALL), MetricFilter.ALL, false, false, false, (k, o) -> { Map v = (Map)o; @@ -106,6 +108,8 @@ public class MetricUtilsTest extends SolrTestCaseJ4 { update = (Map)values.get("bar"); assertEquals(2, update.get("value")); assertEquals(2, update.get("updateCount")); + } else if (k.startsWith("expected.error")) { + assertNull(v); } }); // test compact format @@ -143,6 +147,8 @@ public class MetricUtilsTest extends SolrTestCaseJ4 { update = (Map)values.get("bar"); assertEquals(2, update.get("value")); assertEquals(2, update.get("updateCount")); + } else if (k.startsWith("expected.error")) { + assertNull(o); } else { Map v = (Map)o; assertEquals(1L, v.get("count")); From b46b8278c60102eeaeb45dddf239356746da7de8 Mon Sep 17 00:00:00 2001 From: Steve Rowe Date: Mon, 27 Mar 2017 23:53:55 -0400 Subject: [PATCH 070/563] SOLR-10344: Update Solr default/example and test configs to use WordDelimiterGraphFilterFactory --- solr/CHANGES.txt | 2 + .../solr/collection1/conf/schema.xml | 23 +++++-- .../solr/collection1/conf/synonyms.txt | 2 +- .../conf/dataimport-schema-no-unique-key.xml | 7 +- .../collection1/conf/dataimport-schema.xml | 17 +++-- .../conf/dataimport-solr_id-schema.xml | 23 +++++-- .../solr/collection1/conf/schema.xml | 40 ++++++++--- .../solr/collection1/conf/synonyms.txt | 2 +- .../uima/solr/collection1/conf/schema.xml | 37 +++++++--- .../uima/solr/collection1/conf/synonyms.txt | 2 +- .../uima/uima-tokenizers-schema.xml | 34 +++++++--- .../conf/schema-HighlighterMaxOffsetTest.xml | 5 +- .../conf/schema-copyfield-test.xml | 29 +++++--- .../collection1/conf/schema-docValuesJoin.xml | 5 +- .../solr/collection1/conf/schema-folding.xml | 2 +- .../solr/collection1/conf/schema-hash.xml | 59 +++++++++++----- .../collection1/conf/schema-psuedo-fields.xml | 5 +- .../conf/schema-required-fields.xml | 18 +++-- .../solr/collection1/conf/schema-rest.xml | 63 +++++++++++------ .../conf/schema-single-dynamic-copy-field.xml | 63 +++++++++++------ .../solr/collection1/conf/schema-sql.xml | 59 +++++++++++----- .../solr/collection1/conf/schema-trie.xml | 23 +++++-- .../solr/collection1/conf/schema.xml | 58 ++++++++++++---- .../solr/collection1/conf/schema11.xml | 27 ++++++-- .../solr/collection1/conf/schema12.xml | 63 +++++++++++------ .../solr/collection1/conf/schema15.xml | 63 +++++++++++------ .../solr/collection1/conf/schema_latest.xml | 27 ++++++-- .../solr/collection1/conf/schemasurround.xml | 68 +++++++++++++------ .../solr/collection1/conf/synonyms.txt | 2 +- .../solr/collection1/conf/wdftypes.txt | 2 +- .../configsets/cloud-dynamic/conf/schema.xml | 35 +++++++--- .../configsets/doc-expiry/conf/schema.xml | 35 +++++++--- .../org/apache/solr/ConvertedLegacyTest.java | 2 +- .../FieldAnalysisRequestHandlerTest.java | 14 ++-- .../solr/rest/schema/TestBulkSchemaAPI.java | 29 +++++++- .../example-DIH/solr/db/conf/managed-schema | 26 +++++-- .../example-DIH/solr/db/conf/synonyms.txt | 2 +- .../example-DIH/solr/mail/conf/managed-schema | 26 +++++-- .../example-DIH/solr/mail/conf/synonyms.txt | 2 +- .../example-DIH/solr/rss/conf/managed-schema | 31 ++++++--- .../example-DIH/solr/rss/conf/synonyms.txt | 2 +- .../example-DIH/solr/solr/conf/managed-schema | 26 +++++-- .../example-DIH/solr/solr/conf/synonyms.txt | 2 +- .../example-DIH/solr/tika/conf/managed-schema | 23 +++++-- solr/example/files/conf/managed-schema | 20 ++++-- solr/example/files/conf/synonyms.txt | 2 +- .../basic_configs/conf/managed-schema | 26 +++++-- .../basic_configs/conf/synonyms.txt | 2 +- .../conf/managed-schema | 26 +++++-- .../conf/synonyms.txt | 2 +- .../conf/managed-schema | 26 +++++-- .../conf/synonyms.txt | 2 +- .../solr/collection1/conf/schema-sql.xml | 59 +++++++++++----- .../solrj/solr/collection1/conf/schema.xml | 59 +++++++++++----- .../solr/configsets/streaming/conf/schema.xml | 56 ++++++++++----- .../solr/client/solrj/request/SchemaTest.java | 2 +- 56 files changed, 984 insertions(+), 383 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index b380e6b23da..3403c9082f1 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -151,6 +151,8 @@ Other Changes * SOLR-10352: bin/solr script now prints warning when available system entropy is lower than 300 (Esther Quansah via Ishan Chattopadhyaya) + +* SOLR-10344: Update Solr default/example and test configs to use WordDelimiterGraphFilterFactory. (Steve Rowe) ================== 6.5.0 ================== diff --git a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml index 0c06a48adc2..02626a0eba4 100644 --- a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml +++ b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml @@ -155,7 +155,7 @@ - - @@ -198,11 +199,23 @@ - + - + + + + + + + + + + diff --git a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/synonyms.txt b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/synonyms.txt index b0e31cb7ec8..26d237a5943 100644 --- a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/synonyms.txt +++ b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/synonyms.txt @@ -23,7 +23,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-schema-no-unique-key.xml b/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-schema-no-unique-key.xml index 068322e21b3..84d0dbe0a54 100644 --- a/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-schema-no-unique-key.xml +++ b/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-schema-no-unique-key.xml @@ -158,7 +158,7 @@ - - + - - + - - + - + + + + + + + + + + - + - + + + + + + - + - + + + + + + @@ -233,15 +248,16 @@ - + - @@ -253,13 +269,14 @@ - + - @@ -269,13 +286,14 @@ - + - diff --git a/solr/contrib/ltr/src/test-files/solr/collection1/conf/synonyms.txt b/solr/contrib/ltr/src/test-files/solr/collection1/conf/synonyms.txt index 0ef0e8daaba..461ed4df6e4 100644 --- a/solr/contrib/ltr/src/test-files/solr/collection1/conf/synonyms.txt +++ b/solr/contrib/ltr/src/test-files/solr/collection1/conf/synonyms.txt @@ -21,7 +21,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/schema.xml b/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/schema.xml index 5e0273e5f29..89d44e6616d 100644 --- a/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/schema.xml +++ b/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/schema.xml @@ -191,7 +191,7 @@ - - + @@ -224,7 +224,7 @@ ignoreCase="true" expand="true" /> - @@ -240,23 +240,36 @@ --> - + - + + + + + + + + + @@ -271,10 +284,11 @@ - + @@ -282,7 +296,7 @@ ignoreCase="true" expand="true" /> - @@ -301,13 +315,14 @@ - + @@ -315,7 +330,7 @@ ignoreCase="true" expand="true" /> - diff --git a/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/synonyms.txt b/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/synonyms.txt index b0e31cb7ec8..26d237a5943 100644 --- a/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/synonyms.txt +++ b/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/synonyms.txt @@ -23,7 +23,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/contrib/uima/src/test-files/uima/uima-tokenizers-schema.xml b/solr/contrib/uima/src/test-files/uima/uima-tokenizers-schema.xml index 0007be79d2b..229d69bd586 100644 --- a/solr/contrib/uima/src/test-files/uima/uima-tokenizers-schema.xml +++ b/solr/contrib/uima/src/test-files/uima/uima-tokenizers-schema.xml @@ -191,7 +191,7 @@ - + + - @@ -235,19 +237,29 @@ --> - + - + + + + + + + + @@ -277,14 +289,15 @@ positionIncrementGap="100"> - + - @@ -301,17 +314,18 @@ positionIncrementGap="100"> - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-HighlighterMaxOffsetTest.xml b/solr/core/src/test-files/solr/collection1/conf/schema-HighlighterMaxOffsetTest.xml index 808453c1766..122d4ce8a35 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-HighlighterMaxOffsetTest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-HighlighterMaxOffsetTest.xml @@ -67,16 +67,17 @@ Test for HighlighterMaxOffsetTest which requires the use of ReversedWildcardFilt - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml b/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml index 47ddeb2aeff..a9df7f80c9b 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml @@ -60,17 +60,25 @@ - + - + - + + + + + + @@ -229,15 +237,16 @@ - + - @@ -249,13 +258,14 @@ - + - @@ -265,13 +275,14 @@ - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml index 1d559bbf7ef..baed872dd7f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml @@ -42,11 +42,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -55,7 +56,7 @@ ignoreCase="true" words="stopwords.txt" /> - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml b/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml index cc9cae3f553..2b0bf32139a 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-folding.xml @@ -47,7 +47,7 @@ - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml b/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml index b1324738933..6a7987cd551 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-hash.xml @@ -72,26 +72,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -191,9 +206,16 @@ - + - + + + + + + @@ -265,15 +287,16 @@ - + - @@ -285,15 +308,16 @@ - + - @@ -304,8 +328,9 @@ - + @@ -318,13 +343,14 @@ - + - @@ -334,13 +360,14 @@ - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-psuedo-fields.xml b/solr/core/src/test-files/solr/collection1/conf/schema-psuedo-fields.xml index 20f2d2dd299..8491ce820a7 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-psuedo-fields.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-psuedo-fields.xml @@ -50,11 +50,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -63,7 +64,7 @@ ignoreCase="true" words="stopwords.txt" /> - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml b/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml index d3a78185507..c92b90156ac 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-required-fields.xml @@ -60,9 +60,10 @@ - + @@ -219,15 +220,16 @@ - + - @@ -239,13 +241,14 @@ - + - @@ -255,13 +258,14 @@ - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml index 8bd603bc6ca..7d9bf02bc42 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml @@ -45,26 +45,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -90,11 +105,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -103,7 +119,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -153,11 +169,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -166,7 +183,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -300,15 +317,16 @@ - + - @@ -320,16 +338,17 @@ - + - @@ -341,9 +360,10 @@ - + @@ -356,13 +376,14 @@ - + - @@ -372,13 +393,14 @@ - + - @@ -401,16 +423,17 @@ --> - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-single-dynamic-copy-field.xml b/solr/core/src/test-files/solr/collection1/conf/schema-single-dynamic-copy-field.xml index f7cbf6dfd6c..72dc723b66b 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-single-dynamic-copy-field.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-single-dynamic-copy-field.xml @@ -45,26 +45,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -90,11 +105,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -103,7 +119,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -153,11 +169,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -166,7 +183,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -300,15 +317,16 @@ - + - @@ -320,16 +338,17 @@ - + - @@ -341,9 +360,10 @@ - + @@ -356,13 +376,14 @@ - + - @@ -372,13 +393,14 @@ - + - @@ -401,16 +423,17 @@ --> - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-sql.xml b/solr/core/src/test-files/solr/collection1/conf/schema-sql.xml index 818a6c8624d..dbaac576533 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-sql.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-sql.xml @@ -72,26 +72,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -191,9 +206,16 @@ - + - + + + + + + @@ -265,15 +287,16 @@ - + - @@ -285,15 +308,16 @@ - + - @@ -304,8 +328,9 @@ - + @@ -318,13 +343,14 @@ - + - @@ -334,13 +360,14 @@ - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml b/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml index 1df0a0fc252..375036f0fe3 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml @@ -151,7 +151,7 @@ - - + - @@ -189,11 +190,23 @@ - + - + + + + + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema.xml b/solr/core/src/test-files/solr/collection1/conf/schema.xml index 6f5eddcaddf..23ac3260078 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema.xml @@ -55,26 +55,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -196,7 +211,15 @@ - + + + + + + + + - + - @@ -294,16 +318,17 @@ - + - @@ -315,9 +340,10 @@ - + @@ -330,13 +356,14 @@ - + - @@ -346,13 +373,14 @@ - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema11.xml b/solr/core/src/test-files/solr/collection1/conf/schema11.xml index 7591c96d7c5..db0770baf3b 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml @@ -147,7 +147,7 @@ - - + + - + @@ -183,11 +184,22 @@ - + - + + + + + + + + + + + + @@ -417,17 +429,18 @@ valued. --> --> - + + - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema12.xml b/solr/core/src/test-files/solr/collection1/conf/schema12.xml index 85774408763..db913771f8c 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema12.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema12.xml @@ -54,26 +54,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -99,11 +114,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -112,7 +128,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -129,11 +145,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -142,7 +159,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -270,15 +287,16 @@ - + - @@ -290,16 +308,17 @@ - + - @@ -311,9 +330,10 @@ - + @@ -326,13 +346,14 @@ - + - @@ -342,13 +363,14 @@ - + - @@ -371,16 +393,17 @@ --> - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema15.xml b/solr/core/src/test-files/solr/collection1/conf/schema15.xml index e2c14f0f76b..8fb8d4410b2 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema15.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema15.xml @@ -46,26 +46,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -91,11 +106,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -104,7 +120,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -121,11 +137,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -134,7 +151,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -268,15 +285,16 @@ - + - @@ -288,16 +306,17 @@ - + - @@ -309,9 +328,10 @@ - + @@ -324,13 +344,14 @@ - + - @@ -340,13 +361,14 @@ - + - @@ -369,16 +391,17 @@ --> - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml index dfeac32c167..1163ab5a01d 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml @@ -541,7 +541,7 @@ - + - + possible with WordDelimiterGraphFilter in conjuncton with stemming. --> + + + + + + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml b/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml index e22eb2bdc9f..30719685672 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml @@ -47,26 +47,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -92,11 +107,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -105,7 +121,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -122,11 +138,12 @@ ignoreCase="true" words="stopwords.txt" /> - + @@ -135,7 +152,7 @@ ignoreCase="true" words="stopwords.txt" /> - @@ -263,15 +280,16 @@ - + - @@ -283,16 +301,17 @@ - + - @@ -304,9 +323,10 @@ - + @@ -319,13 +339,14 @@ - + - @@ -335,13 +356,14 @@ - + - @@ -364,16 +386,17 @@ --> - + - @@ -384,15 +407,16 @@ - + - diff --git a/solr/core/src/test-files/solr/collection1/conf/synonyms.txt b/solr/core/src/test-files/solr/collection1/conf/synonyms.txt index 340abd7847c..f64cb724a57 100644 --- a/solr/core/src/test-files/solr/collection1/conf/synonyms.txt +++ b/solr/core/src/test-files/solr/collection1/conf/synonyms.txt @@ -23,7 +23,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/core/src/test-files/solr/collection1/conf/wdftypes.txt b/solr/core/src/test-files/solr/collection1/conf/wdftypes.txt index 7378b0802e7..9a453c8da9f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/wdftypes.txt +++ b/solr/core/src/test-files/solr/collection1/conf/wdftypes.txt @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # -# A customized type mapping for WordDelimiterFilterFactory +# A customized type mapping for WordDelimiterGraphFilterFactory # the allowable types are: LOWER, UPPER, ALPHA, DIGIT, ALPHANUM, SUBWORD_DELIM # # the default for any character without a mapping is always computed from diff --git a/solr/core/src/test-files/solr/configsets/cloud-dynamic/conf/schema.xml b/solr/core/src/test-files/solr/configsets/cloud-dynamic/conf/schema.xml index af201c0364b..d15c19908ca 100644 --- a/solr/core/src/test-files/solr/configsets/cloud-dynamic/conf/schema.xml +++ b/solr/core/src/test-files/solr/configsets/cloud-dynamic/conf/schema.xml @@ -45,26 +45,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -86,14 +101,15 @@ - + - @@ -111,14 +127,15 @@ - + - diff --git a/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml b/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml index 8400fe81d65..c622eba8a9f 100644 --- a/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml +++ b/solr/core/src/test-files/solr/configsets/doc-expiry/conf/schema.xml @@ -45,26 +45,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -86,14 +101,15 @@ - + - @@ -111,14 +127,15 @@ - + - diff --git a/solr/core/src/test/org/apache/solr/ConvertedLegacyTest.java b/solr/core/src/test/org/apache/solr/ConvertedLegacyTest.java index 767b811964d..bf7925a318d 100644 --- a/solr/core/src/test/org/apache/solr/ConvertedLegacyTest.java +++ b/solr/core/src/test/org/apache/solr/ConvertedLegacyTest.java @@ -872,7 +872,7 @@ public class ConvertedLegacyTest extends SolrTestCaseJ4 { ); - // intra-word delimiter testing (WordDelimiterFilter) + // intra-word delimiter testing (WordDelimiterGraphFilter) assertU("42foo bar"); assertU(""); diff --git a/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java index d2ef5553333..fc0f6bedbed 100644 --- a/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/handler/FieldAnalysisRequestHandlerTest.java @@ -382,7 +382,7 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB } @Test - public void testPositionHistoryWithWDF() throws Exception { + public void testPositionHistoryWithWDGF() throws Exception { FieldAnalysisRequest request = new FieldAnalysisRequest(); request.addFieldType("skutype1"); @@ -407,12 +407,12 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB assertToken(tokenList.get(1), new TokenInfo("3456-12", null, "word", 4, 11, 2, new int[]{2}, null, false)); assertToken(tokenList.get(2), new TokenInfo("a", null, "word", 12, 13, 3, new int[]{3}, null, false)); assertToken(tokenList.get(3), new TokenInfo("Test", null, "word", 14, 18, 4, new int[]{4}, null, false)); - tokenList = indexPart.get("org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter"); - assertNotNull("Expcting WordDelimiterFilter analysis breakdown", tokenList); + tokenList = indexPart.get("org.apache.lucene.analysis.miscellaneous.WordDelimiterGraphFilter"); + assertNotNull("Expcting WordDelimiterGraphFilter analysis breakdown", tokenList); assertEquals(6, tokenList.size()); assertToken(tokenList.get(0), new TokenInfo("hi", null, "word", 0, 2, 1, new int[]{1,1}, null, false)); - assertToken(tokenList.get(1), new TokenInfo("3456", null, "word", 4, 8, 2, new int[]{2,2}, null, false)); - assertToken(tokenList.get(2), new TokenInfo("345612", null, "word", 4, 11, 2, new int[]{2,2}, null, false)); + assertToken(tokenList.get(1), new TokenInfo("345612", null, "word", 4, 11, 2, new int[]{2,2}, null, false)); + assertToken(tokenList.get(2), new TokenInfo("3456", null, "word", 4, 8, 2, new int[]{2,2}, null, false)); assertToken(tokenList.get(3), new TokenInfo("12", null, "word", 9, 11, 3, new int[]{2,3}, null, false)); assertToken(tokenList.get(4), new TokenInfo("a", null, "word", 12, 13, 4, new int[]{3,4}, null, false)); assertToken(tokenList.get(5), new TokenInfo("Test", null, "word", 14, 18, 5, new int[]{4,5}, null, false)); @@ -420,8 +420,8 @@ public class FieldAnalysisRequestHandlerTest extends AnalysisRequestHandlerTestB assertNotNull("Expcting LowerCaseFilter analysis breakdown", tokenList); assertEquals(6, tokenList.size()); assertToken(tokenList.get(0), new TokenInfo("hi", null, "word", 0, 2, 1, new int[]{1,1,1}, null, false)); - assertToken(tokenList.get(1), new TokenInfo("3456", null, "word", 4, 8, 2, new int[]{2,2,2}, null, false)); - assertToken(tokenList.get(2), new TokenInfo("345612", null, "word", 4, 11, 2, new int[]{2,2,2}, null, false)); + assertToken(tokenList.get(1), new TokenInfo("345612", null, "word", 4, 11, 2, new int[]{2,2,2}, null, false)); + assertToken(tokenList.get(2), new TokenInfo("3456", null, "word", 4, 8, 2, new int[]{2,2,2}, null, false)); assertToken(tokenList.get(3), new TokenInfo("12", null, "word", 9, 11, 3, new int[]{2,3,3}, null, false)); assertToken(tokenList.get(4), new TokenInfo("a", null, "word", 12, 13, 4, new int[]{3,4,4}, null, false)); assertToken(tokenList.get(5), new TokenInfo("test", null, "word", 14, 18, 5, new int[]{4,5,5}, null, false)); diff --git a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java index 166d1fc183c..9f379677098 100644 --- a/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java +++ b/solr/core/src/test/org/apache/solr/rest/schema/TestBulkSchemaAPI.java @@ -396,7 +396,7 @@ public class TestBulkSchemaAPI extends RestTestBase { " 'name' : 'myNewTxtField',\n" + " 'class':'solr.TextField',\n" + " 'positionIncrementGap':'100',\n" + - " 'analyzer' : {\n" + + " 'indexAnalyzer' : {\n" + " 'charFilters':[\n" + " {\n" + " 'class':'solr.PatternReplaceCharFilterFactory',\n" + @@ -407,7 +407,32 @@ public class TestBulkSchemaAPI extends RestTestBase { " 'tokenizer':{'class':'solr.WhitespaceTokenizerFactory'},\n" + " 'filters':[\n" + " {\n" + - " 'class':'solr.WordDelimiterFilterFactory',\n" + + " 'class':'solr.WordDelimiterGraphFilterFactory',\n" + + " 'preserveOriginal':'0'\n" + + " },\n" + + " {\n" + + " 'class':'solr.StopFilterFactory',\n" + + " 'words':'stopwords.txt',\n" + + " 'ignoreCase':'true'\n" + + " },\n" + + " {'class':'solr.LowerCaseFilterFactory'},\n" + + " {'class':'solr.ASCIIFoldingFilterFactory'},\n" + + " {'class':'solr.KStemFilterFactory'},\n" + + " {'class':'solr.FlattenGraphFilterFactory'}\n" + + " ]\n" + + " },\n" + + " 'queryAnalyzer' : {\n" + + " 'charFilters':[\n" + + " {\n" + + " 'class':'solr.PatternReplaceCharFilterFactory',\n" + + " 'replacement':'$1$1',\n" + + " 'pattern':'([a-zA-Z])\\\\\\\\1+'\n" + + " }\n" + + " ],\n" + + " 'tokenizer':{'class':'solr.WhitespaceTokenizerFactory'},\n" + + " 'filters':[\n" + + " {\n" + + " 'class':'solr.WordDelimiterGraphFilterFactory',\n" + " 'preserveOriginal':'0'\n" + " },\n" + " {\n" + diff --git a/solr/example/example-DIH/solr/db/conf/managed-schema b/solr/example/example-DIH/solr/db/conf/managed-schema index eead56f7b07..1a1012fe850 100644 --- a/solr/example/example-DIH/solr/db/conf/managed-schema +++ b/solr/example/example-DIH/solr/db/conf/managed-schema @@ -500,7 +500,7 @@ - + - + + possible with WordDelimiterGraphFilter in conjuncton with stemming. --> + + + + + + + + + + + + diff --git a/solr/example/example-DIH/solr/db/conf/synonyms.txt b/solr/example/example-DIH/solr/db/conf/synonyms.txt index 7f72128303b..eab4ee87537 100644 --- a/solr/example/example-DIH/solr/db/conf/synonyms.txt +++ b/solr/example/example-DIH/solr/db/conf/synonyms.txt @@ -21,7 +21,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/example/example-DIH/solr/mail/conf/managed-schema b/solr/example/example-DIH/solr/mail/conf/managed-schema index 076f83f6e6d..016f105513c 100644 --- a/solr/example/example-DIH/solr/mail/conf/managed-schema +++ b/solr/example/example-DIH/solr/mail/conf/managed-schema @@ -419,7 +419,7 @@ - + - + + possible with WordDelimiterGraphFilter in conjuncton with stemming. --> + + + + + + + + + + + + diff --git a/solr/example/example-DIH/solr/mail/conf/synonyms.txt b/solr/example/example-DIH/solr/mail/conf/synonyms.txt index 7f72128303b..eab4ee87537 100644 --- a/solr/example/example-DIH/solr/mail/conf/synonyms.txt +++ b/solr/example/example-DIH/solr/mail/conf/synonyms.txt @@ -21,7 +21,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/example/example-DIH/solr/rss/conf/managed-schema b/solr/example/example-DIH/solr/rss/conf/managed-schema index e35f49d42c1..2064c5895b0 100644 --- a/solr/example/example-DIH/solr/rss/conf/managed-schema +++ b/solr/example/example-DIH/solr/rss/conf/managed-schema @@ -242,18 +242,19 @@ --> - + + - + @@ -450,7 +451,7 @@ - + - + + possible with WordDelimiterGraphFilter in conjuncton with stemming. --> + + + + + + + + + + + + diff --git a/solr/example/example-DIH/solr/rss/conf/synonyms.txt b/solr/example/example-DIH/solr/rss/conf/synonyms.txt index 7f72128303b..eab4ee87537 100644 --- a/solr/example/example-DIH/solr/rss/conf/synonyms.txt +++ b/solr/example/example-DIH/solr/rss/conf/synonyms.txt @@ -21,7 +21,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/example/example-DIH/solr/solr/conf/managed-schema b/solr/example/example-DIH/solr/solr/conf/managed-schema index 6be0ad9a11e..04c85c08bb5 100644 --- a/solr/example/example-DIH/solr/solr/conf/managed-schema +++ b/solr/example/example-DIH/solr/solr/conf/managed-schema @@ -500,7 +500,7 @@ - + - + + possible with WordDelimiterGraphFilter in conjuncton with stemming. --> + + + + + + + + + + + + diff --git a/solr/example/example-DIH/solr/solr/conf/synonyms.txt b/solr/example/example-DIH/solr/solr/conf/synonyms.txt index 7f72128303b..eab4ee87537 100644 --- a/solr/example/example-DIH/solr/solr/conf/synonyms.txt +++ b/solr/example/example-DIH/solr/solr/conf/synonyms.txt @@ -21,7 +21,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/example/example-DIH/solr/tika/conf/managed-schema b/solr/example/example-DIH/solr/tika/conf/managed-schema index c4dccb208e9..58b2a80b6e1 100644 --- a/solr/example/example-DIH/solr/tika/conf/managed-schema +++ b/solr/example/example-DIH/solr/tika/conf/managed-schema @@ -353,7 +353,7 @@ - + - + + possible with WordDelimiterGraphFilter in conjuncton with stemming. --> + + + + + + + + + diff --git a/solr/example/files/conf/managed-schema b/solr/example/files/conf/managed-schema index e936bcde4e3..ff209be05dc 100644 --- a/solr/example/files/conf/managed-schema +++ b/solr/example/files/conf/managed-schema @@ -145,27 +145,39 @@ - + + - + - + - + + + + + + + + + + + + diff --git a/solr/example/files/conf/synonyms.txt b/solr/example/files/conf/synonyms.txt index 7f72128303b..eab4ee87537 100644 --- a/solr/example/files/conf/synonyms.txt +++ b/solr/example/files/conf/synonyms.txt @@ -21,7 +21,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/server/solr/configsets/basic_configs/conf/managed-schema b/solr/server/solr/configsets/basic_configs/conf/managed-schema index d7aacc48644..09aaae31d53 100644 --- a/solr/server/solr/configsets/basic_configs/conf/managed-schema +++ b/solr/server/solr/configsets/basic_configs/conf/managed-schema @@ -436,7 +436,7 @@ - + - + + possible with WordDelimiterGraphFilter in conjuncton with stemming. --> + + + + + + + + + + + + diff --git a/solr/server/solr/configsets/basic_configs/conf/synonyms.txt b/solr/server/solr/configsets/basic_configs/conf/synonyms.txt index 7f72128303b..eab4ee87537 100644 --- a/solr/server/solr/configsets/basic_configs/conf/synonyms.txt +++ b/solr/server/solr/configsets/basic_configs/conf/synonyms.txt @@ -21,7 +21,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema index b716f9c6d06..0319eb05b45 100644 --- a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema +++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema @@ -435,7 +435,7 @@ - + - + + possible with WordDelimiterGraphFilter in conjuncton with stemming. --> + + + + + + + + + + + + diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/synonyms.txt b/solr/server/solr/configsets/data_driven_schema_configs/conf/synonyms.txt index 7f72128303b..eab4ee87537 100644 --- a/solr/server/solr/configsets/data_driven_schema_configs/conf/synonyms.txt +++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/synonyms.txt @@ -21,7 +21,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema index 9caf3d6e2b0..17dadd48159 100644 --- a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema +++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema @@ -544,7 +544,7 @@ - + - + + possible with WordDelimiterGraphFilter in conjuncton with stemming. --> + + + + + + + + + + + + diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/synonyms.txt b/solr/server/solr/configsets/sample_techproducts_configs/conf/synonyms.txt index 7f72128303b..eab4ee87537 100644 --- a/solr/server/solr/configsets/sample_techproducts_configs/conf/synonyms.txt +++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/synonyms.txt @@ -21,7 +21,7 @@ fooaaa,baraaa,bazaaa GB,gib,gigabyte,gigabytes MB,mib,megabyte,megabytes Television, Televisions, TV, TVs -#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#notice we use "gib" instead of "GiB" so any WordDelimiterGraphFilter coming #after us won't split it into two words. # Synonym mappings can be used for spelling correction too diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-sql.xml b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-sql.xml index 56bf625b967..96bbcd817e8 100644 --- a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-sql.xml +++ b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema-sql.xml @@ -72,26 +72,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -191,9 +206,16 @@ - + - + + + + + + @@ -265,15 +287,16 @@ - + - @@ -285,15 +308,16 @@ - + - @@ -304,8 +328,9 @@ - + @@ -318,13 +343,14 @@ - + - @@ -334,13 +360,14 @@ - + - diff --git a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema.xml index 49f08b876d6..fb1a1ac6625 100644 --- a/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema.xml +++ b/solr/solrj/src/test-files/solrj/solr/collection1/conf/schema.xml @@ -47,26 +47,41 @@ - + - + - + + + + + + - + - + + + + + + @@ -166,9 +181,16 @@ - + - + + + + + + @@ -240,15 +262,16 @@ - + - @@ -260,15 +283,16 @@ - + - @@ -279,8 +303,9 @@ - + @@ -293,13 +318,14 @@ - + - @@ -309,13 +335,14 @@ - + - diff --git a/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml b/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml index b61a2e93ee8..a3b6a4e6c8e 100644 --- a/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml +++ b/solr/solrj/src/test-files/solrj/solr/configsets/streaming/conf/schema.xml @@ -73,24 +73,37 @@ - + + - + - + + + + + + + - + - + + + + + + + @@ -179,9 +192,15 @@ - + - + + + + + + + @@ -252,14 +271,15 @@ - + + - + @@ -270,14 +290,15 @@ - + + - + @@ -287,7 +308,8 @@ - + + @@ -300,12 +322,13 @@ - + + - + @@ -314,12 +337,13 @@ - + + - + diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java index 0826c27259a..3bbcff1c8d3 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/SchemaTest.java @@ -529,7 +529,7 @@ public class SchemaTest extends RestTestBase { tokenizerAttributes.put("class", "solr.WhitespaceTokenizerFactory"); analyzerDefinition.setTokenizer(tokenizerAttributes); Map filterAttributes = new LinkedHashMap<>(); - filterAttributes.put("class", "solr.WordDelimiterFilterFactory"); + filterAttributes.put("class", "solr.WordDelimiterGraphFilterFactory"); filterAttributes.put("preserveOriginal", "0"); analyzerDefinition.setFilters(Collections.singletonList(filterAttributes)); fieldTypeDefinition.setAnalyzer(analyzerDefinition); From 68eb078af854fbb923dee4a93541c7ff0fd0b982 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20H=C3=B8ydahl?= Date: Tue, 28 Mar 2017 14:24:09 +0200 Subject: [PATCH 071/563] SOLR-10369: bin\solr.cmd delete and healthcheck now works again (fixed continuation chars ^) --- solr/CHANGES.txt | 2 ++ solr/bin/solr.cmd | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 3403c9082f1..b68c62fa97d 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -139,6 +139,8 @@ Bug Fixes * SOLR-10362: "Memory Pool not found" error when reporting JVM metrics. (ab) +* SOLR-10369: bin\solr.cmd delete and healthcheck now works again; fixed continuation chars ^ (Luis Goes via janhoy) + Other Changes ---------------------- diff --git a/solr/bin/solr.cmd b/solr/bin/solr.cmd index ee76a218e73..2aa4f0073b4 100644 --- a/solr/bin/solr.cmd +++ b/solr/bin/solr.cmd @@ -1234,7 +1234,7 @@ goto parse_healthcheck_args :run_healthcheck IF NOT DEFINED HEALTHCHECK_COLLECTION goto healthcheck_usage IF NOT DEFINED HEALTHCHECK_ZK_HOST set "HEALTHCHECK_ZK_HOST=localhost:9983" -"%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^ +"%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^ -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI healthcheck -collection !HEALTHCHECK_COLLECTION! -zkHost !HEALTHCHECK_ZK_HOST! @@ -1428,7 +1428,7 @@ if "!DELETE_CONFIG!"=="" ( set DELETE_CONFIG=true ) -"%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^ +"%JAVA%" %SOLR_SSL_OPTS% %AUTHC_OPTS% %SOLR_ZK_CREDS_AND_ACLS% -Dsolr.install.dir="%SOLR_TIP%" ^ -Dlog4j.configuration="file:%DEFAULT_SERVER_DIR%\scripts\cloud-scripts\log4j.properties" ^ -classpath "%DEFAULT_SERVER_DIR%\solr-webapp\webapp\WEB-INF\lib\*;%DEFAULT_SERVER_DIR%\lib\ext\*" ^ org.apache.solr.util.SolrCLI delete -name !DELETE_NAME! -deleteConfig !DELETE_CONFIG! ^ From 53064e46ddfc94a0b0e1d9c9f3e94786fb6701cd Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 28 Mar 2017 15:15:45 +0200 Subject: [PATCH 072/563] LUCENE-7754: Inner classes should be static whenever possible. --- lucene/CHANGES.txt | 6 +++++- .../charfilter/HTMLStripCharFilter.java | 2 +- .../charfilter/HTMLStripCharFilter.jflex | 7 +++---- .../lucene/analysis/shingle/ShingleFilter.java | 2 +- .../analysis/hunspell/TestDictionary.java | 2 +- .../miscellaneous/TestWordDelimiterFilter.java | 2 +- .../TestWordDelimiterGraphFilter.java | 2 +- .../NumericPayloadTokenFilterTest.java | 2 +- .../payloads/TypeAsPayloadTokenFilterTest.java | 2 +- .../analysis/sinks/TestTeeSinkTokenFilter.java | 4 ++-- .../lucene/analysis/snowball/TestSnowball.java | 2 +- .../standard/TestUAX29URLEmailTokenizer.java | 4 ++-- .../analysis/icu/ICUTransformFilter.java | 2 +- .../CachingNaiveBayesClassifierTest.java | 2 +- .../SimpleNaiveBayesClassifierTest.java | 2 +- .../bloom/BloomFilteringPostingsFormat.java | 8 ++++---- .../codecs/memory/MemoryPostingsFormat.java | 2 +- .../SimpleTextTermVectorsReader.java | 2 +- .../CompressingTermVectorsReader.java | 2 +- .../lucene/index/ParallelLeafReader.java | 2 +- .../lucene/search/ConstantScoreQuery.java | 2 +- .../org/apache/lucene/index/TestCodecs.java | 8 ++++---- .../index/TestFlushByRamOrCountsPolicy.java | 2 +- .../index/TestIndexWriterExceptions.java | 2 +- .../lucene/index/TestIndexWriterMerging.java | 2 +- .../index/TestIndexWriterWithThreads.java | 2 +- .../lucene/index/TestIndexableField.java | 2 +- .../lucene/index/TestMaxTermFrequency.java | 2 +- .../org/apache/lucene/index/TestNorms.java | 2 +- .../lucene/index/TestPerSegmentDeletes.java | 2 +- .../lucene/index/TestStressIndexing2.java | 2 +- .../lucene/index/TestTermVectorsReader.java | 2 +- .../lucene/index/TestTransactionRollback.java | 6 +++--- .../lucene/index/TestUniqueTermCount.java | 2 +- .../lucene/search/TestCustomSearcherSort.java | 4 ++-- .../TestEarlyTerminatingSortingCollector.java | 4 ++-- .../apache/lucene/search/TestPrefixRandom.java | 4 ++-- .../lucene/search/TestRegexpRandom2.java | 2 +- .../lucene/search/TestSimilarityProvider.java | 6 +++--- .../apache/lucene/search/TestTermScorer.java | 2 +- .../search/TestTimeLimitingCollector.java | 2 +- .../apache/lucene/store/TestLockFactory.java | 4 ++-- .../lucene/util/TestCloseableThreadLocal.java | 2 +- .../apache/lucene/util/TestQueryBuilder.java | 2 +- .../org/apache/lucene/util/fst/TestFSTs.java | 2 +- .../taxonomy/writercache/CollisionMap.java | 2 +- .../grouping/GroupFacetCollectorTest.java | 4 ++-- .../TestUnifiedHighlighterExtensibility.java | 2 +- .../lucene/queries/CustomScoreQuery.java | 2 +- .../lucene/queries/TestCustomScoreQuery.java | 2 +- .../queries/payloads/TestPayloadSpans.java | 6 +++--- .../queryparser/classic/TestMultiAnalyzer.java | 8 ++++---- .../classic/TestMultiFieldQueryParser.java | 2 +- .../standard/TestMultiAnalyzerQPHelper.java | 8 ++++---- .../flexible/standard/TestQPHelper.java | 8 ++++---- .../queryparser/util/QueryParserTestBase.java | 10 +++++----- .../sandbox/queries/FuzzyLikeThisQuery.java | 2 +- .../lucene/payloads/TestPayloadSpanUtil.java | 4 ++-- .../search/spell/WordBreakSpellChecker.java | 10 +++++----- .../search/suggest/document/ContextQuery.java | 2 +- .../search/suggest/fst/ExternalRefSorter.java | 2 +- .../suggest/fst/WFSTCompletionLookup.java | 2 +- .../search/suggest/jaspell/JaspellLookup.java | 8 ++++---- .../jaspell/JaspellTernarySearchTrie.java | 2 +- .../search/suggest/DocumentDictionaryTest.java | 2 +- .../analyzing/AnalyzingInfixSuggesterTest.java | 2 +- .../suggest/analyzing/FuzzySuggesterTest.java | 2 +- .../lucene/store/BaseLockFactoryTestCase.java | 2 +- .../dependencies/GetMavenDependenciesTask.java | 2 +- .../validation/LibVersionsCheckTask.java | 2 +- .../dataimport/MailEntityProcessor.java | 4 ++-- .../handler/dataimport/VariableResolver.java | 2 +- .../dataimport/AbstractDIHCacheTestCase.java | 2 +- .../AbstractSqlEntityProcessorTestCase.java | 4 ++-- .../TestContentStreamDataSource.java | 2 +- .../handler/extraction/XLSXResponseWriter.java | 2 +- .../org/apache/solr/ltr/LTRScoringQuery.java | 2 +- .../solr/ltr/TestLTRReRankingPipeline.java | 2 +- .../solr/response/VelocityResponseWriter.java | 2 +- .../java/org/apache/solr/cloud/Overseer.java | 2 +- .../apache/solr/cloud/OverseerTaskQueue.java | 2 +- .../org/apache/solr/cloud/ZkController.java | 2 +- .../solr/core/CachingDirectoryFactory.java | 2 +- .../apache/solr/handler/CdcrReplicator.java | 2 +- .../solr/handler/CdcrReplicatorState.java | 6 +++--- .../org/apache/solr/handler/ExportWriter.java | 18 +++++++++--------- .../org/apache/solr/handler/IndexFetcher.java | 4 ++-- .../org/apache/solr/handler/SQLHandler.java | 2 +- .../handler/component/ExpandComponent.java | 6 +++--- .../component/HttpShardHandlerFactory.java | 2 +- .../PivotFacetFieldValueCollection.java | 4 ++-- .../solr/handler/component/TermsComponent.java | 2 +- .../solr/highlight/DefaultSolrHighlighter.java | 2 +- .../apache/solr/schema/ManagedIndexSchema.java | 4 ++-- .../apache/solr/schema/RandomSortField.java | 2 +- .../search/ComplexPhraseQParserPlugin.java | 4 ++-- .../solr/search/ExportQParserPlugin.java | 2 +- .../solr/search/ExtendedDismaxQParser.java | 4 ++-- .../apache/solr/search/HashQParserPlugin.java | 18 +++++++++--------- .../apache/solr/search/ReRankCollector.java | 2 +- .../solr/search/ReRankQParserPlugin.java | 2 +- .../solr/search/facet/FacetFieldMerger.java | 2 +- .../apache/solr/search/facet/UniqueAgg.java | 2 +- .../search/function/CollapseScoreFunction.java | 2 +- .../apache/solr/search/join/GraphQuery.java | 2 +- .../solr/spelling/PossibilityIterator.java | 6 +++--- .../java/org/apache/solr/update/UpdateLog.java | 2 +- .../src/java/org/apache/solr/util/RTimer.java | 2 +- .../org/apache/solr/util/SimplePostTool.java | 4 ++-- .../solr/cloud/BaseCdcrDistributedZkTest.java | 4 ++-- .../cloud/ChaosMonkeyNothingIsSafeTest.java | 4 ++-- .../solr/cloud/ConnectionManagerTest.java | 2 +- .../solr/cloud/DistributedQueueTest.java | 2 +- .../apache/solr/cloud/LeaderElectionTest.java | 4 ++-- ...riddenZkACLAndCredentialsProvidersTest.java | 4 ++-- ...erseerCollectionConfigSetProcessorTest.java | 2 +- .../apache/solr/cloud/ZkSolrClientTest.java | 2 +- .../solr/core/CachingDirectoryFactoryTest.java | 2 +- .../AnalysisRequestHandlerTestBase.java | 2 +- .../solr/handler/PingRequestHandlerTest.java | 2 +- .../handler/XmlUpdateRequestHandlerTest.java | 2 +- .../CoreMergeIndexesAdminHandlerTest.java | 2 +- .../solr/internal/csv/CSVParserTest.java | 2 +- .../reporters/SolrGraphiteReporterTest.java | 2 +- .../response/TestGraphMLResponseWriter.java | 2 +- .../apache/solr/rest/TestManagedResource.java | 4 ++-- .../org/apache/solr/rest/TestRestManager.java | 2 +- .../TestCloudManagedSchemaConcurrent.java | 2 +- .../search/AnalyticsTestQParserPlugin.java | 10 +++++----- .../solr/search/TestExtendedDismaxParser.java | 10 +++++----- .../org/apache/solr/search/TestFiltering.java | 2 +- .../solr/search/TestRankQueryPlugin.java | 14 +++++++------- .../ConjunctionSolrSpellCheckerTest.java | 4 ++-- .../spelling/IndexBasedSpellCheckerTest.java | 2 +- .../store/blockcache/BlockDirectoryTest.java | 2 +- .../apache/solr/util/SimplePostToolTest.java | 2 +- .../apache/solr/util/SolrPluginUtilsTest.java | 2 +- .../solr/util/TestSolrCLIRunExample.java | 2 +- .../solrj/impl/ConcurrentUpdateSolrClient.java | 2 +- .../solr/client/solrj/io/ModelCache.java | 4 ++-- .../solrj/io/graph/GatherNodesStream.java | 2 +- .../solrj/io/graph/ShortestPathStream.java | 2 +- .../client/solrj/io/ops/GroupOperation.java | 2 +- .../client/solrj/io/stream/RankStream.java | 2 +- .../io/stream/SignificantTermsStream.java | 2 +- .../solrj/io/stream/TextLogitStream.java | 2 +- .../solrj/response/SpellCheckResponse.java | 4 ++-- .../solr/common/cloud/ZkStateReader.java | 2 +- .../client/solrj/TestLBHttpSolrClient.java | 2 +- .../embedded/SolrExampleStreamingTest.java | 4 ++-- .../impl/ConcurrentUpdateSolrClientTest.java | 6 +++--- .../java/org/apache/solr/SolrTestCaseJ4.java | 2 +- 152 files changed, 253 insertions(+), 250 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 15f4ff3458a..e6da58639d3 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -81,7 +81,11 @@ Other * LUCENE-7681: MemoryIndex uses new DocValues API (Alan Woodward) ======================= Lucene 6.6.0 ======================= -(No Changes) + +Other + +* LUCENE-7754: Inner classes should be static whenever possible. + (Daniel Jelinski via Adrien Grand) ======================= Lucene 6.5.0 ======================= diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.java index ba44dd8c40e..a2364975168 100644 --- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.java +++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.java @@ -30021,7 +30021,7 @@ public final class HTMLStripCharFilter extends BaseCharFilter { return ZZ_BUFFERSIZE; } - private class TextSegment extends OpenStringBuilder { + private static class TextSegment extends OpenStringBuilder { /** The position from which the next char will be read. */ int pos = 0; diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.jflex b/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.jflex index 98c39462179..d810d79babf 100644 --- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.jflex +++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/HTMLStripCharFilter.jflex @@ -19,13 +19,12 @@ package org.apache.lucene.analysis.charfilter; import java.io.IOException; import java.io.Reader; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Set; -import org.apache.lucene.analysis.util.CharArrayMap; -import org.apache.lucene.analysis.util.CharArraySet; +import org.apache.lucene.analysis.CharArrayMap; +import org.apache.lucene.analysis.CharArraySet; import org.apache.lucene.analysis.util.OpenStringBuilder; /** @@ -238,7 +237,7 @@ InlineElment = ( [aAbBiIqQsSuU] | return ZZ_BUFFERSIZE; } - private class TextSegment extends OpenStringBuilder { + private static class TextSegment extends OpenStringBuilder { /** The position from which the next char will be read. */ int pos = 0; diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java index e3fa8033cf2..0775cf7d936 100644 --- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java +++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/shingle/ShingleFilter.java @@ -583,7 +583,7 @@ public final class ShingleFilter extends TokenFilter { } } - private class InputWindowToken { + private static class InputWindowToken { final AttributeSource attSource; final CharTermAttribute termAtt; final OffsetAttribute offsetAtt; diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java index 5e32afe577f..b7312cb7563 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java @@ -157,7 +157,7 @@ public class TestDictionary extends LuceneTestCase { tempDir.close(); } - private class CloseCheckInputStream extends FilterInputStream { + private static class CloseCheckInputStream extends FilterInputStream { private boolean closed = false; public CloseCheckInputStream(InputStream delegate) { diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java index 7f0481fb9fd..2804bfd5a07 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterFilter.java @@ -184,7 +184,7 @@ public class TestWordDelimiterFilter extends BaseTokenStreamTestCase { /* * Set a large position increment gap of 10 if the token is "largegap" or "/" */ - private final class LargePosIncTokenFilter extends TokenFilter { + private static final class LargePosIncTokenFilter extends TokenFilter { private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); private PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterGraphFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterGraphFilter.java index f4e8b7931d1..7516a23fd65 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterGraphFilter.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/TestWordDelimiterGraphFilter.java @@ -171,7 +171,7 @@ public class TestWordDelimiterGraphFilter extends BaseTokenStreamTestCase { /* * Set a large position increment gap of 10 if the token is "largegap" or "/" */ - private final class LargePosIncTokenFilter extends TokenFilter { + private static final class LargePosIncTokenFilter extends TokenFilter { private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); private PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilterTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilterTest.java index 3e54cc0c49a..81da458726d 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilterTest.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/payloads/NumericPayloadTokenFilterTest.java @@ -57,7 +57,7 @@ public class NumericPayloadTokenFilterTest extends BaseTokenStreamTestCase { assertTrue(seenDogs + " does not equal: " + true, seenDogs == true); } - private final class WordTokenFilter extends TokenFilter { + private static final class WordTokenFilter extends TokenFilter { private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilterTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilterTest.java index f9010166714..5333cf0fc60 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilterTest.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/payloads/TypeAsPayloadTokenFilterTest.java @@ -47,7 +47,7 @@ public class TypeAsPayloadTokenFilterTest extends BaseTokenStreamTestCase { assertTrue(count + " does not equal: " + 10, count == 10); } - private final class WordTokenFilter extends TokenFilter { + private static final class WordTokenFilter extends TokenFilter { private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java index c0127a330e3..5c2c2bb63d6 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/TestTeeSinkTokenFilter.java @@ -223,7 +223,7 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase { } - class ModuloTokenFilter extends TokenFilter { + static class ModuloTokenFilter extends TokenFilter { int modCount; @@ -248,7 +248,7 @@ public class TestTeeSinkTokenFilter extends BaseTokenStreamTestCase { } } - class ModuloSinkFilter extends FilteringTokenFilter { + static class ModuloSinkFilter extends FilteringTokenFilter { int count = 0; int modCount; diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/snowball/TestSnowball.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/snowball/TestSnowball.java index 881bc2f576f..783b7073da3 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/snowball/TestSnowball.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/snowball/TestSnowball.java @@ -71,7 +71,7 @@ public class TestSnowball extends BaseTokenStreamTestCase { assertEquals(new BytesRef(new byte[]{0,1,2,3}), payloadAtt.getPayload()); } - private final class TestTokenStream extends TokenStream { + private static final class TestTokenStream extends TokenStream { private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class); private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class); diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/TestUAX29URLEmailTokenizer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/TestUAX29URLEmailTokenizer.java index 8d3c706ee1f..eaa5a4453f7 100644 --- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/TestUAX29URLEmailTokenizer.java +++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/TestUAX29URLEmailTokenizer.java @@ -127,7 +127,7 @@ public class TestUAX29URLEmailTokenizer extends BaseTokenStreamTestCase { } /** Passes through tokens with type "" and blocks all other types. */ - private class URLFilter extends TokenFilter { + private static class URLFilter extends TokenFilter { private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class); public URLFilter(TokenStream in) { super(in); @@ -146,7 +146,7 @@ public class TestUAX29URLEmailTokenizer extends BaseTokenStreamTestCase { } /** Passes through tokens with type "" and blocks all other types. */ - private class EmailFilter extends TokenFilter { + private static class EmailFilter extends TokenFilter { private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class); public EmailFilter(TokenStream in) { super(in); diff --git a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUTransformFilter.java b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUTransformFilter.java index e48c4ba731d..bfe7c12a2a9 100644 --- a/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUTransformFilter.java +++ b/lucene/analysis/icu/src/java/org/apache/lucene/analysis/icu/ICUTransformFilter.java @@ -112,7 +112,7 @@ public final class ICUTransformFilter extends TokenFilter { /** * Wrap a {@link CharTermAttribute} with the Replaceable API. */ - final class ReplaceableTermAttribute implements Replaceable { + static final class ReplaceableTermAttribute implements Replaceable { private char buffer[]; private int length; private CharTermAttribute token; diff --git a/lucene/classification/src/test/org/apache/lucene/classification/CachingNaiveBayesClassifierTest.java b/lucene/classification/src/test/org/apache/lucene/classification/CachingNaiveBayesClassifierTest.java index acbfe82be9a..ccdb207659a 100644 --- a/lucene/classification/src/test/org/apache/lucene/classification/CachingNaiveBayesClassifierTest.java +++ b/lucene/classification/src/test/org/apache/lucene/classification/CachingNaiveBayesClassifierTest.java @@ -82,7 +82,7 @@ public class CachingNaiveBayesClassifierTest extends ClassificationTestBase fields; SimpleTVFields(SortedMap fields) { diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java index aa19f20e266..2e8ed630307 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java @@ -742,7 +742,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem } - private class TVTerms extends Terms { + private static class TVTerms extends Terms { private final int numTerms, flags; private final int[] prefixLengths, suffixLengths, termFreqs, positionIndex, positions, startOffsets, lengths, payloadIndex; diff --git a/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java index c67d07bfe1c..eee0c16a463 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java @@ -160,7 +160,7 @@ public class ParallelLeafReader extends LeafReader { } // Single instance of this, per ParallelReader instance - private final class ParallelFields extends Fields { + private static final class ParallelFields extends Fields { final Map fields = new TreeMap<>(); ParallelFields() { diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java index dbd05e8e96c..8827a9f9385 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java @@ -68,7 +68,7 @@ public final class ConstantScoreQuery extends Query { * wraps a query with its own optimized top-level * scorer (e.g. BooleanScorer) we can use that * top-level scorer. */ - protected class ConstantBulkScorer extends BulkScorer { + protected static class ConstantBulkScorer extends BulkScorer { final BulkScorer bulkScorer; final Weight weight; final float theScore; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java index 79783b500ca..c8426060921 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java @@ -75,7 +75,7 @@ public class TestCodecs extends LuceneTestCase { NUM_TEST_ITER = atLeast(20); } - class FieldData implements Comparable { + static class FieldData implements Comparable { final FieldInfo fieldInfo; final TermData[] terms; final boolean omitTF; @@ -107,7 +107,7 @@ public class TestCodecs extends LuceneTestCase { } } - class PositionData { + static class PositionData { int pos; BytesRef payload; @@ -117,7 +117,7 @@ public class TestCodecs extends LuceneTestCase { } } - class TermData implements Comparable { + static class TermData implements Comparable { String text2; final BytesRef text; int[] docs; @@ -300,7 +300,7 @@ public class TestCodecs extends LuceneTestCase { dir.close(); } - private class Verify extends Thread { + private static class Verify extends Thread { final Fields termsDict; final FieldData[] fields; final SegmentInfo si; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java index 993a521a36e..aa2901cabbe 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java @@ -290,7 +290,7 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase { assertEquals(bytesUsed, flushControl.activeBytes()); } - public class IndexThread extends Thread { + public static class IndexThread extends Thread { IndexWriter writer; LiveIndexWriterConfig iwc; LineFileDocs docs; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java index 987852f8e26..c0907a5e682 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java @@ -346,7 +346,7 @@ public class TestIndexWriterExceptions extends LuceneTestCase { private static String CRASH_FAIL_MESSAGE = "I'm experiencing problems"; - private class CrashingFilter extends TokenFilter { + private static class CrashingFilter extends TokenFilter { String fieldName; int count; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java index e4d4018fe7a..6931efaed89 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMerging.java @@ -308,7 +308,7 @@ public class TestIndexWriterMerging extends LuceneTestCase { // Just intercepts all merges & verifies that we are never // merging a segment with >= 20 (maxMergeDocs) docs - private class MyMergeScheduler extends MergeScheduler { + private static class MyMergeScheduler extends MergeScheduler { @Override synchronized public void merge(IndexWriter writer, MergeTrigger trigger, boolean newMergesFound) throws IOException { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java index 6b599d2e4b8..6c950b84f9e 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterWithThreads.java @@ -51,7 +51,7 @@ import org.apache.lucene.util.LuceneTestCase.Slow; public class TestIndexWriterWithThreads extends LuceneTestCase { // Used by test cases below - private class IndexerThread extends Thread { + private static class IndexerThread extends Thread { boolean diskFull; Throwable error; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java index e60a3e17f4f..510a8997f9b 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java @@ -41,7 +41,7 @@ import org.apache.lucene.util.TestUtil; public class TestIndexableField extends LuceneTestCase { - private class MyField implements IndexableField { + private static class MyField implements IndexableField { private final int counter; private final IndexableFieldType fieldType = new IndexableFieldType() { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java b/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java index 900ec2b6b1c..4f74c30cfe2 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java @@ -99,7 +99,7 @@ public class TestMaxTermFrequency extends LuceneTestCase { /** * Simple similarity that encodes maxTermFrequency directly as a byte */ - class TestSimilarity extends TFIDFSimilarity { + static class TestSimilarity extends TFIDFSimilarity { @Override public float lengthNorm(FieldInvertState state) { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java index 363f57e7835..16ce61daf9d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java @@ -50,7 +50,7 @@ import org.apache.lucene.util.TestUtil; public class TestNorms extends LuceneTestCase { final String byteTestField = "normsTestByte"; - class CustomNormEncodingSimilarity extends TFIDFSimilarity { + static class CustomNormEncodingSimilarity extends TFIDFSimilarity { @Override public long encodeNormValue(float f) { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java index 34d8afbd19c..58ef89082d6 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java @@ -242,7 +242,7 @@ public class TestPerSegmentDeletes extends LuceneTestCase { return Arrays.copyOf(docs, numDocs); } - public class RangeMergePolicy extends MergePolicy { + public static class RangeMergePolicy extends MergePolicy { boolean doMerge = false; int start; int length; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java index 3cd9a6a6c0a..d386f395edd 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java @@ -686,7 +686,7 @@ public class TestStressIndexing2 extends LuceneTestCase { assertFalse(fieldsEnum2.hasNext()); } - private class IndexingThread extends Thread { + private static class IndexingThread extends Thread { IndexWriter w; int base; int range; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java index 4b2a06095a6..76947dd6852 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestTermVectorsReader.java @@ -50,7 +50,7 @@ public class TestTermVectorsReader extends LuceneTestCase { private FieldInfos fieldInfos = new FieldInfos(new FieldInfo[0]); private static int TERM_FREQ = 3; - private class TestToken implements Comparable { + private static class TestToken implements Comparable { String text; int pos; int startOffset; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestTransactionRollback.java b/lucene/core/src/test/org/apache/lucene/index/TestTransactionRollback.java index e7de028cd68..eb08475f8b2 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestTransactionRollback.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestTransactionRollback.java @@ -157,7 +157,7 @@ public class TestTransactionRollback extends LuceneTestCase { } // Rolls back to previous commit point - class RollbackDeletionPolicy extends IndexDeletionPolicy { + static class RollbackDeletionPolicy extends IndexDeletionPolicy { private int rollbackPoint; public RollbackDeletionPolicy(int rollbackPoint) { @@ -197,7 +197,7 @@ public class TestTransactionRollback extends LuceneTestCase { } } - class DeleteLastCommitPolicy extends IndexDeletionPolicy { + static class DeleteLastCommitPolicy extends IndexDeletionPolicy { @Override public void onCommit(List commits) throws IOException {} @@ -222,7 +222,7 @@ public class TestTransactionRollback extends LuceneTestCase { } // Keeps all commit points (used to build index) - class KeepAllDeletionPolicy extends IndexDeletionPolicy { + static class KeepAllDeletionPolicy extends IndexDeletionPolicy { @Override public void onCommit(List commits) throws IOException {} @Override diff --git a/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java b/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java index ee47fffbca1..a0fca4c62b4 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestUniqueTermCount.java @@ -98,7 +98,7 @@ public class TestUniqueTermCount extends LuceneTestCase { /** * Simple similarity that encodes maxTermFrequency directly */ - class TestSimilarity extends Similarity { + static class TestSimilarity extends Similarity { @Override public long computeNorm(FieldInvertState state) { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java b/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java index a9bac609621..f09dfde0955 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestCustomSearcherSort.java @@ -183,7 +183,7 @@ public class TestCustomSearcherSort extends LuceneTestCase { if (VERBOSE) System.out.println(message); } - public class CustomSearcher extends IndexSearcher { + public static class CustomSearcher extends IndexSearcher { private int switcher; public CustomSearcher(IndexReader r, int switcher) { @@ -212,7 +212,7 @@ public class TestCustomSearcherSort extends LuceneTestCase { } } - private class RandomGen { + private static class RandomGen { RandomGen(Random random) { this.random = random; base.set(1980, 1, 1); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java index d12caf101d1..6d699e8a1c1 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java @@ -201,7 +201,7 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase { } } - private class TestTerminatedEarlySimpleCollector extends SimpleCollector { + private static class TestTerminatedEarlySimpleCollector extends SimpleCollector { private boolean collectedSomething; public boolean collectedSomething() { return collectedSomething; @@ -216,7 +216,7 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase { } } - private class TestEarlyTerminatingSortingcollectorQueryTimeout implements QueryTimeout { + private static class TestEarlyTerminatingSortingcollectorQueryTimeout implements QueryTimeout { final private boolean shouldExit; public TestEarlyTerminatingSortingcollectorQueryTimeout(boolean shouldExit) { this.shouldExit = shouldExit; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPrefixRandom.java b/lucene/core/src/test/org/apache/lucene/search/TestPrefixRandom.java index 72fdc7aa689..f332a36a3ae 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestPrefixRandom.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestPrefixRandom.java @@ -75,7 +75,7 @@ public class TestPrefixRandom extends LuceneTestCase { } /** a stupid prefix query that just blasts thru the terms */ - private class DumbPrefixQuery extends MultiTermQuery { + private static class DumbPrefixQuery extends MultiTermQuery { private final BytesRef prefix; DumbPrefixQuery(Term term) { @@ -88,7 +88,7 @@ public class TestPrefixRandom extends LuceneTestCase { return new SimplePrefixTermsEnum(terms.iterator(), prefix); } - private class SimplePrefixTermsEnum extends FilteredTermsEnum { + private static class SimplePrefixTermsEnum extends FilteredTermsEnum { private final BytesRef prefix; private SimplePrefixTermsEnum(TermsEnum tenum, BytesRef prefix) { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java b/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java index a704d5479e1..78a43525342 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestRegexpRandom2.java @@ -102,7 +102,7 @@ public class TestRegexpRandom2 extends LuceneTestCase { } /** a stupid regexp query that just blasts thru the terms */ - private class DumbRegexpQuery extends MultiTermQuery { + private static class DumbRegexpQuery extends MultiTermQuery { private final Automaton automaton; DumbRegexpQuery(Term term, int flags) { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java b/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java index de96755b1d6..fbc0b355a0d 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSimilarityProvider.java @@ -90,7 +90,7 @@ public class TestSimilarityProvider extends LuceneTestCase { assertTrue(foodocs.scoreDocs[0].score < bardocs.scoreDocs[0].score); } - private class ExampleSimilarityProvider extends PerFieldSimilarityWrapper { + private static class ExampleSimilarityProvider extends PerFieldSimilarityWrapper { private Similarity sim1 = new Sim1(); private Similarity sim2 = new Sim2(); @@ -104,7 +104,7 @@ public class TestSimilarityProvider extends LuceneTestCase { } } - private class Sim1 extends TFIDFSimilarity { + private static class Sim1 extends TFIDFSimilarity { @Override public long encodeNormValue(float f) { @@ -142,7 +142,7 @@ public class TestSimilarityProvider extends LuceneTestCase { } } - private class Sim2 extends TFIDFSimilarity { + private static class Sim2 extends TFIDFSimilarity { @Override public long encodeNormValue(float f) { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java index d00e520a371..f0ad9b9d3b1 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java @@ -153,7 +153,7 @@ public class TestTermScorer extends LuceneTestCase { assertTrue("doc should be number 5", ts.docID() == 5); } - private class TestHit { + private static class TestHit { public int doc; public float score; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java index d26006cad0a..6a02e587eb5 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java @@ -320,7 +320,7 @@ public class TestTimeLimitingCollector extends LuceneTestCase { } // counting collector that can slow down at collect(). - private class MyHitCollector extends SimpleCollector { + private static class MyHitCollector extends SimpleCollector { private final BitSet bits = new BitSet(); private int slowdown = 0; private int lastDocCollected = -1; diff --git a/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java b/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java index fa7a3fb1c05..c2578612cad 100644 --- a/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java +++ b/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java @@ -76,7 +76,7 @@ public class TestLockFactory extends LuceneTestCase { } } - class MockLockFactory extends LockFactory { + static class MockLockFactory extends LockFactory { public Map locksCreated = Collections.synchronizedMap(new HashMap()); @@ -87,7 +87,7 @@ public class TestLockFactory extends LuceneTestCase { return lock; } - public class MockLock extends Lock { + public static class MockLock extends Lock { @Override public void close() { diff --git a/lucene/core/src/test/org/apache/lucene/util/TestCloseableThreadLocal.java b/lucene/core/src/test/org/apache/lucene/util/TestCloseableThreadLocal.java index 9f56d7b45ed..ca0f811e97d 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestCloseableThreadLocal.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestCloseableThreadLocal.java @@ -40,7 +40,7 @@ public class TestCloseableThreadLocal extends LuceneTestCase { assertNull(ctl.get()); } - public class InitValueThreadLocal extends CloseableThreadLocal { + public static class InitValueThreadLocal extends CloseableThreadLocal { @Override protected Object initialValue() { return TEST_VALUE; diff --git a/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java b/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java index fc04c5ed286..a9d803bcc56 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestQueryBuilder.java @@ -266,7 +266,7 @@ public class TestQueryBuilder extends LuceneTestCase { } } - private class SimpleCJKAnalyzer extends Analyzer { + private static class SimpleCJKAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { return new TokenStreamComponents(new SimpleCJKTokenizer()); 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 7a8e5f5f6e9..6f704c94f5b 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 @@ -1458,7 +1458,7 @@ public class TestFSTs extends LuceneTestCase { } // used by slowcompletor - class TwoLongs { + static class TwoLongs { long a; long b; diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/CollisionMap.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/CollisionMap.java index 205a540f35a..b80f2bd08cd 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/CollisionMap.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/CollisionMap.java @@ -185,7 +185,7 @@ public class CollisionMap { return memoryUsage; } - private class EntryIterator implements Iterator { + private static class EntryIterator implements Iterator { Entry next; // next entry to return int index; // current slot Entry[] ents; diff --git a/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java b/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java index 46141bcdfb9..c590502f929 100644 --- a/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java +++ b/lucene/grouping/src/test/org/apache/lucene/search/grouping/GroupFacetCollectorTest.java @@ -720,7 +720,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase { return null; } - private class IndexContext { + private static class IndexContext { final int numDocs; final DirectoryReader indexReader; @@ -744,7 +744,7 @@ public class GroupFacetCollectorTest extends AbstractGroupingTestCase { } } - private class GroupedFacetResult { + private static class GroupedFacetResult { final int totalCount; final int totalMissingCount; diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java index 08055a27298..e5d7e82e639 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/visibility/TestUnifiedHighlighterExtensibility.java @@ -202,7 +202,7 @@ public class TestUnifiedHighlighterExtensibility extends LuceneTestCase { } /** Tests maintaining extensibility/visibility of {@link org.apache.lucene.search.uhighlight.FieldHighlighter} out of package. */ - private class CustomFieldHighlighter extends FieldHighlighter { + private static class CustomFieldHighlighter extends FieldHighlighter { CustomFieldHighlighter(String field, FieldOffsetStrategy fieldOffsetStrategy, BreakIterator breakIterator, PassageScorer passageScorer, int maxPassages, int maxNoHighlightPassages, PassageFormatter passageFormatter) { super(field, fieldOffsetStrategy, breakIterator, passageScorer, maxPassages, maxNoHighlightPassages, passageFormatter); } diff --git a/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java index 2ae6c0173f0..632bcc7b74d 100644 --- a/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java +++ b/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java @@ -238,7 +238,7 @@ public class CustomScoreQuery extends Query implements Cloneable { /** * A scorer that applies a (callback) function on scores of the subQuery. */ - private class CustomScorer extends FilterScorer { + private static class CustomScorer extends FilterScorer { private final float qWeight; private final Scorer subQueryScorer; private final Scorer[] valSrcScorers; diff --git a/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java b/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java index 540ecf8dc73..d001cc8f968 100644 --- a/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java +++ b/lucene/queries/src/test/org/apache/lucene/queries/TestCustomScoreQuery.java @@ -153,7 +153,7 @@ public class TestCustomScoreQuery extends FunctionTestSetup { } } - private final class CustomExternalQuery extends CustomScoreQuery { + private static final class CustomExternalQuery extends CustomScoreQuery { @Override protected CustomScoreProvider getCustomScoreProvider(LeafReaderContext context) throws IOException { diff --git a/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadSpans.java b/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadSpans.java index 3c5b80f7cca..c39421b1e4f 100644 --- a/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadSpans.java +++ b/lucene/queries/src/test/org/apache/lucene/queries/payloads/TestPayloadSpans.java @@ -461,7 +461,7 @@ public class TestPayloadSpans extends LuceneTestCase { assertEquals("expected numSpans", numSpans, cnt); } - final class PayloadAnalyzer extends Analyzer { + static final class PayloadAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { @@ -470,7 +470,7 @@ public class TestPayloadSpans extends LuceneTestCase { } } - final class PayloadFilter extends TokenFilter { + static final class PayloadFilter extends TokenFilter { Set entities = new HashSet<>(); Set nopayload = new HashSet<>(); int pos; @@ -515,7 +515,7 @@ public class TestPayloadSpans extends LuceneTestCase { } } - public final class TestPayloadAnalyzer extends Analyzer { + public static final class TestPayloadAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiAnalyzer.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiAnalyzer.java index bdb6256d80e..85a5f3942e9 100644 --- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiAnalyzer.java +++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiAnalyzer.java @@ -120,7 +120,7 @@ public class TestMultiAnalyzer extends BaseTokenStreamTestCase { * Expands "multi" to "multi" and "multi2", both at the same position, * and expands "triplemulti" to "triplemulti", "multi3", and "multi2". */ - private class MultiAnalyzer extends Analyzer { + private static class MultiAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { @@ -129,7 +129,7 @@ public class TestMultiAnalyzer extends BaseTokenStreamTestCase { } } - private final class TestFilter extends TokenFilter { + private static final class TestFilter extends TokenFilter { private String prevType; private int prevStartOffset; @@ -191,7 +191,7 @@ public class TestMultiAnalyzer extends BaseTokenStreamTestCase { * Analyzes "the quick brown" as: quick(incr=2) brown(incr=1). * Does not work correctly for input other than "the quick brown ...". */ - private class PosIncrementAnalyzer extends Analyzer { + private static class PosIncrementAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { @@ -200,7 +200,7 @@ public class TestMultiAnalyzer extends BaseTokenStreamTestCase { } } - private final class TestPosIncrementFilter extends TokenFilter { + private static final class TestPosIncrementFilter extends TokenFilter { CharTermAttribute termAtt; PositionIncrementAttribute posIncrAtt; diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java index 7490e8cb608..4c28e8f8b23 100644 --- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java +++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestMultiFieldQueryParser.java @@ -336,7 +336,7 @@ public class TestMultiFieldQueryParser extends LuceneTestCase { } /** whitespace+lowercase analyzer with synonyms (dogs,dog) and (guinea pig,cavy) */ - private class MockSynonymAnalyzer extends Analyzer { + private static class MockSynonymAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, true); diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java index 56ef65af778..220ce021bb8 100644 --- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java +++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java @@ -139,7 +139,7 @@ public class TestMultiAnalyzerQPHelper extends LuceneTestCase { * Expands "multi" to "multi" and "multi2", both at the same position, and * expands "triplemulti" to "triplemulti", "multi3", and "multi2". */ - private class MultiAnalyzer extends Analyzer { + private static class MultiAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { @@ -148,7 +148,7 @@ public class TestMultiAnalyzerQPHelper extends LuceneTestCase { } } - private final class TestFilter extends TokenFilter { + private static final class TestFilter extends TokenFilter { private String prevType; private int prevStartOffset; @@ -206,7 +206,7 @@ public class TestMultiAnalyzerQPHelper extends LuceneTestCase { * Analyzes "the quick brown" as: quick(incr=2) brown(incr=1). Does not work * correctly for input other than "the quick brown ...". */ - private class PosIncrementAnalyzer extends Analyzer { + private static class PosIncrementAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { @@ -215,7 +215,7 @@ public class TestMultiAnalyzerQPHelper extends LuceneTestCase { } } - private class TestPosIncrementFilter extends TokenFilter { + private static class TestPosIncrementFilter extends TokenFilter { private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class); diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java index 2d5ee43d452..2aa96beaaaf 100644 --- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java +++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java @@ -350,7 +350,7 @@ public class TestQPHelper extends LuceneTestCase { } //individual CJK chars as terms, like StandardAnalyzer - private class SimpleCJKTokenizer extends Tokenizer { + private static class SimpleCJKTokenizer extends Tokenizer { private CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); public SimpleCJKTokenizer() { @@ -368,7 +368,7 @@ public class TestQPHelper extends LuceneTestCase { } } - private class SimpleCJKAnalyzer extends Analyzer { + private static class SimpleCJKAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { return new TokenStreamComponents(new SimpleCJKTokenizer()); @@ -1252,7 +1252,7 @@ public class TestQPHelper extends LuceneTestCase { super.tearDown(); } - private class CannedTokenizer extends Tokenizer { + private static class CannedTokenizer extends Tokenizer { private int upto = 0; private final PositionIncrementAttribute posIncr = addAttribute(PositionIncrementAttribute.class); private final CharTermAttribute term = addAttribute(CharTermAttribute.class); @@ -1291,7 +1291,7 @@ public class TestQPHelper extends LuceneTestCase { } } - private class CannedAnalyzer extends Analyzer { + private static class CannedAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String ignored) { return new TokenStreamComponents(new CannedTokenizer()); diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java index 004110634e1..1b8ee96bc40 100644 --- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java +++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java @@ -264,7 +264,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase { } } - private class SimpleCJKAnalyzer extends Analyzer { + private static class SimpleCJKAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { return new TokenStreamComponents(new SimpleCJKTokenizer()); @@ -1095,7 +1095,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase { } /** whitespace+lowercase analyzer with synonyms */ - protected class Analyzer1 extends Analyzer { + protected static class Analyzer1 extends Analyzer { public Analyzer1(){ super(); } @@ -1107,7 +1107,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase { } /** whitespace+lowercase analyzer without synonyms */ - protected class Analyzer2 extends Analyzer { + protected static class Analyzer2 extends Analyzer { public Analyzer2(){ super(); } @@ -1122,7 +1122,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase { /** * Mock collation analyzer: indexes terms as "collated" + term */ - private class MockCollationFilter extends TokenFilter { + private static class MockCollationFilter extends TokenFilter { private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); protected MockCollationFilter(TokenStream input) { @@ -1141,7 +1141,7 @@ public abstract class QueryParserTestBase extends LuceneTestCase { } } - private class MockCollationAnalyzer extends Analyzer { + private static class MockCollationAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, true); diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java index 8bd7b894afa..840ade3ad20 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/FuzzyLikeThisQuery.java @@ -112,7 +112,7 @@ public class FuzzyLikeThisQuery extends Query this.maxNumTerms = maxNumTerms; } - class FieldVals + static class FieldVals { String queryString; String fieldName; diff --git a/lucene/sandbox/src/test/org/apache/lucene/payloads/TestPayloadSpanUtil.java b/lucene/sandbox/src/test/org/apache/lucene/payloads/TestPayloadSpanUtil.java index ecf2ff63617..af19e3d978d 100644 --- a/lucene/sandbox/src/test/org/apache/lucene/payloads/TestPayloadSpanUtil.java +++ b/lucene/sandbox/src/test/org/apache/lucene/payloads/TestPayloadSpanUtil.java @@ -72,7 +72,7 @@ public class TestPayloadSpanUtil extends LuceneTestCase { directory.close(); } - final class PayloadAnalyzer extends Analyzer { + final static class PayloadAnalyzer extends Analyzer { @Override public TokenStreamComponents createComponents(String fieldName) { @@ -81,7 +81,7 @@ public class TestPayloadSpanUtil extends LuceneTestCase { } } - final class PayloadFilter extends TokenFilter { + static final class PayloadFilter extends TokenFilter { Set entities = new HashSet<>(); Set nopayload = new HashSet<>(); int pos; diff --git a/lucene/suggest/src/java/org/apache/lucene/search/spell/WordBreakSpellChecker.java b/lucene/suggest/src/java/org/apache/lucene/search/spell/WordBreakSpellChecker.java index f3cc5bd5972..c075ad47918 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/spell/WordBreakSpellChecker.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/spell/WordBreakSpellChecker.java @@ -441,7 +441,7 @@ public class WordBreakSpellChecker { this.maxEvaluations = maxEvaluations; } - private class LengthThenMaxFreqComparator implements + private static class LengthThenMaxFreqComparator implements Comparator { @Override public int compare(SuggestWordArrayWrapper o1, SuggestWordArrayWrapper o2) { @@ -455,7 +455,7 @@ public class WordBreakSpellChecker { } } - private class LengthThenSumFreqComparator implements + private static class LengthThenSumFreqComparator implements Comparator { @Override public int compare(SuggestWordArrayWrapper o1, SuggestWordArrayWrapper o2) { @@ -469,7 +469,7 @@ public class WordBreakSpellChecker { } } - private class CombinationsThenFreqComparator implements + private static class CombinationsThenFreqComparator implements Comparator { @Override public int compare(CombineSuggestionWrapper o1, CombineSuggestionWrapper o2) { @@ -484,7 +484,7 @@ public class WordBreakSpellChecker { } } - private class SuggestWordArrayWrapper { + private static class SuggestWordArrayWrapper { final SuggestWord[] suggestWords; final int freqMax; final int freqSum; @@ -502,7 +502,7 @@ public class WordBreakSpellChecker { } } - private class CombineSuggestionWrapper { + private static class CombineSuggestionWrapper { final CombineSuggestion combineSuggestion; final int numCombinations; diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java index 7a5e3e00ef5..4a29f244432 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/ContextQuery.java @@ -238,7 +238,7 @@ public class ContextQuery extends CompletionQuery { } } - private class ContextCompletionWeight extends CompletionWeight { + private static class ContextCompletionWeight extends CompletionWeight { private final Map contextMap; private final int[] contextLengths; diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java index fb876d2b451..5b7e7142349 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java @@ -105,7 +105,7 @@ public class ExternalRefSorter implements BytesRefSorter, Closeable { * Iterate over byte refs in a file. */ // TODO: this class is a bit silly ... sole purpose is to "remove" Closeable from what #iterator returns: - class ByteSequenceIterator implements BytesRefIterator { + static class ByteSequenceIterator implements BytesRefIterator { private final OfflineSorter.ByteSequencesReader reader; private BytesRef scratch; diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java index 1cc89a4d076..d6c1a97e4fa 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java @@ -267,7 +267,7 @@ public class WFSTCompletionLookup extends Lookup implements Accountable { return Integer.MAX_VALUE - (int)value; } - private final class WFSTInputIterator extends SortedInputIterator { + private static final class WFSTInputIterator extends SortedInputIterator { WFSTInputIterator(Directory tempDir, String tempFileNamePrefix, InputIterator source) throws IOException { super(tempDir, tempFileNamePrefix, source); diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java index 3e78f59e9fb..948ebeb0151 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellLookup.java @@ -147,17 +147,17 @@ public class JaspellLookup extends Lookup implements Accountable { node.data = Long.valueOf(in.readLong()); } if ((mask & LO_KID) != 0) { - TSTNode kid = trie.new TSTNode('\0', node); + TSTNode kid = new TSTNode('\0', node); node.relatives[TSTNode.LOKID] = kid; readRecursively(in, kid); } if ((mask & EQ_KID) != 0) { - TSTNode kid = trie.new TSTNode('\0', node); + TSTNode kid = new TSTNode('\0', node); node.relatives[TSTNode.EQKID] = kid; readRecursively(in, kid); } if ((mask & HI_KID) != 0) { - TSTNode kid = trie.new TSTNode('\0', node); + TSTNode kid = new TSTNode('\0', node); node.relatives[TSTNode.HIKID] = kid; readRecursively(in, kid); } @@ -196,7 +196,7 @@ public class JaspellLookup extends Lookup implements Accountable { @Override public boolean load(DataInput input) throws IOException { count = input.readVLong(); - TSTNode root = trie.new TSTNode('\0', null); + TSTNode root = new TSTNode('\0', null); readRecursively(input, root); trie.setRoot(root); return true; diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellTernarySearchTrie.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellTernarySearchTrie.java index 15deaea6beb..2a4a43963e5 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellTernarySearchTrie.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/jaspell/JaspellTernarySearchTrie.java @@ -71,7 +71,7 @@ public class JaspellTernarySearchTrie implements Accountable { /** * An inner class of Ternary Search Trie that represents a node in the trie. */ - protected final class TSTNode implements Accountable { + protected static final class TSTNode implements Accountable { /** Index values for accessing relatives array. */ protected final static int PARENT = 0, LOKID = 1, EQKID = 2, HIKID = 3; diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java index a7e9b002c5c..57173e246d6 100644 --- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java +++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java @@ -411,7 +411,7 @@ public class DocumentDictionaryTest extends LuceneTestCase { return suggestionList; } - private class Suggestion { + private static class Suggestion { private long weight; private BytesRef payload; private Set contexts; diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java index 478358b5c92..423e62a0ee5 100644 --- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java +++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java @@ -910,7 +910,7 @@ public class AnalyzingInfixSuggesterTest extends LuceneTestCase { a.close(); } - private class IndexDocument implements Runnable { + private static class IndexDocument implements Runnable { AnalyzingInfixSuggester suggester; String key; diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java index adda0fcf37d..165c9f26e43 100644 --- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java +++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java @@ -1118,7 +1118,7 @@ public class FuzzySuggesterTest extends LuceneTestCase { private static final Comparator CHARSEQUENCE_COMPARATOR = new CharSequenceComparator(); - public class CompareByCostThenAlpha implements Comparator { + public static class CompareByCostThenAlpha implements Comparator { @Override public int compare(LookupResult a, LookupResult b) { if (a.value > b.value) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java index 01791b792d1..312b644e9f5 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java @@ -274,7 +274,7 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase { } } - private class SearcherThread extends Thread { + private static class SearcherThread extends Thread { private Directory dir; private int numIteration; public boolean hitException = false; diff --git a/lucene/tools/src/java/org/apache/lucene/dependencies/GetMavenDependenciesTask.java b/lucene/tools/src/java/org/apache/lucene/dependencies/GetMavenDependenciesTask.java index 5b2f0b80b00..2c20abdbd81 100644 --- a/lucene/tools/src/java/org/apache/lucene/dependencies/GetMavenDependenciesTask.java +++ b/lucene/tools/src/java/org/apache/lucene/dependencies/GetMavenDependenciesTask.java @@ -786,7 +786,7 @@ public class GetMavenDependenciesTask extends Task { /** * Stores information about an external dependency */ - private class ExternalDependency implements Comparable { + private static class ExternalDependency implements Comparable { String groupId; String artifactId; boolean isTestDependency; diff --git a/lucene/tools/src/java/org/apache/lucene/validation/LibVersionsCheckTask.java b/lucene/tools/src/java/org/apache/lucene/validation/LibVersionsCheckTask.java index 3179c78964c..d7844b00b97 100644 --- a/lucene/tools/src/java/org/apache/lucene/validation/LibVersionsCheckTask.java +++ b/lucene/tools/src/java/org/apache/lucene/validation/LibVersionsCheckTask.java @@ -150,7 +150,7 @@ public class LibVersionsCheckTask extends Task { */ private Map> ignoreConflictVersions = new HashMap<>(); - private class Dependency { + private static class Dependency { String org; String name; String directVersion; 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 7545eacb18c..d4418daec81 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 @@ -728,8 +728,8 @@ public class MailEntityProcessor extends EntityProcessorBase { return true; } } - - class MailsSinceLastCheckFilter implements CustomFilter { + + static class MailsSinceLastCheckFilter implements CustomFilter { private Date since; diff --git a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java index 51b5841aea6..8fced583e2c 100644 --- a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java +++ b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java @@ -167,7 +167,7 @@ public class VariableResolver { return TemplateUpdateProcessorFactory.getVariables(expr, cache); } - class CurrentLevel { + static class CurrentLevel { final Map map; final int level; CurrentLevel(int level, Map map) { diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java index b72f3797a48..2ef5a911834 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java @@ -73,7 +73,7 @@ public class AbstractDIHCacheTestCase { //A limitation of this test class is that the primary key needs to be the first one in the list. //DIHCaches, however, can handle any field being the primary key. - class ControlData implements Comparable, Iterable { + static class ControlData implements Comparable, Iterable { Object[] data; ControlData(Object[] data) { diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java index 06b89c3a4a6..277404451c1 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java @@ -568,8 +568,8 @@ public abstract class AbstractSqlEntityProcessorTestCase extends } return changeSet.toArray(new String[changeSet.size()]); } - - class IntChanges { + + static class IntChanges { public Integer[] changedKeys; public Integer[] deletedKeys; public Integer[] addedKeys; diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java index 95af5f9494b..089a133d333 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContentStreamDataSource.java @@ -109,7 +109,7 @@ public class TestContentStreamDataSource extends AbstractDataImportHandlerTestCa fail("Commit should have occured but it did not"); } - private class SolrInstance { + private static class SolrInstance { String name; Integer port; File homeDir; diff --git a/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/XLSXResponseWriter.java b/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/XLSXResponseWriter.java index 27a30d170e0..92bd01ff443 100644 --- a/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/XLSXResponseWriter.java +++ b/solr/contrib/extraction/src/java/org/apache/solr/handler/extraction/XLSXResponseWriter.java @@ -98,7 +98,7 @@ class XLSXWriter extends TextResponseWriter { SolrQueryRequest req; SolrQueryResponse rsp; - class SerialWriteWorkbook { + static class SerialWriteWorkbook { SXSSFWorkbook swb; Sheet sh; diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java index 37990ea4dd0..6ecbb6f66fc 100644 --- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java +++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/LTRScoringQuery.java @@ -300,7 +300,7 @@ public class LTRScoringQuery extends Query { return field; } - public class FeatureInfo { + public static class FeatureInfo { final private String name; private float value; private boolean used; diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestLTRReRankingPipeline.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestLTRReRankingPipeline.java index a98fc4f5e34..75d35388235 100644 --- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestLTRReRankingPipeline.java +++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/TestLTRReRankingPipeline.java @@ -80,7 +80,7 @@ public class TestLTRReRankingPipeline extends LuceneTestCase { return features; } - private class MockModel extends LTRScoringModel { + private static class MockModel extends LTRScoringModel { public MockModel(String name, List features, List norms, diff --git a/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java b/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java index e7cc9b7fb15..133bc63284d 100644 --- a/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java +++ b/solr/contrib/velocity/src/java/org/apache/solr/response/VelocityResponseWriter.java @@ -376,7 +376,7 @@ public class VelocityResponseWriter implements QueryResponseWriter, SolrCoreAwar } // see: http://svn.apache.org/repos/asf/velocity/tools/branches/2.0.x/src/main/java/org/apache/velocity/tools/generic/ResourceTool.java - private class SolrVelocityResourceTool extends ResourceTool { + private static class SolrVelocityResourceTool extends ResourceTool { private ClassLoader solrClassLoader; diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java index e0449b4a0f1..4d3cee7d737 100644 --- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java +++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java @@ -431,7 +431,7 @@ public class Overseer implements Closeable { } - class OverseerThread extends Thread implements Closeable { + static class OverseerThread extends Thread implements Closeable { protected volatile boolean isClosed; private Closeable thread; diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java index c7604d60017..e031303ad78 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java @@ -108,7 +108,7 @@ public class OverseerTaskQueue extends DistributedQueue { /** * Watcher that blocks until a WatchedEvent occurs for a znode. */ - private final class LatchWatcher implements Watcher { + private static final class LatchWatcher implements Watcher { private final Object lock; private WatchedEvent event; 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 69a77f94820..a3f1fd477cd 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java @@ -239,7 +239,7 @@ public class ZkController { } // notifies registered listeners after the ZK reconnect in the background - private class OnReconnectNotifyAsync implements Callable { + private static class OnReconnectNotifyAsync implements Callable { private final OnReconnect listener; diff --git a/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java b/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java index 08a8a4e20f2..e710063f606 100644 --- a/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java +++ b/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java @@ -48,7 +48,7 @@ import org.slf4j.LoggerFactory; * */ public abstract class CachingDirectoryFactory extends DirectoryFactory { - protected class CacheValue { + protected static class CacheValue { final public String path; final public Directory directory; // for debug diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java index 75a787b8804..a151a43061a 100644 --- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java +++ b/solr/core/src/java/org/apache/solr/handler/CdcrReplicator.java @@ -207,7 +207,7 @@ public class CdcrReplicator implements Runnable { /** * Exception to catch update request issues with the target cluster. */ - public class CdcrReplicatorException extends Exception { + public static class CdcrReplicatorException extends Exception { private final UpdateRequest req; private final UpdateResponse rsp; diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java index 2ca0d800be5..bf80608745a 100644 --- a/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java +++ b/solr/core/src/java/org/apache/solr/handler/CdcrReplicatorState.java @@ -197,7 +197,7 @@ class CdcrReplicatorState { } - class BenchmarkTimer { + static class BenchmarkTimer { private long startTime; private long runTime = 0; @@ -266,7 +266,7 @@ class CdcrReplicatorState { } - private class ErrorQueueEntry { + private static class ErrorQueueEntry { private ErrorType type; private Date timestamp; @@ -277,7 +277,7 @@ class CdcrReplicatorState { } } - private class FixedQueue extends LinkedList { + private static class FixedQueue extends LinkedList { private int maxSize; diff --git a/solr/core/src/java/org/apache/solr/handler/ExportWriter.java b/solr/core/src/java/org/apache/solr/handler/ExportWriter.java index 7602d9eb57b..bd43d5fc31a 100644 --- a/solr/core/src/java/org/apache/solr/handler/ExportWriter.java +++ b/solr/core/src/java/org/apache/solr/handler/ExportWriter.java @@ -939,7 +939,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable { public int resetValue(); } - class IntDesc implements IntComp { + static class IntDesc implements IntComp { public int resetValue() { return Integer.MIN_VALUE; @@ -956,7 +956,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable { } } - class IntAsc implements IntComp { + static class IntAsc implements IntComp { public int resetValue() { return Integer.MAX_VALUE; @@ -1032,7 +1032,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable { public long resetValue(); } - class LongDesc implements LongComp { + static class LongDesc implements LongComp { public long resetValue() { return Long.MIN_VALUE; @@ -1049,7 +1049,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable { } } - class LongAsc implements LongComp { + static class LongAsc implements LongComp { public long resetValue() { return Long.MAX_VALUE; @@ -1125,7 +1125,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable { public float resetValue(); } - public class FloatDesc implements FloatComp { + public static class FloatDesc implements FloatComp { public float resetValue() { return -Float.MAX_VALUE; } @@ -1141,7 +1141,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable { } } - public class FloatAsc implements FloatComp { + public static class FloatAsc implements FloatComp { public float resetValue() { return Float.MAX_VALUE; } @@ -1219,7 +1219,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable { public double resetValue(); } - public class DoubleDesc implements DoubleComp { + public static class DoubleDesc implements DoubleComp { public double resetValue() { return -Double.MAX_VALUE; } @@ -1235,7 +1235,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable { } } - public class DoubleAsc implements DoubleComp { + public static class DoubleAsc implements DoubleComp { public double resetValue() { return Double.MAX_VALUE; } @@ -1712,7 +1712,7 @@ public class ExportWriter implements SolrCore.RawWriter, Closeable { } } - public class IgnoreException extends IOException { + public static class IgnoreException extends IOException { public void printStackTrace(PrintWriter pw) { pw.print("Early Client Disconnect"); } 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 5efb6c5fe58..d79effd83d1 100644 --- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java +++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java @@ -1690,7 +1690,7 @@ public class IndexFetcher { } } - private class DirectoryFile implements FileInterface { + private static class DirectoryFile implements FileInterface { private final String saveAs; private Directory copy2Dir; private IndexOutput outStream; @@ -1725,7 +1725,7 @@ public class IndexFetcher { } } - private class LocalFsFile implements FileInterface { + private static class LocalFsFile implements FileInterface { private File copy2Dir; FileChannel fileChannel; 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 7563fe84ced..f307baa72ea 100644 --- a/solr/core/src/java/org/apache/solr/handler/SQLHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/SQLHandler.java @@ -141,7 +141,7 @@ public class SQLHandler extends RequestHandlerBase implements SolrCoreAware, Per /* * Only necessary for SolrJ JDBC driver since metadata has to be passed back */ - private class SqlHandlerStream extends JDBCStream { + private static class SqlHandlerStream extends JDBCStream { private final boolean includeMetadata; private boolean firstTuple = true; List metadataFields = new ArrayList<>(); diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java index 8078bdcea47..f6ecd8d888f 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java @@ -521,7 +521,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia rb.rsp.add("expanded", expanded); } - private class GroupExpandCollector implements Collector, GroupCollector { + private static class GroupExpandCollector implements Collector, GroupCollector { private SortedDocValues docValues; private MultiDocValues.OrdinalMap ordinalMap; private SortedDocValues segmentValues; @@ -614,7 +614,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia } } - private class NumericGroupExpandCollector implements Collector, GroupCollector { + private static class NumericGroupExpandCollector implements Collector, GroupCollector { private NumericDocValues docValues; private String field; @@ -763,7 +763,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia // this reader alters the content of the given reader so it should not // delegate the caching stuff - private class ReaderWrapper extends FilterLeafReader { + private static class ReaderWrapper extends FilterLeafReader { private String field; diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java index 83a2d60e72a..1c016c73326 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java +++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java @@ -283,7 +283,7 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org. * If all nodes prefer local-cores then a bad/heavily-loaded node will receive less requests from healthy nodes. * This will help prevent a distributed deadlock or timeouts in all the healthy nodes due to one bad node. */ - private class IsOnPreferredHostComparator implements Comparator { + private static class IsOnPreferredHostComparator implements Comparator { final private String preferredHostAddress; public IsOnPreferredHostComparator(String preferredHostAddress) { this.preferredHostAddress = preferredHostAddress; diff --git a/solr/core/src/java/org/apache/solr/handler/component/PivotFacetFieldValueCollection.java b/solr/core/src/java/org/apache/solr/handler/component/PivotFacetFieldValueCollection.java index 6aae231a3e4..5c2b07f61d1 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/PivotFacetFieldValueCollection.java +++ b/solr/core/src/java/org/apache/solr/handler/component/PivotFacetFieldValueCollection.java @@ -301,7 +301,7 @@ public class PivotFacetFieldValueCollection implements Iterable } /** Sorts {@link PivotFacetValue} instances by their count */ - public class PivotFacetCountComparator implements Comparator { + public static class PivotFacetCountComparator implements Comparator { public int compare(PivotFacetValue left, PivotFacetValue right) { int countCmp = right.getCount() - left.getCount(); return (0 != countCmp) ? countCmp : @@ -310,7 +310,7 @@ public class PivotFacetFieldValueCollection implements Iterable } /** Sorts {@link PivotFacetValue} instances by their value */ - public class PivotFacetValueComparator implements Comparator { + public static class PivotFacetValueComparator implements Comparator { public int compare(PivotFacetValue left, PivotFacetValue right) { return compareWithNullLast(left.getValue(), right.getValue()); } diff --git a/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java b/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java index b76537a05a4..e00120c65e1 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java @@ -353,7 +353,7 @@ public class TermsComponent extends SearchComponent { return sreq; } - public class TermsHelper { + public static class TermsHelper { // map to store returned terms private HashMap> fieldmap; private SolrParams params; diff --git a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java index e4ada59dbf7..098e1f76f9c 100644 --- a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java +++ b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java @@ -750,7 +750,7 @@ public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInf } // Wraps FVH to allow pass-by-reference. Public access to allow use in 3rd party subclasses - public class FvhContainer { + public static class FvhContainer { FastVectorHighlighter fvh; FieldQuery fieldQuery; diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java index b22feabba2c..9bc888dd767 100644 --- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java +++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java @@ -368,13 +368,13 @@ public final class ManagedIndexSchema extends IndexSchema { } - public class FieldExistsException extends SolrException { + public static class FieldExistsException extends SolrException { public FieldExistsException(ErrorCode code, String msg) { super(code, msg); } } - public class SchemaChangedInZkException extends SolrException { + public static class SchemaChangedInZkException extends SolrException { public SchemaChangedInZkException(ErrorCode code, String msg) { super(code, msg); } diff --git a/solr/core/src/java/org/apache/solr/schema/RandomSortField.java b/solr/core/src/java/org/apache/solr/schema/RandomSortField.java index 0ecb195be76..44bb420947f 100644 --- a/solr/core/src/java/org/apache/solr/schema/RandomSortField.java +++ b/solr/core/src/java/org/apache/solr/schema/RandomSortField.java @@ -160,7 +160,7 @@ public class RandomSortField extends FieldType { - public class RandomValueSource extends ValueSource { + public static class RandomValueSource extends ValueSource { private final String field; public RandomValueSource(String field) { diff --git a/solr/core/src/java/org/apache/solr/search/ComplexPhraseQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/ComplexPhraseQParserPlugin.java index 22702dcdb85..2a00a0989c4 100644 --- a/solr/core/src/java/org/apache/solr/search/ComplexPhraseQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/ComplexPhraseQParserPlugin.java @@ -59,9 +59,9 @@ public class ComplexPhraseQParserPlugin extends QParserPlugin { /** * Modified from {@link org.apache.solr.search.LuceneQParser} and {@link org.apache.solr.search.SurroundQParserPlugin.SurroundQParser} */ - class ComplexPhraseQParser extends QParser { + static class ComplexPhraseQParser extends QParser { - final class SolrQueryParserDelegate extends SolrQueryParser { + static final class SolrQueryParserDelegate extends SolrQueryParser { private SolrQueryParserDelegate(QParser parser, String defaultField) { super(parser, defaultField); } diff --git a/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java index 38bb74f9533..6c1714cebb7 100644 --- a/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java @@ -121,7 +121,7 @@ public class ExportQParserPlugin extends QParserPlugin { } } - private class ExportCollector extends TopDocsCollector { + private static class ExportCollector extends TopDocsCollector { private FixedBitSet[] sets; diff --git a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java index c0aee881c93..a4f9312ac14 100644 --- a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java +++ b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java @@ -956,7 +956,7 @@ public class ExtendedDismaxQParser extends QParser { /** A simple container for storing alias info */ - protected class Alias { + protected static class Alias { public float tie; public Map fields; } @@ -1597,7 +1597,7 @@ public class ExtendedDismaxQParser extends QParser { /** * Simple container for configuration information used when parsing queries */ - public class ExtendedDismaxConfiguration { + public static class ExtendedDismaxConfiguration { /** * The field names specified by 'qf' that (most) clauses will diff --git a/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java index 3e0fc2275c9..dc752892d64 100644 --- a/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/HashQParserPlugin.java @@ -59,7 +59,7 @@ public class HashQParserPlugin extends QParserPlugin { return new HashQParser(query, localParams, params, request); } - private class HashQParser extends QParser { + private static class HashQParser extends QParser { public HashQParser(String query, SolrParams localParams, SolrParams params, SolrQueryRequest request) { super(query, localParams, params, request); @@ -74,7 +74,7 @@ public class HashQParserPlugin extends QParserPlugin { } } - private class HashQuery extends ExtendedQueryBase implements PostFilter { + private static class HashQuery extends ExtendedQueryBase implements PostFilter { private String keysParam; private int workers; @@ -135,7 +135,7 @@ public class HashQParserPlugin extends QParserPlugin { return searcher.rewrite(constantScoreQuery).createWeight(searcher, false, boost); } - public class BitsFilter extends Filter { + public static class BitsFilter extends Filter { private FixedBitSet[] bitSets; public BitsFilter(FixedBitSet[] bitSets) { this.bitSets = bitSets; @@ -166,7 +166,7 @@ public class HashQParserPlugin extends QParserPlugin { } - class SegmentPartitioner implements Runnable { + static class SegmentPartitioner implements Runnable { public LeafReaderContext context; private int worker; @@ -238,7 +238,7 @@ public class HashQParserPlugin extends QParserPlugin { } } - private class HashCollector extends DelegatingCollector { + private static class HashCollector extends DelegatingCollector { private int worker; private int workers; private HashKey hashKey; @@ -271,7 +271,7 @@ public class HashQParserPlugin extends QParserPlugin { public long hashCode(int doc) throws IOException; } - private class BytesHash implements HashKey { + private static class BytesHash implements HashKey { private SortedDocValues values; private String field; @@ -303,7 +303,7 @@ public class HashQParserPlugin extends QParserPlugin { } } - private class NumericHash implements HashKey { + private static class NumericHash implements HashKey { private NumericDocValues values; private String field; @@ -331,7 +331,7 @@ public class HashQParserPlugin extends QParserPlugin { } } - private class ZeroHash implements HashKey { + private static class ZeroHash implements HashKey { public long hashCode(int doc) { return 0; @@ -342,7 +342,7 @@ public class HashQParserPlugin extends QParserPlugin { } } - private class CompositeHash implements HashKey { + private static class CompositeHash implements HashKey { private HashKey key1; private HashKey key2; diff --git a/solr/core/src/java/org/apache/solr/search/ReRankCollector.java b/solr/core/src/java/org/apache/solr/search/ReRankCollector.java index 1ac1eaff436..8f6bb76baa0 100644 --- a/solr/core/src/java/org/apache/solr/search/ReRankCollector.java +++ b/solr/core/src/java/org/apache/solr/search/ReRankCollector.java @@ -139,7 +139,7 @@ public class ReRankCollector extends TopDocsCollector { } } - public class BoostedComp implements Comparator { + public static class BoostedComp implements Comparator { IntFloatHashMap boostedMap; public BoostedComp(IntIntHashMap boostedDocs, ScoreDoc[] scoreDocs, float maxScore) { diff --git a/solr/core/src/java/org/apache/solr/search/ReRankQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/ReRankQParserPlugin.java index 3e8bf864be5..1190cdb2bca 100644 --- a/solr/core/src/java/org/apache/solr/search/ReRankQParserPlugin.java +++ b/solr/core/src/java/org/apache/solr/search/ReRankQParserPlugin.java @@ -71,7 +71,7 @@ public class ReRankQParserPlugin extends QParserPlugin { } } - private final class ReRankQueryRescorer extends QueryRescorer { + private static final class ReRankQueryRescorer extends QueryRescorer { final double reRankWeight; diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java index 9f9991918d3..63e8743c32c 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java @@ -169,7 +169,7 @@ public class FacetFieldMerger extends FacetRequestSortedMerger { - private class FacetNumBucketsMerger extends FacetMerger { + private static class FacetNumBucketsMerger extends FacetMerger { long sumBuckets; long shardsMissingSum; long shardsTruncatedSum; diff --git a/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java b/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java index 5e1e97c900b..4cdfe02b5a5 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java +++ b/solr/core/src/java/org/apache/solr/search/facet/UniqueAgg.java @@ -187,7 +187,7 @@ public class UniqueAgg extends StrAggValueSource { } - class NumericAcc extends SlotAcc { + static class NumericAcc extends SlotAcc { SchemaField sf; LongSet[] sets; NumericDocValues values; diff --git a/solr/core/src/java/org/apache/solr/search/function/CollapseScoreFunction.java b/solr/core/src/java/org/apache/solr/search/function/CollapseScoreFunction.java index 5378ea0fd40..3932f56c3a3 100644 --- a/solr/core/src/java/org/apache/solr/search/function/CollapseScoreFunction.java +++ b/solr/core/src/java/org/apache/solr/search/function/CollapseScoreFunction.java @@ -45,7 +45,7 @@ public class CollapseScoreFunction extends ValueSource { return new CollapseScoreFunctionValues(context); } - public class CollapseScoreFunctionValues extends FunctionValues { + public static class CollapseScoreFunctionValues extends FunctionValues { private CollapseScore cscore; diff --git a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java index 3f762e399c8..db416519602 100644 --- a/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java +++ b/solr/core/src/java/org/apache/solr/search/join/GraphQuery.java @@ -315,7 +315,7 @@ public class GraphQuery extends Query { } - private class GraphScorer extends Scorer { + private static class GraphScorer extends Scorer { final DocIdSetIterator iter; final float score; diff --git a/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java b/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java index c4a75031317..0203f18270b 100644 --- a/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java +++ b/solr/core/src/java/org/apache/solr/spelling/PossibilityIterator.java @@ -360,7 +360,7 @@ public class PossibilityIterator implements throw new UnsupportedOperationException(); } - public class RankedSpellPossibility { + public static class RankedSpellPossibility { public List corrections; public int rank; public int index; @@ -406,7 +406,7 @@ public class PossibilityIterator implements } } - private class StartOffsetComparator implements + private static class StartOffsetComparator implements Comparator { @Override public int compare(SpellCheckCorrection o1, SpellCheckCorrection o2) { @@ -414,7 +414,7 @@ public class PossibilityIterator implements } } - private class RankComparator implements Comparator { + private static class RankComparator implements Comparator { // Rank poorer suggestions ahead of better ones for use with a PriorityQueue @Override public int compare(RankedSpellPossibility r1, RankedSpellPossibility r2) { 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 daa50a9621f..84a20052b99 100644 --- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java +++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java @@ -204,7 +204,7 @@ public static final int VERSION_IDX = 1; } }; - public class DBQ { + public static class DBQ { public String q; // the query string public long version; // positive version of the DBQ diff --git a/solr/core/src/java/org/apache/solr/util/RTimer.java b/solr/core/src/java/org/apache/solr/util/RTimer.java index 7bf075cafef..19624603ecc 100644 --- a/solr/core/src/java/org/apache/solr/util/RTimer.java +++ b/solr/core/src/java/org/apache/solr/util/RTimer.java @@ -41,7 +41,7 @@ public class RTimer { double elapsed(); } - private class NanoTimeTimerImpl implements TimerImpl { + private static class NanoTimeTimerImpl implements TimerImpl { private long start; public void start() { start = System.nanoTime(); 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 a8ef372acb8..e64208936f5 100644 --- a/solr/core/src/java/org/apache/solr/util/SimplePostTool.java +++ b/solr/core/src/java/org/apache/solr/util/SimplePostTool.java @@ -1065,7 +1065,7 @@ public class SimplePostTool { /** * Inner class to filter files based on glob wildcards */ - class GlobFileFilter implements FileFilter + static class GlobFileFilter implements FileFilter { private String _pattern; private Pattern p; @@ -1258,7 +1258,7 @@ public class SimplePostTool { /** * Utility class to hold the result form a page fetch */ - public class PageFetcherResult { + public static class PageFetcherResult { int httpStatus = 200; String contentType = "text/html"; URL redirectUrl = null; diff --git a/solr/core/src/test/org/apache/solr/cloud/BaseCdcrDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/BaseCdcrDistributedZkTest.java index 35cc44476ea..8a88959fe25 100644 --- a/solr/core/src/test/org/apache/solr/cloud/BaseCdcrDistributedZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/BaseCdcrDistributedZkTest.java @@ -816,7 +816,7 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase { return info; } - protected class CollectionInfo { + protected static class CollectionInfo { List coreInfos = new ArrayList<>(); @@ -869,7 +869,7 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase { this.coreInfos.add(info); } - public class CoreInfo { + public static class CoreInfo { String collectionName; String shard; boolean isLeader; 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 628884c3cda..ffc5262a991 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyNothingIsSafeTest.java @@ -400,8 +400,8 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase SolrInputDocument doc = getDoc(fields); indexDoc(doc); } - - class ErrorLoggingConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient { + + static class ErrorLoggingConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient { public ErrorLoggingConcurrentUpdateSolrClient(String serverUrl, HttpClient httpClient, int queueSize, int threadCount) { super(serverUrl, httpClient, queueSize, threadCount, null, false); } diff --git a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java b/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java index 17d4c545ebe..90d9cc1445d 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ConnectionManagerTest.java @@ -153,7 +153,7 @@ public class ConnectionManagerTest extends SolrTestCaseJ4 { } } - private class MockZkClientConnectionStrategy extends DefaultConnectionStrategy { + private static class MockZkClientConnectionStrategy extends DefaultConnectionStrategy { int called = 0; boolean exceptionThrown = false; diff --git a/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java b/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java index d1192a182c4..b6754c71bf6 100644 --- a/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java @@ -202,7 +202,7 @@ public class DistributedQueueTest extends SolrTestCaseJ4 { return new DistributedQueue(zkClient, setupNewDistributedQueueZNode(dqZNode)); } - private class QueueChangerThread extends Thread { + private static class QueueChangerThread extends Thread { DistributedQueue dq; long waitBeforeOfferMs; diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java index 2582872259a..cab5ee3876a 100644 --- a/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/LeaderElectionTest.java @@ -83,8 +83,8 @@ public class LeaderElectionTest extends SolrTestCaseJ4 { zkClient.makePath("/collections/collection1", true); zkClient.makePath("/collections/collection2", true); } - - class TestLeaderElectionContext extends ShardLeaderElectionContextBase { + + static class TestLeaderElectionContext extends ShardLeaderElectionContextBase { private long runLeaderDelay = 0; public TestLeaderElectionContext(LeaderElector leaderElector, diff --git a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java index 56c0df90d3b..f4cbc77a7ca 100644 --- a/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/OverriddenZkACLAndCredentialsProvidersTest.java @@ -214,7 +214,7 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 { } } - private class SolrZkClientFactoryUsingCompletelyNewProviders { + private static class SolrZkClientFactoryUsingCompletelyNewProviders { final String digestUsername; final String digestPassword; @@ -274,7 +274,7 @@ public class OverriddenZkACLAndCredentialsProvidersTest extends SolrTestCaseJ4 { } - private class SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames extends SolrZkClient { + private static class SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames extends SolrZkClient { public SolrZkClientUsingVMParamsProvidersButWithDifferentVMParamsNames(String zkServerAddress, int zkClientTimeout) { super(zkServerAddress, zkClientTimeout); diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java index 635292a2226..48ac91f4a6c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/OverseerCollectionConfigSetProcessorTest.java @@ -78,7 +78,7 @@ public class OverseerCollectionConfigSetProcessorTest extends SolrTestCaseJ4 { private Thread thread; private Queue queue = new ArrayBlockingQueue<>(10); - private class OverseerCollectionConfigSetProcessorToBeTested extends + private static class OverseerCollectionConfigSetProcessorToBeTested extends OverseerCollectionConfigSetProcessor { diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java index faa2ba74a05..b3a3d7eeaa0 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java @@ -40,7 +40,7 @@ public class ZkSolrClientTest extends AbstractSolrTestCase { initCore("solrconfig.xml", "schema.xml"); } - class ZkConnection implements AutoCloseable { + static class ZkConnection implements AutoCloseable { private ZkTestServer server = null; private SolrZkClient zkClient = null; diff --git a/solr/core/src/test/org/apache/solr/core/CachingDirectoryFactoryTest.java b/solr/core/src/test/org/apache/solr/core/CachingDirectoryFactoryTest.java index 7a7fb9c8200..8714054eada 100644 --- a/solr/core/src/test/org/apache/solr/core/CachingDirectoryFactoryTest.java +++ b/solr/core/src/test/org/apache/solr/core/CachingDirectoryFactoryTest.java @@ -41,7 +41,7 @@ public class CachingDirectoryFactoryTest extends SolrTestCaseJ4 { private Map dirs = new HashMap<>(); private volatile boolean stop = false; - private class Tracker { + private static class Tracker { String path; AtomicInteger refCnt = new AtomicInteger(0); Directory dir; diff --git a/solr/core/src/test/org/apache/solr/handler/AnalysisRequestHandlerTestBase.java b/solr/core/src/test/org/apache/solr/handler/AnalysisRequestHandlerTestBase.java index d5f84921d84..2b527208ba2 100644 --- a/solr/core/src/test/org/apache/solr/handler/AnalysisRequestHandlerTestBase.java +++ b/solr/core/src/test/org/apache/solr/handler/AnalysisRequestHandlerTestBase.java @@ -49,7 +49,7 @@ public abstract class AnalysisRequestHandlerTestBase extends SolrTestCaseJ4 { //================================================= Inner Classes ================================================== - protected class TokenInfo { + protected static class TokenInfo { private String text; private String rawText; diff --git a/solr/core/src/test/org/apache/solr/handler/PingRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/PingRequestHandlerTest.java index 96a01c384e0..b9ab2c12d9c 100644 --- a/solr/core/src/test/org/apache/solr/handler/PingRequestHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/handler/PingRequestHandlerTest.java @@ -231,7 +231,7 @@ public class PingRequestHandlerTest extends SolrTestCaseJ4 { return rsp; } - class SolrPingWithDistrib extends SolrPing { + static class SolrPingWithDistrib extends SolrPing { public SolrPing setDistrib(boolean distrib) { getParams().add("distrib", distrib ? "true" : "false"); return this; diff --git a/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java index 6d467224d6a..961bb25837a 100644 --- a/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/handler/XmlUpdateRequestHandlerTest.java @@ -194,7 +194,7 @@ public class XmlUpdateRequestHandlerTest extends SolrTestCaseJ4 { p.assertNoCommandsPending(); } - private class MockUpdateRequestProcessor extends UpdateRequestProcessor { + private static class MockUpdateRequestProcessor extends UpdateRequestProcessor { private Queue deleteCommands = new LinkedList<>(); diff --git a/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java index d026ecdeffe..890ef4ad790 100644 --- a/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java +++ b/solr/core/src/test/org/apache/solr/handler/admin/CoreMergeIndexesAdminHandlerTest.java @@ -50,7 +50,7 @@ public class CoreMergeIndexesAdminHandlerTest extends SolrTestCaseJ4 { private static String FAILING_MSG = "Creating a directory using FailingDirectoryFactoryException always fails"; public static class FailingDirectoryFactory extends MockFSDirectoryFactory { - public class FailingDirectoryFactoryException extends RuntimeException { + public static class FailingDirectoryFactoryException extends RuntimeException { public FailingDirectoryFactoryException() { super(FAILING_MSG); } diff --git a/solr/core/src/test/org/apache/solr/internal/csv/CSVParserTest.java b/solr/core/src/test/org/apache/solr/internal/csv/CSVParserTest.java index 0688c5257d8..ba604f82185 100644 --- a/solr/core/src/test/org/apache/solr/internal/csv/CSVParserTest.java +++ b/solr/core/src/test/org/apache/solr/internal/csv/CSVParserTest.java @@ -37,7 +37,7 @@ public class CSVParserTest extends TestCase { /** * TestCSVParser. */ - class TestCSVParser extends CSVParser { + static class TestCSVParser extends CSVParser { /** * Test parser to investigate the type of the internal Token. * @param in a Reader diff --git a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java index e58c9ddc226..f45b19359db 100644 --- a/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java +++ b/solr/core/src/test/org/apache/solr/metrics/reporters/SolrGraphiteReporterTest.java @@ -78,7 +78,7 @@ public class SolrGraphiteReporterTest extends SolrTestCaseJ4 { } } - private class MockGraphite extends Thread { + private static class MockGraphite extends Thread { private List lines = new ArrayList<>(); private ServerSocket server = null; private int port; diff --git a/solr/core/src/test/org/apache/solr/response/TestGraphMLResponseWriter.java b/solr/core/src/test/org/apache/solr/response/TestGraphMLResponseWriter.java index e974b42614e..72af04b23ae 100644 --- a/solr/core/src/test/org/apache/solr/response/TestGraphMLResponseWriter.java +++ b/solr/core/src/test/org/apache/solr/response/TestGraphMLResponseWriter.java @@ -84,7 +84,7 @@ public class TestGraphMLResponseWriter extends SolrTestCaseJ4 { } - private class TestStream extends TupleStream { + private static class TestStream extends TupleStream { private Iterator tuples; diff --git a/solr/core/src/test/org/apache/solr/rest/TestManagedResource.java b/solr/core/src/test/org/apache/solr/rest/TestManagedResource.java index cce210984f9..c06abf00b51 100644 --- a/solr/core/src/test/org/apache/solr/rest/TestManagedResource.java +++ b/solr/core/src/test/org/apache/solr/rest/TestManagedResource.java @@ -49,7 +49,7 @@ public class TestManagedResource extends SolrTestCaseJ4 { * Mock class that acts like an analysis component that depends on * data managed by a ManagedResource */ - private class MockAnalysisComponent implements ManagedResourceObserver { + private static class MockAnalysisComponent implements ManagedResourceObserver { private boolean wasNotified = false; @@ -121,7 +121,7 @@ public class TestManagedResource extends SolrTestCaseJ4 { /** * Implements a Java serialization based storage format. */ - private class SerializableStorage extends ManagedResourceStorage { + private static class SerializableStorage extends ManagedResourceStorage { SerializableStorage(StorageIO storageIO, SolrResourceLoader loader) { super(storageIO, loader); diff --git a/solr/core/src/test/org/apache/solr/rest/TestRestManager.java b/solr/core/src/test/org/apache/solr/rest/TestRestManager.java index ab2c45b4eed..cc6c58a4c90 100644 --- a/solr/core/src/test/org/apache/solr/rest/TestRestManager.java +++ b/solr/core/src/test/org/apache/solr/rest/TestRestManager.java @@ -62,7 +62,7 @@ public class TestRestManager extends SolrRestletTestBase { } - private class MockAnalysisComponent implements ManagedResourceObserver { + private static class MockAnalysisComponent implements ManagedResourceObserver { @Override public void onManagedResourceInitialized(NamedList args, ManagedResource res) diff --git a/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java b/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java index e724da95ed4..e6e4da19f0c 100644 --- a/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java +++ b/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java @@ -248,7 +248,7 @@ public class TestCloudManagedSchemaConcurrent extends AbstractFullDistribZkTestB schemaLockTest(); } - private class Info { + private static class Info { int numAddFieldPuts = 0; int numAddFieldPosts = 0; int numAddDynamicFieldPuts = 0; diff --git a/solr/core/src/test/org/apache/solr/search/AnalyticsTestQParserPlugin.java b/solr/core/src/test/org/apache/solr/search/AnalyticsTestQParserPlugin.java index 392fb82546b..158df1722d1 100644 --- a/solr/core/src/test/org/apache/solr/search/AnalyticsTestQParserPlugin.java +++ b/solr/core/src/test/org/apache/solr/search/AnalyticsTestQParserPlugin.java @@ -42,7 +42,7 @@ public class AnalyticsTestQParserPlugin extends QParserPlugin { return new TestAnalyticsQueryParser(query, localParams, params, req); } - class TestAnalyticsQueryParser extends QParser { + static class TestAnalyticsQueryParser extends QParser { public TestAnalyticsQueryParser(String query, SolrParams localParams, SolrParams params, SolrQueryRequest req) { super(query, localParams, params, req); @@ -58,7 +58,7 @@ public class AnalyticsTestQParserPlugin extends QParserPlugin { } } - class TestAnalyticsQuery extends AnalyticsQuery { + static class TestAnalyticsQuery extends AnalyticsQuery { private int base; @@ -72,7 +72,7 @@ public class AnalyticsTestQParserPlugin extends QParserPlugin { } } - class TestAnalyticsCollector extends DelegatingCollector { + static class TestAnalyticsCollector extends DelegatingCollector { ResponseBuilder rb; int count; int base; @@ -97,7 +97,7 @@ public class AnalyticsTestQParserPlugin extends QParserPlugin { } } - class TestAnalyticsMergeStrategy implements MergeStrategy { + static class TestAnalyticsMergeStrategy implements MergeStrategy { public boolean mergesIds() { return false; @@ -130,7 +130,7 @@ public class AnalyticsTestQParserPlugin extends QParserPlugin { } } - class TestIterative extends IterativeMergeStrategy { + static class TestIterative extends IterativeMergeStrategy { public void process(ResponseBuilder rb, ShardRequest sreq) throws Exception { int count = 0; diff --git a/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java b/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java index 27bf40fb818..a887fedaeff 100644 --- a/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java +++ b/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java @@ -1830,7 +1830,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { return false; } - class MultilanguageQueryParser extends ExtendedDismaxQParser { + static class MultilanguageQueryParser extends ExtendedDismaxQParser { public MultilanguageQueryParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) { @@ -1857,10 +1857,10 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { } } - - - - class FuzzyDismaxQParser extends ExtendedDismaxQParser { + + + + static class FuzzyDismaxQParser extends ExtendedDismaxQParser { public FuzzyDismaxQParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) { diff --git a/solr/core/src/test/org/apache/solr/search/TestFiltering.java b/solr/core/src/test/org/apache/solr/search/TestFiltering.java index 9f9a51a047f..c7182832b38 100644 --- a/solr/core/src/test/org/apache/solr/search/TestFiltering.java +++ b/solr/core/src/test/org/apache/solr/search/TestFiltering.java @@ -174,7 +174,7 @@ public class TestFiltering extends SolrTestCaseJ4 { } - class Model { + static class Model { int indexSize; FixedBitSet answer; FixedBitSet multiSelect; diff --git a/solr/core/src/test/org/apache/solr/search/TestRankQueryPlugin.java b/solr/core/src/test/org/apache/solr/search/TestRankQueryPlugin.java index b42861ace7b..a526621f8d0 100644 --- a/solr/core/src/test/org/apache/solr/search/TestRankQueryPlugin.java +++ b/solr/core/src/test/org/apache/solr/search/TestRankQueryPlugin.java @@ -75,7 +75,7 @@ public class TestRankQueryPlugin extends QParserPlugin { return new TestRankQueryParser(query, localParams, params, req); } - class TestRankQueryParser extends QParser { + static class TestRankQueryParser extends QParser { public TestRankQueryParser(String query, SolrParams localParams, SolrParams params, SolrQueryRequest req) { super(query, localParams, params, req); @@ -89,7 +89,7 @@ public class TestRankQueryPlugin extends QParserPlugin { } } - class TestRankQuery extends RankQuery { + static class TestRankQuery extends RankQuery { private int mergeStrategy; private int collector; @@ -143,7 +143,7 @@ public class TestRankQueryPlugin extends QParserPlugin { } } - class TestMergeStrategy implements MergeStrategy { + static class TestMergeStrategy implements MergeStrategy { public int getCost() { return 1; @@ -314,7 +314,7 @@ public class TestRankQueryPlugin extends QParserPlugin { } } - class TestMergeStrategy1 implements MergeStrategy { + static class TestMergeStrategy1 implements MergeStrategy { public int getCost() { return 1; @@ -435,7 +435,7 @@ public class TestRankQueryPlugin extends QParserPlugin { } } - private class FakeScorer extends Scorer { + private static class FakeScorer extends Scorer { final int docid; final float score; @@ -674,7 +674,7 @@ public class TestRankQueryPlugin extends QParserPlugin { } - class TestCollector extends TopDocsCollector { + static class TestCollector extends TopDocsCollector { private List list = new ArrayList(); @@ -743,7 +743,7 @@ public class TestRankQueryPlugin extends QParserPlugin { } } - class TestCollector1 extends TopDocsCollector { + static class TestCollector1 extends TopDocsCollector { private List list = new ArrayList(); diff --git a/solr/core/src/test/org/apache/solr/spelling/ConjunctionSolrSpellCheckerTest.java b/solr/core/src/test/org/apache/solr/spelling/ConjunctionSolrSpellCheckerTest.java index 2a7daa99e1b..31f20fb8f04 100644 --- a/solr/core/src/test/org/apache/solr/spelling/ConjunctionSolrSpellCheckerTest.java +++ b/solr/core/src/test/org/apache/solr/spelling/ConjunctionSolrSpellCheckerTest.java @@ -44,8 +44,8 @@ public class ConjunctionSolrSpellCheckerTest extends LuceneTestCase { // correct behavior } } - - class MockSolrSpellChecker extends SolrSpellChecker { + + static class MockSolrSpellChecker extends SolrSpellChecker { final StringDistance sd; diff --git a/solr/core/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java b/solr/core/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java index 75de4db7ca6..b2210444992 100644 --- a/solr/core/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java +++ b/solr/core/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java @@ -230,7 +230,7 @@ public class IndexBasedSpellCheckerTest extends SolrTestCaseJ4 { } } - private class TestSpellChecker extends IndexBasedSpellChecker{ + private static class TestSpellChecker extends IndexBasedSpellChecker{ @Override public SpellChecker getSpellChecker(){ return spellChecker; diff --git a/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java b/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java index 4272c248b8c..99f7ce6df66 100644 --- a/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java +++ b/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java @@ -37,7 +37,7 @@ import org.junit.Test; public class BlockDirectoryTest extends SolrTestCaseJ4 { - private class MapperCache implements Cache { + private static class MapperCache implements Cache { public Map map = Caffeine.newBuilder() .maximumSize(8) .build() diff --git a/solr/core/src/test/org/apache/solr/util/SimplePostToolTest.java b/solr/core/src/test/org/apache/solr/util/SimplePostToolTest.java index 7f70024b9b6..c9f8fc51c0a 100644 --- a/solr/core/src/test/org/apache/solr/util/SimplePostToolTest.java +++ b/solr/core/src/test/org/apache/solr/util/SimplePostToolTest.java @@ -225,7 +225,7 @@ public class SimplePostToolTest extends SolrTestCaseJ4 { @Override public PageFetcherResult readPageFromUrl(URL u) { - PageFetcherResult res = (new SimplePostTool()).new PageFetcherResult(); + PageFetcherResult res = new PageFetcherResult(); if (isDisallowedByRobots(u)) { res.httpStatus = 403; return res; diff --git a/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java b/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java index e41484fbd32..af85ad67e13 100644 --- a/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java +++ b/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java @@ -400,7 +400,7 @@ public class SolrPluginUtilsTest extends SolrTestCaseJ4 { assertEquals(3, q.build().getMinimumNumberShouldMatch()); } - private class InvokeSettersTestClass { + private static class InvokeSettersTestClass { private float aFloat = random().nextFloat(); public float getAFloat() { return aFloat; diff --git a/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java b/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java index 28a58ff69e4..7980560f899 100644 --- a/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java +++ b/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java @@ -72,7 +72,7 @@ public class TestSolrCLIRunExample extends SolrTestCaseJ4 { * Overrides the call to exec bin/solr to start Solr nodes to start them using the Solr test-framework * instead of the script, since the script depends on a full build. */ - private class RunExampleExecutor extends DefaultExecutor implements Closeable { + private static class RunExampleExecutor extends DefaultExecutor implements Closeable { private PrintStream stdout; private List commandsExecuted = new ArrayList<>(); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java index 4eac2a534a2..fa9350321e2 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClient.java @@ -460,7 +460,7 @@ public class ConcurrentUpdateSolrClient extends SolrClient { /** * Class representing an UpdateRequest and an optional collection. */ - class Update { + static class Update { UpdateRequest request; String collection; /** diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java index 4676594835b..1d7e46fa4ef 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ModelCache.java @@ -110,7 +110,7 @@ public class ModelCache { } } - private class Model { + private static class Model { private Tuple tuple; private long lastChecked; @@ -128,7 +128,7 @@ public class ModelCache { } } - private class LRU extends LinkedHashMap { + private static class LRU extends LinkedHashMap { private int maxSize; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java index 805967775a0..bbd76692e44 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/GatherNodesStream.java @@ -624,7 +624,7 @@ public class GatherNodesStream extends TupleStream implements Expressible { return null; } - class NodeStream extends TupleStream { + static class NodeStream extends TupleStream { private List ids; private Iterator it; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java index aa546ae9e0c..5075330fde4 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/graph/ShortestPathStream.java @@ -492,7 +492,7 @@ public class ShortestPathStream extends TupleStream implements Expressible { } } - private class Edge { + private static class Edge { private String from; private String to; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GroupOperation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GroupOperation.java index 4a28cc1e0fd..a2bd8c999c9 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GroupOperation.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/ops/GroupOperation.java @@ -133,7 +133,7 @@ public class GroupOperation implements ReduceOperation { } } - class ReverseComp implements Comparator, Serializable { + static class ReverseComp implements Comparator, Serializable { private StreamComparator comp; public ReverseComp(StreamComparator comp) { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RankStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RankStream.java index a9fb1afd2c8..e3010816ab2 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RankStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/RankStream.java @@ -213,7 +213,7 @@ public class RankStream extends TupleStream implements Expressible { return 0; } - class ReverseComp implements Comparator, Serializable { + static class ReverseComp implements Comparator, Serializable { private static final long serialVersionUID = 1L; private StreamComparator comp; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java index b4decd528ae..45cd27227de 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SignificantTermsStream.java @@ -339,7 +339,7 @@ public class SignificantTermsStream extends TupleStream implements Expressible{ } } - private class ScoreComp implements Comparator { + private static class ScoreComp implements Comparator { public int compare(Map a, Map b) { Float scorea = (Float)a.get("score"); Float scoreb = (Float)b.get("score"); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java index 5a70cedeb79..a6ba9d93019 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java @@ -532,7 +532,7 @@ public class TextLogitStream extends TupleStream implements Expressible { return buf.toString(); } - protected class TermsStream extends TupleStream { + protected static class TermsStream extends TupleStream { private List terms; private Iterator it; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/SpellCheckResponse.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/SpellCheckResponse.java index 946225dceb9..89ef351f136 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/response/SpellCheckResponse.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/SpellCheckResponse.java @@ -211,7 +211,7 @@ public class SpellCheckResponse { } - public class Collation { + public static class Collation { private String collationQueryString; private List misspellingsAndCorrections = new ArrayList<>(); private long numberOfHits; @@ -244,7 +244,7 @@ public class SpellCheckResponse { } - public class Correction { + public static class Correction { private String original; private String correction; 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 80531cb60ee..7cefbee733e 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 @@ -145,7 +145,7 @@ public class ZkStateReader implements Closeable { private final ExecutorService notifications = ExecutorUtil.newMDCAwareCachedThreadPool("watches"); - private class CollectionWatch { + private static class CollectionWatch { int coreRefCount = 0; Set stateWatchers = ConcurrentHashMap.newKeySet(); diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java index 964ea2666ad..3c1c45fcc61 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrClient.java @@ -247,7 +247,7 @@ public class TestLBHttpSolrClient extends SolrTestCaseJ4 { } } - private class SolrInstance { + private static class SolrInstance { String name; File homeDir; File dataDir; diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java index 093238d52d8..02ed7be06fa 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/embedded/SolrExampleStreamingTest.java @@ -104,8 +104,8 @@ public class SolrExampleStreamingTest extends SolrExampleTests { null, failures.get(0)); } } - - class FailureRecordingConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient { + + static class FailureRecordingConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient { private final List failures = new ArrayList<>(); public FailureRecordingConcurrentUpdateSolrClient(String serverUrl, int queueSize, int numThreads) { diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java index de728bdc894..24f08d28ee7 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrClientTest.java @@ -252,7 +252,7 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase { } - class SendDocsRunnable implements Runnable { + static class SendDocsRunnable implements Runnable { private String id; private int numDocs; @@ -289,8 +289,8 @@ public class ConcurrentUpdateSolrClientTest extends SolrJettyTestBase { } } } - - class OutcomeCountingConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient { + + static class OutcomeCountingConcurrentUpdateSolrClient extends ConcurrentUpdateSolrClient { private final AtomicInteger successCounter; private final AtomicInteger failureCounter; private final StringBuilder errors; diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java index 04fc3ffc4c8..f82ccc6e975 100644 --- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java +++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java @@ -1627,7 +1627,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase { } } - protected class FldType { + protected static class FldType { public String fname; public IVals numValues; public Vals vals; From c189139e5222d2a8b6fba6bfc5c1194d68b46a77 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 28 Mar 2017 15:21:35 +0200 Subject: [PATCH 073/563] LUCENE-7751: Avoid boxing primitives only to call compareTo. --- lucene/CHANGES.txt | 3 +++ .../core/src/java/org/apache/lucene/util/VirtualMethod.java | 2 +- .../java/org/apache/solr/spelling/SpellCheckCollation.java | 2 +- .../handler/component/DistributedFacetPivotSmallTest.java | 5 ++--- .../src/java/org/apache/solr/common/cloud/DocRouter.java | 4 ++-- 5 files changed, 9 insertions(+), 7 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index e6da58639d3..a8f24c73ff2 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -87,6 +87,9 @@ Other * LUCENE-7754: Inner classes should be static whenever possible. (Daniel Jelinski via Adrien Grand) +* LUCENE-7751: Avoid boxing primitives only to call compareTo. + (Daniel Jelinski via Adrien Grand) + ======================= Lucene 6.5.0 ======================= API Changes diff --git a/lucene/core/src/java/org/apache/lucene/util/VirtualMethod.java b/lucene/core/src/java/org/apache/lucene/util/VirtualMethod.java index f21c16bbda2..ea56862a21c 100644 --- a/lucene/core/src/java/org/apache/lucene/util/VirtualMethod.java +++ b/lucene/core/src/java/org/apache/lucene/util/VirtualMethod.java @@ -143,7 +143,7 @@ public final class VirtualMethod { public static int compareImplementationDistance(final Class clazz, final VirtualMethod m1, final VirtualMethod m2) { - return Integer.valueOf(m1.getImplementationDistance(clazz)).compareTo(m2.getImplementationDistance(clazz)); + return Integer.compare(m1.getImplementationDistance(clazz), m2.getImplementationDistance(clazz)); } } diff --git a/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollation.java b/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollation.java index bef3ec689f7..7f3bd95080f 100644 --- a/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollation.java +++ b/solr/core/src/java/org/apache/solr/spelling/SpellCheckCollation.java @@ -26,7 +26,7 @@ public class SpellCheckCollation implements Comparable { @Override public int compareTo(SpellCheckCollation scc) { - int c = new Integer(internalRank).compareTo(scc.internalRank); + int c = Integer.compare(internalRank, scc.internalRank); if (c == 0) { return collationQuery.compareTo(scc.collationQuery); } diff --git a/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallTest.java b/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallTest.java index fc7af80dd60..d293e69af25 100644 --- a/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallTest.java +++ b/solr/core/src/test/org/apache/solr/handler/component/DistributedFacetPivotSmallTest.java @@ -1636,8 +1636,7 @@ public class DistributedFacetPivotSmallTest extends BaseDistributedSearchTestCas @Override public int compare(PivotField o1, PivotField o2) { - Integer compare = (Integer.valueOf(o2.getCount())).compareTo(Integer - .valueOf(o1.getCount())); + int compare = Integer.compare(o2.getCount(), o1.getCount()); if (compare == 0) { compare = ((String) o2.getValue()).compareTo((String) o1.getValue()); } @@ -1650,7 +1649,7 @@ public class DistributedFacetPivotSmallTest extends BaseDistributedSearchTestCas } } if (compare == 0) { - compare = Integer.valueOf(o1.getFacetQuery().size()).compareTo( + compare = Integer.compare(o1.getFacetQuery().size(), o2.getFacetQuery().size()); } } diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java index 6fffb3a5a79..846c25e7c5c 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocRouter.java @@ -134,8 +134,8 @@ public abstract class DocRouter { @Override public int compareTo(Range that) { - int mincomp = Integer.valueOf(this.min).compareTo(that.min); - return mincomp == 0 ? Integer.valueOf(this.max).compareTo(that.max) : mincomp; + int mincomp = Integer.compare(this.min, that.min); + return mincomp == 0 ? Integer.compare(this.max, that.max) : mincomp; } } From 390ef9af9fa081c90370f69b001cf7ec83c8f1bb Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 28 Mar 2017 15:25:16 +0200 Subject: [PATCH 074/563] LUCENE-7743: Avoid calling new String(String). --- lucene/CHANGES.txt | 3 +++ .../lucene/analysis/compound/hyphenation/TernaryTree.java | 2 +- .../java/org/apache/lucene/analysis/cn/smart/Utility.java | 8 ++++---- .../src/test/org/apache/lucene/index/TestIndexWriter.java | 2 +- .../org/apache/lucene/util/TestRamUsageEstimator.java | 8 ++++---- .../solr/handler/dataimport/AbstractDIHCacheTestCase.java | 2 +- .../org/apache/solr/highlight/DefaultSolrHighlighter.java | 3 +-- solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java | 4 ++-- .../org/apache/solr/response/TestSolrQueryResponse.java | 2 +- .../schema/TrieIntPrefixActsAsRangeQueryFieldType.java | 2 +- 10 files changed, 19 insertions(+), 17 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index a8f24c73ff2..92f01a9e80f 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -90,6 +90,9 @@ Other * LUCENE-7751: Avoid boxing primitives only to call compareTo. (Daniel Jelinski via Adrien Grand) +* LUCENE-7743: Never call new String(String). + (Daniel Jelinski via Adrien Grand) + ======================= Lucene 6.5.0 ======================= API Changes diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/TernaryTree.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/TernaryTree.java index 6aeb69b2ca0..a331d2a2b14 100644 --- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/TernaryTree.java +++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/hyphenation/TernaryTree.java @@ -516,7 +516,7 @@ public class TernaryTree implements Cloneable { @Override public String nextElement() { - String res = new String(curkey); + String res = curkey; cur = up(); run(); return res; diff --git a/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/Utility.java b/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/Utility.java index aca80e7eb56..81ca52e2c21 100644 --- a/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/Utility.java +++ b/lucene/analysis/smartcn/src/java/org/apache/lucene/analysis/cn/smart/Utility.java @@ -24,16 +24,16 @@ import org.apache.lucene.analysis.cn.smart.hhmm.SegTokenFilter; // for javadoc */ public class Utility { - public static final char[] STRING_CHAR_ARRAY = new String("未##串") + public static final char[] STRING_CHAR_ARRAY = "未##串" .toCharArray(); - public static final char[] NUMBER_CHAR_ARRAY = new String("未##数") + public static final char[] NUMBER_CHAR_ARRAY = "未##数" .toCharArray(); - public static final char[] START_CHAR_ARRAY = new String("始##始") + public static final char[] START_CHAR_ARRAY = "始##始" .toCharArray(); - public static final char[] END_CHAR_ARRAY = new String("末##末").toCharArray(); + public static final char[] END_CHAR_ARRAY = "末##末".toCharArray(); /** * Delimiters will be filtered to this character by {@link SegTokenFilter} diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java index d153ac3dca1..660280b4dd7 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java @@ -1154,7 +1154,7 @@ public class TestIndexWriter extends LuceneTestCase { t.finish = true; t.join(); if (t.failed) { - fail(new String(t.bytesLog.toString("UTF-8"))); + fail(t.bytesLog.toString("UTF-8")); } } diff --git a/lucene/core/src/test/org/apache/lucene/util/TestRamUsageEstimator.java b/lucene/core/src/test/org/apache/lucene/util/TestRamUsageEstimator.java index cf53c2d242e..34128ad4d26 100644 --- a/lucene/core/src/test/org/apache/lucene/util/TestRamUsageEstimator.java +++ b/lucene/core/src/test/org/apache/lucene/util/TestRamUsageEstimator.java @@ -24,7 +24,7 @@ import java.util.Random; public class TestRamUsageEstimator extends LuceneTestCase { public void testSanity() { - assertTrue(sizeOf(new String("test string")) > shallowSizeOfInstance(String.class)); + assertTrue(sizeOf("test string") > shallowSizeOfInstance(String.class)); Holder holder = new Holder(); holder.holder = new Holder("string2", 5000L); @@ -37,9 +37,9 @@ public class TestRamUsageEstimator extends LuceneTestCase { shallowSizeOfInstance(Holder.class) == shallowSizeOfInstance(HolderSubclass2.class)); String[] strings = new String[] { - new String("test string"), - new String("hollow"), - new String("catchmaster") + "test string", + "hollow", + "catchmaster" }; assertTrue(sizeOf(strings) > shallowSizeOf(strings)); } diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java index 2ef5a911834..8c7109d5307 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java @@ -47,7 +47,7 @@ public class AbstractDIHCacheTestCase { @Before public void setup() { try { - APPLE = new SerialClob(new String("Apples grow on trees and they are good to eat.").toCharArray()); + APPLE = new SerialClob("Apples grow on trees and they are good to eat.".toCharArray()); } catch (SQLException sqe) { Assert.fail("Could not Set up Test"); } diff --git a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java index 098e1f76f9c..24304d0a1b6 100644 --- a/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java +++ b/solr/core/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java @@ -733,9 +733,8 @@ public class DefaultSolrHighlighter extends SolrHighlighter implements PluginInf if( alternateFieldLen <= 0 ){ altList.add(encoder.encodeText(altText)); } else{ - //note: seemingly redundant new String(...) releases memory to the larger text. But is copying better? altList.add( len + altText.length() > alternateFieldLen ? - encoder.encodeText(new String(altText.substring( 0, alternateFieldLen - len ))) : + encoder.encodeText(altText.substring(0, alternateFieldLen - len)) : encoder.encodeText(altText) ); len += altText.length(); if( len >= alternateFieldLen ) break; diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java index 8b29d4854bb..dc999f1c2c7 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java @@ -272,7 +272,7 @@ public class ZkCLITest extends SolrTestCaseJ4 { @Test public void testGet() throws Exception { String getNode = "/getNode"; - byte [] data = new String("getNode-data").getBytes(StandardCharsets.UTF_8); + byte [] data = "getNode-data".getBytes(StandardCharsets.UTF_8); this.zkClient.create(getNode, data, CreateMode.PERSISTENT, true); String[] args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd", "get", getNode}; @@ -284,7 +284,7 @@ public class ZkCLITest extends SolrTestCaseJ4 { File tmpDir = createTempDir().toFile(); String getNode = "/getFileNode"; - byte [] data = new String("getFileNode-data").getBytes(StandardCharsets.UTF_8); + byte [] data = "getFileNode-data".getBytes(StandardCharsets.UTF_8); this.zkClient.create(getNode, data, CreateMode.PERSISTENT, true); File file = new File(tmpDir, diff --git a/solr/core/src/test/org/apache/solr/response/TestSolrQueryResponse.java b/solr/core/src/test/org/apache/solr/response/TestSolrQueryResponse.java index 8b17dc65474..046582f8d1d 100644 --- a/solr/core/src/test/org/apache/solr/response/TestSolrQueryResponse.java +++ b/solr/core/src/test/org/apache/solr/response/TestSolrQueryResponse.java @@ -79,7 +79,7 @@ public class TestSolrQueryResponse extends LuceneTestCase { final SolrQueryResponse response = new SolrQueryResponse(); assertEquals("response initial value", null, response.getResponse()); final Object newValue = (random().nextBoolean() - ? (random().nextBoolean() ? new String("answer") : new Integer(42)) : null); + ? (random().nextBoolean() ? "answer" : Integer.valueOf(42)) : null); response.addResponse(newValue); assertEquals("response new value", newValue, response.getResponse()); } diff --git a/solr/core/src/test/org/apache/solr/schema/TrieIntPrefixActsAsRangeQueryFieldType.java b/solr/core/src/test/org/apache/solr/schema/TrieIntPrefixActsAsRangeQueryFieldType.java index a58c2682158..4b799755212 100644 --- a/solr/core/src/test/org/apache/solr/schema/TrieIntPrefixActsAsRangeQueryFieldType.java +++ b/solr/core/src/test/org/apache/solr/schema/TrieIntPrefixActsAsRangeQueryFieldType.java @@ -26,7 +26,7 @@ import org.apache.solr.search.QParser; public class TrieIntPrefixActsAsRangeQueryFieldType extends TrieIntField { public Query getPrefixQuery(QParser parser, SchemaField sf, String termStr) { - return getRangeQuery(parser, sf, termStr, new String(Integer.MAX_VALUE + ""), true, false); + return getRangeQuery(parser, sf, termStr, Integer.MAX_VALUE + "", true, false); } } From 1a80e4d6942dd7af214c999e0e6540564efc02ac Mon Sep 17 00:00:00 2001 From: Steve Rowe Date: Tue, 28 Mar 2017 11:47:02 -0400 Subject: [PATCH 075/563] SOLR-10343: Update Solr default/example and test configs to use SynonymGraphFilterFactory --- solr/CHANGES.txt | 2 ++ .../solr/collection1/conf/schema.xml | 8 ++--- .../conf/dataimport-schema-no-unique-key.xml | 4 +-- .../conf/dataimport-solr_id-schema.xml | 8 ++--- .../solr/collection1/conf/schema.xml | 21 +++++++++--- .../solr/collection1/conf/schema.xml | 2 +- .../uima/solr/collection1/conf/schema.xml | 12 +++---- .../uima/uima-tokenizers-schema.xml | 2 +- .../conf/schema-HighlighterMaxOffsetTest.xml | 4 +-- .../conf/schema-copyfield-test.xml | 21 +++++++++--- .../collection1/conf/schema-docValuesJoin.xml | 2 +- .../collection1/conf/schema-psuedo-fields.xml | 2 +- .../conf/schema-required-fields.xml | 21 +++++++++--- .../solr/collection1/conf/schema-rest.xml | 34 +++++++++++++------ .../conf/schema-single-dynamic-copy-field.xml | 34 +++++++++++++------ .../collection1/conf/schema-spellchecker.xml | 2 +- .../conf/schema-synonym-tokenizer.xml | 5 +-- .../solr/collection1/conf/schema-trie.xml | 8 ++--- .../solr/collection1/conf/schema.xml | 25 ++++++++++---- .../solr/collection1/conf/schema11.xml | 12 +++---- .../solr/collection1/conf/schema12.xml | 29 +++++++++++----- .../solr/collection1/conf/schema15.xml | 29 +++++++++++----- .../solr/collection1/conf/schema_latest.xml | 20 ++++++----- .../solr/collection1/conf/schemasurround.xml | 31 ++++++++++++----- .../example-DIH/solr/db/conf/managed-schema | 20 ++++++----- .../example-DIH/solr/mail/conf/managed-schema | 21 +++++++----- .../example-DIH/solr/rss/conf/managed-schema | 24 +++++++------ .../example-DIH/solr/solr/conf/managed-schema | 20 ++++++----- solr/example/files/conf/managed-schema | 12 +++---- .../basic_configs/conf/managed-schema | 20 ++++++----- .../conf/managed-schema | 20 ++++++----- .../conf/managed-schema | 20 ++++++----- 32 files changed, 317 insertions(+), 178 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index b68c62fa97d..2d180a33217 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -156,6 +156,8 @@ Other Changes * SOLR-10344: Update Solr default/example and test configs to use WordDelimiterGraphFilterFactory. (Steve Rowe) +* SOLR-10343: Update Solr default/example and test configs to use SynonymGraphFilterFactory. (Steve Rowe) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml index 02626a0eba4..e9ef1155cc0 100644 --- a/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml +++ b/solr/contrib/clustering/src/test-files/clustering/solr/collection1/conf/schema.xml @@ -166,7 +166,7 @@ @@ -184,7 +184,7 @@ - + @@ -201,7 +201,7 @@ - @@ -213,7 +213,7 @@ - diff --git a/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-schema-no-unique-key.xml b/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-schema-no-unique-key.xml index 84d0dbe0a54..5ef7eda34fe 100644 --- a/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-schema-no-unique-key.xml +++ b/solr/contrib/dataimporthandler-extras/src/test-files/dihextras/solr/collection1/conf/dataimport-schema-no-unique-key.xml @@ -169,7 +169,7 @@ - + diff --git a/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-solr_id-schema.xml b/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-solr_id-schema.xml index da4bd3941cb..a99a0b42332 100644 --- a/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-solr_id-schema.xml +++ b/solr/contrib/dataimporthandler/src/test-files/dih/solr/collection1/conf/dataimport-solr_id-schema.xml @@ -167,7 +167,7 @@ - + @@ -197,7 +197,7 @@ - + @@ -209,7 +209,7 @@ - + diff --git a/solr/contrib/extraction/src/test-files/extraction/solr/collection1/conf/schema.xml b/solr/contrib/extraction/src/test-files/extraction/solr/collection1/conf/schema.xml index b743ed892d5..b65c3456be8 100644 --- a/solr/contrib/extraction/src/test-files/extraction/solr/collection1/conf/schema.xml +++ b/solr/contrib/extraction/src/test-files/extraction/solr/collection1/conf/schema.xml @@ -301,9 +301,14 @@ - + - + + + + + + @@ -311,9 +316,17 @@ synonyms "better" --> - + - + + + + + + + diff --git a/solr/contrib/ltr/src/test-files/solr/collection1/conf/schema.xml b/solr/contrib/ltr/src/test-files/solr/collection1/conf/schema.xml index 0b958c0aca3..f27f0924ec8 100644 --- a/solr/contrib/ltr/src/test-files/solr/collection1/conf/schema.xml +++ b/solr/contrib/ltr/src/test-files/solr/collection1/conf/schema.xml @@ -63,7 +63,7 @@ - + diff --git a/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/schema.xml b/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/schema.xml index 89d44e6616d..f1aebb61f0a 100644 --- a/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/schema.xml +++ b/solr/contrib/uima/src/test-files/uima/solr/collection1/conf/schema.xml @@ -203,7 +203,7 @@ @@ -220,7 +220,7 @@ - @@ -242,7 +242,7 @@ positionIncrementGap="100"> - @@ -261,7 +261,7 @@ - @@ -292,7 +292,7 @@ - @@ -326,7 +326,7 @@ - diff --git a/solr/contrib/uima/src/test-files/uima/uima-tokenizers-schema.xml b/solr/contrib/uima/src/test-files/uima/uima-tokenizers-schema.xml index 229d69bd586..a6a4943e928 100644 --- a/solr/contrib/uima/src/test-files/uima/uima-tokenizers-schema.xml +++ b/solr/contrib/uima/src/test-files/uima/uima-tokenizers-schema.xml @@ -203,7 +203,7 @@ diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-HighlighterMaxOffsetTest.xml b/solr/core/src/test-files/solr/collection1/conf/schema-HighlighterMaxOffsetTest.xml index 122d4ce8a35..fce717c9b6d 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-HighlighterMaxOffsetTest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-HighlighterMaxOffsetTest.xml @@ -57,7 +57,7 @@ Test for HighlighterMaxOffsetTest which requires the use of ReversedWildcardFilt - + @@ -76,7 +76,7 @@ Test for HighlighterMaxOffsetTest which requires the use of ReversedWildcardFilt - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml b/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml index a9df7f80c9b..665114aa910 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-copyfield-test.xml @@ -290,9 +290,14 @@ - + - + + + + + + @@ -300,9 +305,17 @@ synonyms "better" --> - + - + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml index baed872dd7f..d152d981720 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-docValuesJoin.xml @@ -51,7 +51,7 @@ - + - + - + - + + + + + + @@ -283,9 +288,17 @@ synonyms "better" --> - + - + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml index 7d9bf02bc42..aa4c21d2301 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml @@ -114,7 +114,7 @@ - + @@ -149,7 +150,7 @@ - + @@ -178,7 +179,7 @@ - + - + - + + + + + + @@ -420,7 +426,7 @@ - + @@ -445,9 +451,17 @@ synonyms "better" --> - + - + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-single-dynamic-copy-field.xml b/solr/core/src/test-files/solr/collection1/conf/schema-single-dynamic-copy-field.xml index 72dc723b66b..96be1d97497 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-single-dynamic-copy-field.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-single-dynamic-copy-field.xml @@ -114,7 +114,7 @@ - + @@ -149,7 +150,7 @@ - + @@ -178,7 +179,7 @@ - + - + - + + + + + + @@ -420,7 +426,7 @@ - + @@ -445,9 +451,17 @@ synonyms "better" --> - + - + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-spellchecker.xml b/solr/core/src/test-files/solr/collection1/conf/schema-spellchecker.xml index 11e7dba90e6..42c17da8e58 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-spellchecker.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-spellchecker.xml @@ -54,7 +54,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-synonym-tokenizer.xml b/solr/core/src/test-files/solr/collection1/conf/schema-synonym-tokenizer.xml index b4431388ba1..e3ff5ec197f 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-synonym-tokenizer.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-synonym-tokenizer.xml @@ -15,7 +15,7 @@ limitations under the License. --> - + @@ -23,8 +23,9 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml b/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml index 375036f0fe3..f3d4c101e42 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-trie.xml @@ -162,7 +162,7 @@ - + @@ -192,7 +192,7 @@ - + @@ -204,7 +204,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema.xml b/solr/core/src/test-files/solr/collection1/conf/schema.xml index 23ac3260078..bdda244674a 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema.xml @@ -213,7 +213,7 @@ - + @@ -221,7 +221,7 @@ - + @@ -388,9 +388,14 @@ - + - + + + + + + @@ -398,9 +403,17 @@ synonyms "better" --> - + - + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema11.xml b/solr/core/src/test-files/solr/collection1/conf/schema11.xml index db0770baf3b..cccf79a8dca 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema11.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema11.xml @@ -158,7 +158,7 @@ @@ -170,7 +170,7 @@ - + @@ -186,7 +186,7 @@ - + @@ -197,7 +197,7 @@ - + @@ -426,7 +426,7 @@ valued. --> @@ -438,7 +438,7 @@ valued. --> - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema12.xml b/solr/core/src/test-files/solr/collection1/conf/schema12.xml index db913771f8c..7ea770be1fb 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema12.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema12.xml @@ -123,7 +123,7 @@ - + - + - + - + + + + + + @@ -390,7 +395,7 @@ - + @@ -415,9 +420,17 @@ synonyms "better" --> - + - + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema15.xml b/solr/core/src/test-files/solr/collection1/conf/schema15.xml index 8fb8d4410b2..57c6bf10535 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema15.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema15.xml @@ -115,7 +115,7 @@ - + - + - + - + + + + + + @@ -388,7 +393,7 @@ - + @@ -413,9 +418,17 @@ synonyms "better" --> - + - + + + + + + + diff --git a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml index 1163ab5a01d..b19a5120d2b 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema_latest.xml @@ -484,14 +484,15 @@ - + @@ -505,7 +506,8 @@ @@ -523,7 +525,7 @@ - + @@ -569,7 +571,7 @@ - + - + @@ -602,7 +604,7 @@ - + @@ -627,7 +629,7 @@ - + diff --git a/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml b/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml index 30719685672..a0c29c9ec28 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schemasurround.xml @@ -116,7 +116,7 @@ - + - + - + - + + + + + + @@ -383,7 +388,7 @@ - + @@ -415,7 +420,7 @@ - + @@ -427,9 +432,17 @@ synonyms "better" --> - + - + + + + + + + diff --git a/solr/example/example-DIH/solr/db/conf/managed-schema b/solr/example/example-DIH/solr/db/conf/managed-schema index 1a1012fe850..067767ed460 100644 --- a/solr/example/example-DIH/solr/db/conf/managed-schema +++ b/solr/example/example-DIH/solr/db/conf/managed-schema @@ -443,14 +443,15 @@ - + @@ -464,7 +465,8 @@ @@ -482,7 +484,7 @@ - + @@ -526,7 +528,7 @@ - + - + @@ -556,7 +558,7 @@ - + @@ -580,7 +582,7 @@ - + diff --git a/solr/example/example-DIH/solr/mail/conf/managed-schema b/solr/example/example-DIH/solr/mail/conf/managed-schema index 016f105513c..382dacfb3b2 100644 --- a/solr/example/example-DIH/solr/mail/conf/managed-schema +++ b/solr/example/example-DIH/solr/mail/conf/managed-schema @@ -362,14 +362,15 @@ - + @@ -383,7 +384,8 @@ @@ -401,7 +403,7 @@ - + @@ -445,7 +448,7 @@ - + - + @@ -475,7 +478,7 @@ - + @@ -499,7 +502,7 @@ - + diff --git a/solr/example/example-DIH/solr/rss/conf/managed-schema b/solr/example/example-DIH/solr/rss/conf/managed-schema index 2064c5895b0..9417902c92e 100644 --- a/solr/example/example-DIH/solr/rss/conf/managed-schema +++ b/solr/example/example-DIH/solr/rss/conf/managed-schema @@ -239,7 +239,7 @@ @@ -252,7 +252,7 @@ - + @@ -394,14 +394,15 @@ - + @@ -415,7 +416,8 @@ @@ -433,7 +435,7 @@ - + @@ -477,7 +479,7 @@ - + - + @@ -507,7 +509,7 @@ - + @@ -531,7 +533,7 @@ - + diff --git a/solr/example/example-DIH/solr/solr/conf/managed-schema b/solr/example/example-DIH/solr/solr/conf/managed-schema index 04c85c08bb5..f6c5103a493 100644 --- a/solr/example/example-DIH/solr/solr/conf/managed-schema +++ b/solr/example/example-DIH/solr/solr/conf/managed-schema @@ -443,14 +443,15 @@ - + @@ -464,7 +465,8 @@ @@ -482,7 +484,7 @@ - + @@ -526,7 +528,7 @@ - + - + @@ -556,7 +558,7 @@ - + @@ -580,7 +582,7 @@ - + diff --git a/solr/example/files/conf/managed-schema b/solr/example/files/conf/managed-schema index ff209be05dc..3c47c35a881 100644 --- a/solr/example/files/conf/managed-schema +++ b/solr/example/files/conf/managed-schema @@ -133,7 +133,7 @@ - + @@ -153,7 +153,7 @@ - + @@ -164,7 +164,7 @@ - + @@ -175,7 +175,7 @@ - + @@ -246,7 +246,7 @@ - + @@ -259,7 +259,7 @@ - + diff --git a/solr/server/solr/configsets/basic_configs/conf/managed-schema b/solr/server/solr/configsets/basic_configs/conf/managed-schema index 09aaae31d53..360a2967cbc 100644 --- a/solr/server/solr/configsets/basic_configs/conf/managed-schema +++ b/solr/server/solr/configsets/basic_configs/conf/managed-schema @@ -378,14 +378,15 @@ - + @@ -400,7 +401,8 @@ @@ -418,7 +420,7 @@ - + @@ -463,7 +465,7 @@ - + - + @@ -494,7 +496,7 @@ - + @@ -520,7 +522,7 @@ - + diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema index 0319eb05b45..2635a602ca0 100644 --- a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema +++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema @@ -377,14 +377,15 @@ - + @@ -399,7 +400,8 @@ @@ -417,7 +419,7 @@ - + @@ -462,7 +464,7 @@ - + - + @@ -493,7 +495,7 @@ - + @@ -519,7 +521,7 @@ - + diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema index 17dadd48159..27565163d47 100644 --- a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema +++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema @@ -487,14 +487,15 @@ - + @@ -508,7 +509,8 @@ @@ -526,7 +528,7 @@ - + @@ -570,7 +572,7 @@ - + - + @@ -600,7 +602,7 @@ - + @@ -624,7 +626,7 @@ - + From aa2b46a62a52c0d0117312add2a667bf6b14a709 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Tue, 28 Mar 2017 09:25:25 +0100 Subject: [PATCH 076/563] SOLR-10341: SQL AVG function mis-interprets field type --- .../solr/handler/sql/SolrAggregate.java | 6 +- .../solr/handler/sql/SolrEnumerator.java | 1 + .../apache/solr/handler/sql/SolrProject.java | 2 +- .../org/apache/solr/handler/sql/SolrRel.java | 10 +- .../apache/solr/handler/sql/SolrSchema.java | 4 + .../apache/solr/handler/sql/SolrTable.java | 43 ++- .../sql/SolrToEnumerableConverter.java | 1 + .../apache/solr/handler/TestSQLHandler.java | 250 +++++++++++++++--- .../client/solrj/io/stream/FacetStream.java | 8 +- .../client/solrj/io/stream/StatsStream.java | 14 +- .../solrj/io/stream/metrics/MeanMetric.java | 35 ++- .../solrj/io/stream/metrics/Metric.java | 1 + .../StreamExpressionToExpessionTest.java | 8 +- 13 files changed, 320 insertions(+), 63 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrAggregate.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrAggregate.java index 8c4d46d1c14..f207eeb4419 100644 --- a/solr/core/src/java/org/apache/solr/handler/sql/SolrAggregate.java +++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrAggregate.java @@ -69,16 +69,18 @@ class SolrAggregate extends Aggregate implements SolrRel { for(Pair namedAggCall : getNamedAggCalls()) { - AggregateCall aggCall = namedAggCall.getKey(); Pair metric = toSolrMetric(implementor, aggCall, inNames); implementor.addReverseAggMapping(namedAggCall.getValue(), metric.getKey().toLowerCase(Locale.ROOT)+"("+metric.getValue()+")"); implementor.addMetricPair(namedAggCall.getValue(), metric.getKey(), metric.getValue()); + /* if(aggCall.getName() == null) { + System.out.println("AGG:"+namedAggCall.getValue()+":"+ aggCall.getAggregation().getName() + "(" + inNames.get(aggCall.getArgList().get(0)) + ")"); implementor.addFieldMapping(namedAggCall.getValue(), - aggCall.getAggregation().getName() + "(" + inNames.get(aggCall.getArgList().get(0)) + ")"); + aggCall.getAggregation().getName() + "(" + inNames.get(aggCall.getArgList().get(0)) + ")"); } + */ } for(int group : getGroupSet()) { 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 be6046c98fe..7ba3838ce79 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 @@ -43,6 +43,7 @@ class SolrEnumerator implements Enumerator { * @param fields Fields to get from each Tuple */ SolrEnumerator(TupleStream tupleStream, List> fields) { + this.tupleStream = tupleStream; try { this.tupleStream.open(); diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrProject.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrProject.java index c4217f27e38..bd36ba8e4b1 100644 --- a/solr/core/src/java/org/apache/solr/handler/sql/SolrProject.java +++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrProject.java @@ -58,7 +58,7 @@ class SolrProject extends Project implements SolrRel { for (Pair pair : getNamedProjects()) { final String name = pair.right; final String expr = pair.left.accept(translator); - implementor.addFieldMapping(name, expr); + implementor.addFieldMapping(name, expr, false); } } } diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrRel.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrRel.java index d4de2c68c37..370de16d886 100644 --- a/solr/core/src/java/org/apache/solr/handler/sql/SolrRel.java +++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrRel.java @@ -47,9 +47,11 @@ interface SolrRel extends RelNode { RelOptTable table; SolrTable solrTable; - void addFieldMapping(String key, String val) { - if(key != null && !fieldMappings.containsKey(key)) { - this.fieldMappings.put(key, val); + void addFieldMapping(String key, String val, boolean overwrite) { + if(key != null) { + if(overwrite || !fieldMappings.containsKey(key)) { + this.fieldMappings.put(key, val); + } } } @@ -83,7 +85,7 @@ interface SolrRel extends RelNode { String metricIdentifier = metric.toLowerCase(Locale.ROOT) + "(" + column + ")"; if(outName != null) { - this.addFieldMapping(outName, metricIdentifier); + this.addFieldMapping(outName, metricIdentifier, true); } } diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java index 83fa5379f41..20d01f33b34 100644 --- a/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java +++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrSchema.java @@ -99,10 +99,14 @@ class SolrSchema extends AbstractSchema { case "string": type = typeFactory.createJavaType(String.class); break; + case "tint": + case "tlong": case "int": case "long": type = typeFactory.createJavaType(Long.class); break; + case "tfloat": + case "tdouble": case "float": case "double": type = typeFactory.createJavaType(Double.class); 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 e313b440ce7..b7f552b6adf 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 @@ -128,7 +128,7 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { tupleStream = handleSelect(zk, collection, q, fields, orders, limit); } else { if(buckets.isEmpty()) { - tupleStream = handleStats(zk, collection, q, metricPairs); + tupleStream = handleStats(zk, collection, q, metricPairs, fields); } else { if(mapReduce) { tupleStream = handleGroupByMapReduce(zk, @@ -430,6 +430,11 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { final String limit, final String havingPredicate) throws IOException { + Map fmap = new HashMap(); + for(Map.Entry entry : fields) { + fmap.put(entry.getKey(), entry.getValue()); + } + int numWorkers = Integer.parseInt(properties.getProperty("numWorkers", "1")); Bucket[] buckets = buildBuckets(_buckets, fields); @@ -437,6 +442,13 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { if(metrics.length == 0) { return handleSelectDistinctMapReduce(zk, collection, properties, fields, query, orders, buckets, limit); + } else { + for(Metric metric : metrics) { + Class c = fmap.get(metric.getIdentifier()); + if(Long.class.equals(c)) { + metric.outputLong = true; + } + } } Set fieldSet = getFieldSet(metrics, fields); @@ -556,6 +568,12 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { final String lim, final String havingPredicate) throws IOException { + + Map fmap = new HashMap(); + for(Map.Entry f : fields) { + fmap.put(f.getKey(), f.getValue()); + } + ModifiableSolrParams solrParams = new ModifiableSolrParams(); solrParams.add(CommonParams.Q, query); @@ -564,6 +582,13 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { if(metrics.length == 0) { metrics = new Metric[1]; metrics[0] = new CountMetric(); + } else { + for(Metric metric : metrics) { + Class c = fmap.get(metric.getIdentifier()); + if(Long.class.equals(c)) { + metric.outputLong = true; + } + } } int limit = lim != null ? Integer.parseInt(lim) : 1000; @@ -767,12 +792,26 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { private TupleStream handleStats(String zk, String collection, String query, - List> metricPairs) { + List> metricPairs, + List> fields) { + Map fmap = new HashMap(); + for(Map.Entry entry : fields) { + fmap.put(entry.getKey(), entry.getValue()); + } + ModifiableSolrParams solrParams = new ModifiableSolrParams(); solrParams.add(CommonParams.Q, query); Metric[] metrics = buildMetrics(metricPairs, false).toArray(new Metric[0]); + + for(Metric metric : metrics) { + Class c = fmap.get(metric.getIdentifier()); + if(Long.class.equals(c)) { + metric.outputLong = true; + } + } + return new StatsStream(zk, collection, solrParams, metrics); } diff --git a/solr/core/src/java/org/apache/solr/handler/sql/SolrToEnumerableConverter.java b/solr/core/src/java/org/apache/solr/handler/sql/SolrToEnumerableConverter.java index 10d4d4c9688..c97303b4a42 100644 --- a/solr/core/src/java/org/apache/solr/handler/sql/SolrToEnumerableConverter.java +++ b/solr/core/src/java/org/apache/solr/handler/sql/SolrToEnumerableConverter.java @@ -93,6 +93,7 @@ class SolrToEnumerableConverter extends ConverterImpl implements EnumerableRel { } private List generateFields(List queryFields, Map fieldMappings) { + if(fieldMappings.isEmpty()) { return queryFields; } else { 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 cb16f033a24..4889c9071a1 100644 --- a/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java +++ b/solr/core/src/test/org/apache/solr/handler/TestSQLHandler.java @@ -88,6 +88,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { testWhere(); testMixedCaseFields(); testBasicGrouping(); + testBasicGroupingTint(); testBasicGroupingFacets(); testSelectDistinct(); testSelectDistinctFacets(); @@ -669,7 +670,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { commit(); SolrParams sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "map_reduce", - "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s order by sum(field_i) asc limit 2"); + "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), avg(field_i) from collection1 where text='XXXX' group by str_s order by sum(field_i) asc limit 2"); SolrStream solrStream = new SolrStream(jetty.url, sParams); List tuples = getTuples(solrStream); @@ -684,7 +685,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) tuple = tuples.get(1); assert(tuple.get("str_s").equals("a")); @@ -692,10 +693,36 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 27); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 13.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 14); //avg(field_i) + sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "map_reduce", - "stmt", "select str_s as myString, count(*), sum(field_i) as mySum, min(field_i), max(field_i), cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s order by mySum asc limit 2"); + "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), cast(avg(1.0 * field_i) as float) as blah from collection1 where text='XXXX' group by str_s order by sum(field_i) asc limit 2"); + + solrStream = new SolrStream(jetty.url, sParams); + tuples = getTuples(solrStream); + + //Only two results because of the limit. + assert(tuples.size() == 2); + + tuple = tuples.get(0); + assert(tuple.get("str_s").equals("b")); + assert(tuple.getDouble("EXPR$1") == 2); //count(*) + assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) + assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) + assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) + assert(tuple.getDouble("blah") == 9.5); //avg(field_i) + + tuple = tuples.get(1); + assert(tuple.get("str_s").equals("a")); + assert(tuple.getDouble("EXPR$1") == 2); //count(*) + assert(tuple.getDouble("EXPR$2") == 27); //sum(field_i) + assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) + assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) + assert(tuple.getDouble("blah") == 13.5); //avg(field_i) + + sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "map_reduce", + "stmt", "select str_s as myString, count(*), sum(field_i) as mySum, min(field_i), max(field_i), avg(field_i) from collection1 where text='XXXX' group by str_s order by mySum asc limit 2"); solrStream = new SolrStream(jetty.url, sParams); tuples = getTuples(solrStream); @@ -709,7 +736,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("mySum") == 19); assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) tuple = tuples.get(1); assert(tuple.get("myString").equals("a")); @@ -717,11 +744,11 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("mySum") == 27); assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 13.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 14); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "map_reduce", "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " - + "cast(avg(1.0 * field_i) as float) from collection1 where (text='XXXX' AND NOT ((text='XXXY') AND (text='XXXY' OR text='XXXY'))) " + + "avg(field_i) from collection1 where (text='XXXX' AND NOT ((text='XXXY') AND (text='XXXY' OR text='XXXY'))) " + "group by str_s order by str_s desc"); solrStream = new SolrStream(jetty.url, sParams); @@ -746,7 +773,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10D); //avg(field_i) tuple = tuples.get(2); assert(tuple.get("str_s").equals("a")); @@ -755,11 +782,11 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 13.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 14); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "map_reduce", "stmt", "select str_s as myString, count(*) as myCount, sum(field_i) as mySum, min(field_i) as myMin, " - + "max(field_i) as myMax, cast(avg(1.0 * field_i) as float) as myAvg from collection1 " + + "max(field_i) as myMax, avg(field_i) as myAvg from collection1 " + "where (text='XXXX' AND NOT (text='XXXY')) group by str_s order by str_s desc"); solrStream = new SolrStream(jetty.url, sParams); @@ -784,7 +811,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("mySum") == 19); assert(tuple.getDouble("myMin") == 8); assert(tuple.getDouble("myMax") == 11); - assert(tuple.getDouble("myAvg") == 9.5D); + assert(tuple.getDouble("myAvg") == 10); tuple = tuples.get(2); assert(tuple.get("myString").equals("a")); @@ -792,10 +819,10 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("mySum") == 27); assert(tuple.getDouble("myMin") == 7); assert(tuple.getDouble("myMax") == 20); - assert(tuple.getDouble("myAvg") == 13.5D); + assert(tuple.getDouble("myAvg") == 14); sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "map_reduce", - "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), cast(avg(1.0 * field_i) as float) " + + "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), avg(field_i) " + "from collection1 where text='XXXX' group by str_s having sum(field_i) = 19"); solrStream = new SolrStream(jetty.url, sParams); @@ -809,10 +836,10 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "map_reduce", - "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), cast(avg(1.0 * field_i) as float) " + + "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), avg(field_i) " + "from collection1 where text='XXXX' group by str_s having ((sum(field_i) = 19) AND (min(field_i) = 8))"); solrStream = new SolrStream(jetty.url, sParams); @@ -826,11 +853,11 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "map_reduce", "stmt", "select str_s, count(*), sum(field_i) as mySum, min(field_i), max(field_i), " + - "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s " + + "avg(field_i) from collection1 where text='XXXX' group by str_s " + "having ((sum(field_i) = 19) AND (min(field_i) = 8))"); solrStream = new SolrStream(jetty.url, sParams); @@ -844,11 +871,11 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("mySum") == 19); assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "map_reduce", "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " + - "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s " + + "avg(field_i) from collection1 where text='XXXX' group by str_s " + "having ((sum(field_i) = 19) AND (min(field_i) = 100))"); solrStream = new SolrStream(jetty.url, sParams); @@ -860,6 +887,60 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { } } + + private void testBasicGroupingTint() throws Exception { + try { + + CloudJettyRunner jetty = this.cloudJettys.get(0); + + del("*:*"); + + commit(); + + indexr("id", "1", "text", "XXXX XXXX", "str_s", "a", "field_ti", "7"); + indexr("id", "2", "text", "XXXX XXXX", "str_s", "b", "field_ti", "8"); + indexr("id", "3", "text", "XXXX XXXX", "str_s", "a", "field_ti", "20"); + indexr("id", "4", "text", "XXXX XXXX", "str_s", "b", "field_ti", "11"); + indexr("id", "5", "text", "XXXX XXXX", "str_s", "c", "field_ti", "30"); + indexr("id", "6", "text", "XXXX XXXX", "str_s", "c", "field_ti", "40"); + indexr("id", "7", "text", "XXXX XXXX", "str_s", "c", "field_ti", "50"); + indexr("id", "8", "text", "XXXX XXXX", "str_s", "c", "field_ti", "60"); + indexr("id", "9", "text", "XXXX XXXY", "str_s", "d", "field_ti", "70"); + commit(); + + SolrParams sParams = mapParams(CommonParams.QT, "/sql", + "stmt", "select str_s, count(*), sum(field_ti), min(field_ti), max(field_ti), avg(field_ti) from collection1 where text='XXXX' group by str_s order by sum(field_ti) asc limit 2"); + + SolrStream solrStream = new SolrStream(jetty.url, sParams); + List tuples = getTuples(solrStream); + + //Only two results because of the limit. + assert(tuples.size() == 2); + Tuple tuple; + + tuple = tuples.get(0); + assert(tuple.get("str_s").equals("b")); + assert(tuple.getDouble("EXPR$1") == 2); //count(*) + assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) + assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) + assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) + + tuple = tuples.get(1); + assert(tuple.get("str_s").equals("a")); + assert(tuple.getDouble("EXPR$1") == 2); //count(*) + assert(tuple.getDouble("EXPR$2") == 27); //sum(field_i) + assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) + assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) + assert(tuple.getDouble("EXPR$5") == 14); //avg(field_i) + + + + } finally { + delete(); + } + } + private void testSelectDistinctFacets() throws Exception { try { CloudJettyRunner jetty = this.cloudJettys.get(0); @@ -1506,6 +1587,35 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) assert(tuple.getDouble("EXPR$5") == 13.5D); //avg(field_i) + + sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "facet", + "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " + + "avg(field_i) from collection1 where text='XXXX' group by str_s " + + "order by sum(field_i) asc limit 2"); + + solrStream = new SolrStream(jetty.url, sParams); + tuples = getTuples(solrStream); + + //Only two results because of the limit. + assert(tuples.size() == 2); + + tuple = tuples.get(0); + assert(tuple.get("str_s").equals("b")); + assert(tuple.getDouble("EXPR$1") == 2); //count(*) + assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) + assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) + assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) + + tuple = tuples.get(1); + assert(tuple.get("str_s").equals("a")); + assert(tuple.getDouble("EXPR$1") == 2); //count(*) + assert(tuple.getDouble("EXPR$2") == 27); //sum(field_i) + assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) + assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) + assert(tuple.getDouble("EXPR$5") == 14); //avg(field_i) + + sParams = mapParams(CommonParams.QT, "/sql", "aggregationMode", "facet", "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " + "cast(avg(1.0 * field_i) as float) from collection1 where (text='XXXX' AND NOT (text='XXXY')) " @@ -1667,7 +1777,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { SolrParams sParams = mapParams(CommonParams.QT, "/sql", "numWorkers", "2", "aggregationMode", "map_reduce", "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " + - "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s " + + "avg(field_i) from collection1 where text='XXXX' group by str_s " + "order by sum(field_i) asc limit 2"); SolrStream solrStream = new SolrStream(jetty.url, sParams); @@ -1684,7 +1794,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) tuple = tuples.get(1); assert(tuple.get("str_s").equals("a")); @@ -1692,12 +1802,41 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 27); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 13.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 14); //avg(field_i) + + + sParams = mapParams(CommonParams.QT, "/sql", "numWorkers", "2", "aggregationMode", "map_reduce", + "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " + + "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s " + + "order by sum(field_i) asc limit 2"); + + solrStream = new SolrStream(jetty.url, sParams); + tuples = getTuples(solrStream); + + //Only two results because of the limit. + assert(tuples.size() == 2); + + tuple = tuples.get(0); + assert(tuple.get("str_s").equals("b")); + assert(tuple.getDouble("EXPR$1") == 2); //count(*) + assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) + assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) + assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) + assert(tuple.getDouble("EXPR$5") == 9.5); //avg(field_i) + + tuple = tuples.get(1); + assert(tuple.get("str_s").equals("a")); + assert(tuple.getDouble("EXPR$1") == 2); //count(*) + assert(tuple.getDouble("EXPR$2") == 27); //sum(field_i) + assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) + assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) + assert(tuple.getDouble("EXPR$5") == 13.5); //avg(field_i) + sParams = mapParams(CommonParams.QT, "/sql", "numWorkers", "2", "aggregationMode", "map_reduce", "stmt", "select str_s, count(*), sum(field_i) as mySum, min(field_i), max(field_i), " + - "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s order by mySum asc limit 2"); + "avg(field_i) from collection1 where text='XXXX' group by str_s order by mySum asc limit 2"); solrStream = new SolrStream(jetty.url, sParams); tuples = getTuples(solrStream); @@ -1711,7 +1850,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("mySum") == 19); assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) tuple = tuples.get(1); assert(tuple.get("str_s").equals("a")); @@ -1719,12 +1858,12 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("mySum") == 27); assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 13.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 14); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "numWorkers", "2", "aggregationMode", "map_reduce", "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " + - "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s order by str_s desc"); + "avg(field_i) from collection1 where text='XXXX' group by str_s order by str_s desc"); solrStream = new SolrStream(jetty.url, sParams); tuples = getTuples(solrStream); @@ -1748,7 +1887,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) tuple = tuples.get(2); assert(tuple.get("str_s").equals("a")); @@ -1756,12 +1895,12 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 27); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 13.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 14); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "numWorkers", "2", "aggregationMode", "map_reduce", "stmt", "select str_s as myString, count(*), sum(field_i), min(field_i), max(field_i), " + - "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s order by myString desc"); + "avg(field_i) from collection1 where text='XXXX' group by str_s order by myString desc"); solrStream = new SolrStream(jetty.url, sParams); tuples = getTuples(solrStream); @@ -1785,7 +1924,7 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) tuple = tuples.get(2); assert(tuple.get("myString").equals("a")); @@ -1793,12 +1932,12 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 27); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 7); //min(field_i) assert(tuple.getDouble("EXPR$4") == 20); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 13.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 14); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "numWorkers", "2", "aggregationMode", "map_reduce", "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " + - "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s having sum(field_i) = 19"); + "avg(field_i) from collection1 where text='XXXX' group by str_s having sum(field_i) = 19"); solrStream = new SolrStream(jetty.url, sParams); tuples = getTuples(solrStream); @@ -1811,11 +1950,11 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "numWorkers", "2", "aggregationMode", "map_reduce", "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " + - "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s " + + "avg(field_i) from collection1 where text='XXXX' group by str_s " + "having ((sum(field_i) = 19) AND (min(field_i) = 8))"); solrStream = new SolrStream(jetty.url, sParams); @@ -1829,11 +1968,11 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assert(tuple.getDouble("EXPR$2") == 19); //sum(field_i) assert(tuple.getDouble("EXPR$3") == 8); //min(field_i) assert(tuple.getDouble("EXPR$4") == 11); //max(field_i) - assert(tuple.getDouble("EXPR$5") == 9.5D); //avg(field_i) + assert(tuple.getDouble("EXPR$5") == 10); //avg(field_i) sParams = mapParams(CommonParams.QT, "/sql", "numWorkers", "2", "aggregationMode", "map_reduce", "stmt", "select str_s, count(*), sum(field_i), min(field_i), max(field_i), " + - "cast(avg(1.0 * field_i) as float) from collection1 where text='XXXX' group by str_s " + + "avg(field_i) from collection1 where text='XXXX' group by str_s " + "having ((sum(field_i) = 19) AND (min(field_i) = 100))"); solrStream = new SolrStream(jetty.url, sParams); @@ -1933,6 +2072,45 @@ public class TestSQLHandler extends AbstractFullDistribZkTestBase { assertTrue(maxf == 10.0D); assertTrue(avgf == 5.5D); + + //Test without cast on average int field + sParams = mapParams(CommonParams.QT, "/sql", + "stmt", "select count(*) as myCount, sum(a_i) as mySum, min(a_i) as myMin, max(a_i) as myMax, " + + "avg(a_i) as myAvg, sum(a_f), min(a_f), max(a_f), avg(a_f) from collection1"); + + solrStream = new SolrStream(jetty.url, sParams); + + tuples = getTuples(solrStream); + + assert(tuples.size() == 1); + + //Test Long and Double Sums + + tuple = tuples.get(0); + + count = tuple.getDouble("myCount"); + sumi = tuple.getDouble("mySum"); + mini = tuple.getDouble("myMin"); + maxi = tuple.getDouble("myMax"); + avgi = tuple.getDouble("myAvg"); + assertTrue(tuple.get("myAvg") instanceof Long); + sumf = tuple.getDouble("EXPR$5"); //sum(a_f) + minf = tuple.getDouble("EXPR$6"); //min(a_f) + maxf = tuple.getDouble("EXPR$7"); //max(a_f) + avgf = tuple.getDouble("EXPR$8"); //avg(a_f) + + assertTrue(count == 10); + assertTrue(mini == 0.0D); + assertTrue(maxi == 14.0D); + assertTrue(sumi == 70); + assertTrue(avgi == 7); + assertTrue(sumf == 55.0D); + assertTrue(minf == 1.0D); + assertTrue(maxf == 10.0D); + assertTrue(avgf == 5.5D); + + + // Test where clause hits sParams = mapParams(CommonParams.QT, "/sql", "stmt", "select count(*), sum(a_i), min(a_i), max(a_i), cast(avg(1.0 * a_i) as float), sum(a_f), " + diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java index 94d937da566..0180764ff92 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FacetStream.java @@ -234,7 +234,6 @@ public class FacetStream extends TupleStream implements Expressible { this.zkHost = zkHost; this.params = params; this.buckets = buckets; - System.out.println("####### Bucket count:"+buckets.length); this.metrics = metrics; this.bucketSizeLimit = bucketSizeLimit; this.collection = collection; @@ -356,6 +355,7 @@ public class FacetStream extends TupleStream implements Expressible { NamedList response = cloudSolrClient.request(request, collection); getTuples(response, buckets, metrics); Collections.sort(tuples, getStreamSort()); + } catch (Exception e) { throw new IOException(e); } @@ -509,7 +509,11 @@ public class FacetStream extends TupleStream implements Expressible { String identifier = metric.getIdentifier(); if(!identifier.startsWith("count(")) { double d = (double)bucket.get("facet_"+m); - t.put(identifier, d); + if(metric.outputLong) { + t.put(identifier, Math.round(d)); + } else { + t.put(identifier, d); + } ++m; } else { long l = ((Number)bucket.get("count")).longValue(); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StatsStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StatsStream.java index 65389028ee4..f6b5818be72 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StatsStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StatsStream.java @@ -58,6 +58,7 @@ public class StatsStream extends TupleStream implements Expressible { private String collection; private boolean done; private boolean doCount; + private Map metricMap; protected transient SolrClientCache cache; protected transient CloudSolrClient cloudSolrClient; @@ -82,6 +83,10 @@ public class StatsStream extends TupleStream implements Expressible { this.params = params; this.metrics = metrics; this.collection = collection; + metricMap = new HashMap(); + for(Metric metric : metrics) { + metricMap.put(metric.getIdentifier(), metric); + } } public StatsStream(StreamExpression expression, StreamFactory factory) throws IOException{ @@ -321,7 +326,14 @@ public class StatsStream extends TupleStream implements Expressible { private void addStat(Map map, String field, String stat, Object val) { if(stat.equals("mean")) { - map.put("avg("+field+")", val); + String name = "avg("+field+")"; + Metric m = metricMap.get(name); + if(m.outputLong) { + Number num = (Number) val; + map.put(name, Math.round(num.doubleValue())); + } else { + map.put(name, val); + } } else { map.put(stat+"("+field+")", val); } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MeanMetric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MeanMetric.java index 03c037a73ee..14f93b81496 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MeanMetric.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/MeanMetric.java @@ -37,27 +37,36 @@ public class MeanMetric extends Metric { private long count; public MeanMetric(String columnName){ - init("avg", columnName); + init("avg", columnName, false); + } + + public MeanMetric(String columnName, boolean outputLong){ + init("avg", columnName, outputLong); } public MeanMetric(StreamExpression expression, StreamFactory factory) throws IOException{ // grab all parameters out String functionName = expression.getFunctionName(); String columnName = factory.getValueOperand(expression, 0); - + String outputLong = factory.getValueOperand(expression, 1); + + // validate expression contains only what we want. if(null == columnName){ throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expected %s(columnName)", expression, functionName)); } - if(1 != expression.getParameters().size()){ - throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - unknown operands found", expression)); + + boolean ol = false; + if(outputLong != null) { + ol = Boolean.parseBoolean(outputLong); } - init(functionName, columnName); + init(functionName, columnName, ol); } - private void init(String functionName, String columnName){ + private void init(String functionName, String columnName, boolean outputLong){ this.columnName = columnName; + this.outputLong = outputLong; setFunctionName(functionName); setIdentifier(functionName, "(", columnName, ")"); } @@ -75,25 +84,29 @@ public class MeanMetric extends Metric { } public Metric newInstance() { - return new MeanMetric(columnName); + return new MeanMetric(columnName, outputLong); } public String[] getColumns() { return new String[]{columnName}; } - public Double getValue() { + public Number getValue() { double dcount = (double)count; if(longSum == 0) { return doubleSum/dcount; - } else { - return longSum/dcount; + double mean = longSum/dcount; + if(outputLong) { + return Math.round(mean); + } else { + return mean; + } } } @Override public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException { - return new StreamExpression(getFunctionName()).withParameter(columnName); + return new StreamExpression(getFunctionName()).withParameter(columnName).withParameter(Boolean.toString(outputLong)); } } \ No newline at end of file diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Metric.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Metric.java index 582b54ae441..87f7852c526 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Metric.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/metrics/Metric.java @@ -30,6 +30,7 @@ public abstract class Metric implements Expressible { private UUID metricNodeId = UUID.randomUUID(); private String functionName; private String identifier; + public boolean outputLong; // This is only used for SQL in facet mode. public String getFunctionName(){ return functionName; diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java index 4ddf4ce8dce..0a597b7ab80 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionToExpessionTest.java @@ -155,7 +155,7 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase { assertTrue(expressionString.contains("sort=\"a_f asc, a_i asc\"")); assertTrue(expressionString.contains("min(a_i)")); assertTrue(expressionString.contains("max(a_i)")); - assertTrue(expressionString.contains("avg(a_i)")); + assertTrue(expressionString.contains("avg(a_i,false)")); assertTrue(expressionString.contains("count(*)")); assertTrue(expressionString.contains("sum(a_i)")); @@ -274,8 +274,8 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase { assertTrue(expressionString.contains("min(a_f)")); assertTrue(expressionString.contains("max(a_i)")); assertTrue(expressionString.contains("max(a_f)")); - assertTrue(expressionString.contains("avg(a_i)")); - assertTrue(expressionString.contains("avg(a_f)")); + assertTrue(expressionString.contains("avg(a_i,false)")); + assertTrue(expressionString.contains("avg(a_f,false)")); assertTrue(expressionString.contains("count(*)")); } @@ -427,7 +427,7 @@ public class StreamExpressionToExpessionTest extends LuceneTestCase { metric = new MeanMetric(StreamExpressionParser.parse("avg(foo)"), factory); expressionString = metric.toExpression(factory).toString(); - assertEquals("avg(foo)", expressionString); + assertEquals("avg(foo,false)", expressionString); } @Test From 0322068ea4648c93405da5b60fcbcc3467f5b009 Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Wed, 29 Mar 2017 00:26:31 +0530 Subject: [PATCH 077/563] SOLR-10365: Handle a SolrCoreInitializationException while publishing core state during SolrCore creation --- solr/CHANGES.txt | 3 ++ .../org/apache/solr/cloud/ZkController.java | 4 +++ .../org/apache/solr/core/CoreContainer.java | 6 ++-- .../core/SolrCoreInitializationException.java | 32 +++++++++++++++++++ 4 files changed, 41 insertions(+), 4 deletions(-) create mode 100644 solr/core/src/java/org/apache/solr/core/SolrCoreInitializationException.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 2d180a33217..887516037a9 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -158,6 +158,9 @@ Other Changes * SOLR-10343: Update Solr default/example and test configs to use SynonymGraphFilterFactory. (Steve Rowe) +* SOLR-10365: Handle a SolrCoreInitializationException while publishing core state during SolrCore creation + (Ishan Chattopadhyaya) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. 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 a3f1fd477cd..677bf290621 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java @@ -85,6 +85,7 @@ import org.apache.solr.core.CloudConfig; import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreDescriptor; import org.apache.solr.core.SolrCore; +import org.apache.solr.core.SolrCoreInitializationException; import org.apache.solr.logging.MDCLoggingContext; import org.apache.solr.update.UpdateLog; import org.apache.zookeeper.CreateMode; @@ -1232,6 +1233,9 @@ public class ZkController { } } } + } catch (SolrCoreInitializationException ex) { + // The core had failed to initialize (in a previous request, not this one), hence nothing to do here. + log.info("The core '{}' had failed to initialize before.", cd.getName()); } ZkNodeProps m = new ZkNodeProps(props); 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 9e22f91b49c..9db32616e8c 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -1288,7 +1288,7 @@ public class CoreContainer { * @see SolrCore#close() * @param name the core name * @return the core if found, null if a SolrCore by this name does not exist - * @exception SolrException if a SolrCore with this name failed to be initialized + * @exception SolrCoreInitializationException if a SolrCore with this name failed to be initialized */ public SolrCore getCore(String name) { @@ -1307,9 +1307,7 @@ public class CoreContainer { // error with the details for clients attempting to access it. CoreLoadFailure loadFailure = getCoreInitFailures().get(name); if (null != loadFailure) { - throw new SolrException(ErrorCode.SERVER_ERROR, "SolrCore '" + name + - "' is not available due to init failure: " + - loadFailure.exception.getMessage(), loadFailure.exception); + throw new SolrCoreInitializationException(name, loadFailure.exception); } // otherwise the user is simply asking for something that doesn't exist. return null; diff --git a/solr/core/src/java/org/apache/solr/core/SolrCoreInitializationException.java b/solr/core/src/java/org/apache/solr/core/SolrCoreInitializationException.java new file mode 100644 index 00000000000..93b653cdd96 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/core/SolrCoreInitializationException.java @@ -0,0 +1,32 @@ +/* + * 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.core; + +import org.apache.solr.common.SolrException; + +public class SolrCoreInitializationException extends SolrException { + + public SolrCoreInitializationException(ErrorCode code, String msg) { + super(code, msg); + } + + public SolrCoreInitializationException(String coreName, Exception loadException) { + super(ErrorCode.SERVER_ERROR, "SolrCore '" + coreName + + "' is not available due to init failure: " + + loadException.getMessage(), loadException); + } +} \ No newline at end of file From 6b0217b7cbff1216bb4ffbecdba02eb8c5dd3df6 Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Wed, 29 Mar 2017 00:44:27 +0530 Subject: [PATCH 078/563] SOLR-6736: Adding support for uploading zipped configsets using ConfigSets API --- solr/CHANGES.txt | 3 + .../dataimport/EntityProcessorWrapper.java | 23 + .../solr/cloud/ZkSolrResourceLoader.java | 2 +- .../java/org/apache/solr/core/ConfigSet.java | 10 +- .../apache/solr/core/ConfigSetService.java | 29 +- .../org/apache/solr/core/CoreContainer.java | 3 +- .../org/apache/solr/core/CoreDescriptor.java | 17 + .../solr/core/RunExecutableListener.java | 13 +- .../java/org/apache/solr/core/SolrCore.java | 1 + .../solr/handler/admin/ConfigSetsHandler.java | 93 +++- .../apache/solr/handler/loader/XMLLoader.java | 7 + .../solr/security/PermissionNameProvider.java | 2 +- .../org/apache/solr/update/UpdateHandler.java | 16 + ...StatelessScriptUpdateProcessorFactory.java | 6 + .../dih-script-transformer/managed-schema | 25 ++ .../dih-script-transformer/solrconfig.xml | 61 +++ .../configsets/upload/regular/managed-schema | 25 ++ .../configsets/upload/regular/solrconfig.xml | 61 +++ .../regular/xslt/xsl-update-handler-test.xsl | 49 +++ .../managed-schema | 25 ++ .../solrconfig.xml | 69 +++ .../with-script-processor/managed-schema | 25 ++ ...ssleading.extension.updateprocessor.js.txt | 23 + .../with-script-processor/solrconfig.xml | 65 +++ .../apache/solr/cloud/TestConfigSetsAPI.java | 401 +++++++++++++++++- .../apache/solr/core/TestCodecSupport.java | 2 +- .../apache/solr/core/TestDynamicLoading.java | 11 +- .../solr/common/params/ConfigSetParams.java | 1 + .../solr/cloud/MiniSolrCloudCluster.java | 6 +- 29 files changed, 1056 insertions(+), 18 deletions(-) create mode 100644 solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/managed-schema create mode 100644 solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/solrconfig.xml create mode 100644 solr/core/src/test-files/solr/configsets/upload/regular/managed-schema create mode 100644 solr/core/src/test-files/solr/configsets/upload/regular/solrconfig.xml create mode 100644 solr/core/src/test-files/solr/configsets/upload/regular/xslt/xsl-update-handler-test.xsl create mode 100644 solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/managed-schema create mode 100644 solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/solrconfig.xml create mode 100644 solr/core/src/test-files/solr/configsets/upload/with-script-processor/managed-schema create mode 100644 solr/core/src/test-files/solr/configsets/upload/with-script-processor/missleading.extension.updateprocessor.js.txt create mode 100644 solr/core/src/test-files/solr/configsets/upload/with-script-processor/solrconfig.xml diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 887516037a9..49300fe1559 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -123,6 +123,9 @@ New Features * SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs. (Mano Kovacs via Mark Miller) +* SOLR-6736: Adding support for uploading zipped configsets using ConfigSets API (Varun Rajput, Ishan Chattopadhyaya, + Noble Paul, Anshum Gupta, Gregory Chanan) + Optimizations ---------------------- diff --git a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java index 6370d24cf27..8a76e11b0b9 100644 --- a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java +++ b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java @@ -17,6 +17,8 @@ package org.apache.solr.handler.dataimport; import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrException.ErrorCode; +import org.apache.solr.core.SolrCore; import org.apache.solr.handler.dataimport.config.ConfigNameConstants; import org.apache.solr.handler.dataimport.config.Entity; @@ -98,6 +100,9 @@ public class EntityProcessorWrapper extends EntityProcessor { for (String aTransArr : transArr) { String trans = aTransArr.trim(); if (trans.startsWith("script:")) { + // The script transformer is a potential vulnerability, esp. when the script is + // provided from an untrusted source. Check and don't proceed if source is untrusted. + checkIfTrusted(trans); String functionName = trans.substring("script:".length()); ScriptTransformer scriptTransformer = new ScriptTransformer(); scriptTransformer.setFunctionName(functionName); @@ -126,6 +131,24 @@ public class EntityProcessorWrapper extends EntityProcessor { } + private void checkIfTrusted(String trans) { + if (docBuilder != null) { + SolrCore core = docBuilder.dataImporter.getCore(); + boolean trusted = (core != null)? core.getCoreDescriptor().isConfigSetTrusted(): true; + if (!trusted) { + Exception ex = new SolrException(ErrorCode.UNAUTHORIZED, "The configset for this collection was uploaded " + + "without any authentication in place," + + " and this transformer is not available for collections with untrusted configsets. To use this transformer," + + " re-upload the configset after enabling authentication and authorization."); + String msg = "Transformer: " + + trans + + ". " + ex.getMessage(); + log.error(msg); + wrapAndThrow(SEVERE, ex, msg); + } + } + } + @SuppressWarnings("unchecked") static class ReflectionTransformer extends Transformer { final Method meth; diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java index b4137b31d6c..5f32ef2127c 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkSolrResourceLoader.java @@ -81,7 +81,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader { @Override public InputStream openResource(String resource) throws IOException { InputStream is; - String file = configSetZkPath + "/" + resource; + String file = (".".equals(resource)) ? configSetZkPath : configSetZkPath + "/" + resource; int maxTries = 10; Exception exception = null; while (maxTries -- > 0) { diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSet.java b/solr/core/src/java/org/apache/solr/core/ConfigSet.java index c406506d2cf..e0c9fe44def 100644 --- a/solr/core/src/java/org/apache/solr/core/ConfigSet.java +++ b/solr/core/src/java/org/apache/solr/core/ConfigSet.java @@ -32,11 +32,15 @@ public class ConfigSet { private final NamedList properties; - public ConfigSet(String name, SolrConfig solrConfig, IndexSchema indexSchema, NamedList properties) { + private final boolean trusted; + + public ConfigSet(String name, SolrConfig solrConfig, IndexSchema indexSchema, + NamedList properties, boolean trusted) { this.name = name; this.solrconfig = solrConfig; this.indexSchema = indexSchema; this.properties = properties; + this.trusted = trusted; } public String getName() { @@ -54,4 +58,8 @@ public class ConfigSet { public NamedList getProperties() { return properties; } + + public boolean isTrusted() { + return trusted; + } } diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java index e4a135e4c65..13ac9ce1db4 100644 --- a/solr/core/src/java/org/apache/solr/core/ConfigSetService.java +++ b/solr/core/src/java/org/apache/solr/core/ConfigSetService.java @@ -28,6 +28,7 @@ import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import org.apache.solr.cloud.CloudConfigSetService; import org.apache.solr.cloud.ZkController; +import org.apache.solr.cloud.ZkSolrResourceLoader; import org.apache.solr.common.SolrException; import org.apache.solr.common.util.NamedList; import org.apache.solr.schema.IndexSchema; @@ -74,10 +75,22 @@ public abstract class ConfigSetService { SolrResourceLoader coreLoader = createCoreResourceLoader(dcore); try { + + // ConfigSet properties are loaded from ConfigSetProperties.DEFAULT_FILENAME file. + // ConfigSet flags are loaded from the metadata of the ZK node of the configset. + NamedList properties = createConfigSetProperties(dcore, coreLoader); + NamedList flags = getConfigSetFlags(dcore, coreLoader); + + boolean trusted = + (coreLoader instanceof ZkSolrResourceLoader + && flags != null + && flags.get("trusted") != null + && !flags.getBooleanArg("trusted") + ) ? false: true; + SolrConfig solrConfig = createSolrConfig(dcore, coreLoader); IndexSchema schema = createIndexSchema(dcore, solrConfig); - NamedList properties = createConfigSetProperties(dcore, coreLoader); - return new ConfigSet(configName(dcore), solrConfig, schema, properties); + return new ConfigSet(configName(dcore), solrConfig, schema, properties, trusted); } catch (Exception e) { throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not load conf for core " + dcore.getName() + @@ -116,6 +129,18 @@ public abstract class ConfigSetService { return ConfigSetProperties.readFromResourceLoader(loader, cd.getConfigSetPropertiesName()); } + protected NamedList getConfigSetFlags(CoreDescriptor cd, SolrResourceLoader loader) { + if (loader instanceof ZkSolrResourceLoader) { + try { + return ConfigSetProperties.readFromResourceLoader(loader, "."); + } catch (Exception ex) { + return null; + } + } else { + return null; + } + } + /** * Create a SolrResourceLoader for a core * @param cd the core's CoreDescriptor 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 9db32616e8c..5ec34ba8ee1 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -945,7 +945,8 @@ public class CoreContainer { } ConfigSet coreConfig = coreConfigService.getConfig(dcore); - log.info("Creating SolrCore '{}' using configuration from {}", dcore.getName(), coreConfig.getName()); + dcore.setConfigSetTrusted(coreConfig.isTrusted()); + log.info("Creating SolrCore '{}' using configuration from {}, trusted={}", dcore.getName(), coreConfig.getName(), dcore.isConfigSetTrusted()); try { core = new SolrCore(dcore, coreConfig); } catch (SolrException e) { diff --git a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java index a58de4896ce..0dc2c71fe75 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java +++ b/solr/core/src/java/org/apache/solr/core/CoreDescriptor.java @@ -65,6 +65,14 @@ public class CoreDescriptor { public static final String DEFAULT_EXTERNAL_PROPERTIES_FILE = "conf" + File.separator + "solrcore.properties"; + /** + * Whether this core was configured using a configSet that was trusted. + * This helps in avoiding the loading of plugins that have potential + * vulnerabilities, when the configSet was not uploaded from a trusted + * user. + */ + private boolean trustedConfigSet = true; + /** * Get the standard properties in persistable form * @return the standard core properties in persistable form @@ -170,6 +178,7 @@ public class CoreDescriptor { this.coreProperties.setProperty(CORE_NAME, coreName); this.originalCoreProperties.setProperty(CORE_NAME, coreName); this.substitutableProperties.setProperty(SOLR_CORE_PROP_PREFIX + CORE_NAME, coreName); + this.trustedConfigSet = other.trustedConfigSet; } /** @@ -372,4 +381,12 @@ public class CoreDescriptor { public String getConfigSetPropertiesName() { return coreProperties.getProperty(CORE_CONFIGSET_PROPERTIES); } + + public boolean isConfigSetTrusted() { + return trustedConfigSet; + } + + public void setConfigSetTrusted(boolean trusted) { + this.trustedConfigSet = trusted; + } } diff --git a/solr/core/src/java/org/apache/solr/core/RunExecutableListener.java b/solr/core/src/java/org/apache/solr/core/RunExecutableListener.java index ba0046e4089..c6d0090fe37 100644 --- a/solr/core/src/java/org/apache/solr/core/RunExecutableListener.java +++ b/solr/core/src/java/org/apache/solr/core/RunExecutableListener.java @@ -17,8 +17,10 @@ package org.apache.solr.core; import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.util.NamedList; import org.apache.solr.search.SolrIndexSearcher; +import org.apache.solr.util.plugin.SolrCoreAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,7 +32,7 @@ import java.util.ArrayList; /** */ -class RunExecutableListener extends AbstractSolrEventListener { +class RunExecutableListener extends AbstractSolrEventListener implements SolrCoreAware { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); public RunExecutableListener(SolrCore core) { @@ -66,6 +68,15 @@ class RunExecutableListener extends AbstractSolrEventListener { if ("false".equals(args.get("wait")) || Boolean.FALSE.equals(args.get("wait"))) wait=false; } + @Override + public void inform(SolrCore core) { + if (!core.getCoreDescriptor().isConfigSetTrusted()) { + throw new SolrException(ErrorCode.UNAUTHORIZED, "The configset for this collection was uploaded without any authentication in place," + + " and this operation is not available for collections with untrusted configsets. To have this component, re-upload the configset" + + " after enabling authentication and authorization."); + } + } + /** * External executable listener. * 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 9d77c7e2651..a6ba2dca8b4 100644 --- a/solr/core/src/java/org/apache/solr/core/SolrCore.java +++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java @@ -962,6 +962,7 @@ public final class SolrCore implements SolrInfoMBean, SolrMetricProducer, Closea // Finally tell anyone who wants to know resourceLoader.inform(resourceLoader); resourceLoader.inform(this); // last call before the latch is released. + this.updateHandler.informEventListeners(this); } catch (Throwable e) { // release the latch, otherwise we block trying to do the close. This // should be fine, since counting down on a latch of 0 is still fine diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java index 5d6f02c9d6b..d3489df05b0 100644 --- a/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/admin/ConfigSetsHandler.java @@ -16,14 +16,20 @@ */ package org.apache.solr.handler.admin; +import java.io.InputStream; import java.lang.invoke.MethodHandles; - +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.fs.Path; import org.apache.solr.api.Api; import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.cloud.OverseerSolrResponse; @@ -36,6 +42,7 @@ import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.params.ConfigSetParams; import org.apache.solr.common.params.ConfigSetParams.ConfigSetAction; import org.apache.solr.common.params.SolrParams; +import org.apache.solr.common.util.ContentStream; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.common.util.Utils; @@ -43,6 +50,10 @@ import org.apache.solr.core.CoreContainer; import org.apache.solr.handler.RequestHandlerBase; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.response.SolrQueryResponse; +import org.apache.solr.security.AuthenticationPlugin; +import org.apache.solr.security.AuthorizationContext; +import org.apache.solr.security.PermissionNameProvider; +import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET; import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.CONFIGSETS_ACTION_PREFIX; @@ -59,7 +70,7 @@ import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION; /** * A {@link org.apache.solr.request.SolrRequestHandler} for ConfigSets API requests. */ -public class ConfigSetsHandler extends RequestHandlerBase { +public class ConfigSetsHandler extends RequestHandlerBase implements PermissionNameProvider { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); protected final CoreContainer coreContainer; public static long DEFAULT_ZK_TIMEOUT = 300*1000; @@ -95,6 +106,10 @@ public class ConfigSetsHandler extends RequestHandlerBase { ConfigSetAction action = ConfigSetAction.get(a); if (action == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown action: " + a); + if (action == ConfigSetAction.UPLOAD) { + handleConfigUploadRequest(req, rsp); + return; + } invokeAction(req, rsp, action); } else { throw new SolrException(ErrorCode.BAD_REQUEST, "action is a required param"); @@ -121,6 +136,68 @@ public class ConfigSetsHandler extends RequestHandlerBase { } } + private void handleConfigUploadRequest(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception { + String configSetName = req.getParams().get(NAME); + if (StringUtils.isBlank(configSetName)) { + throw new SolrException(ErrorCode.BAD_REQUEST, + "The configuration name should be provided in the \"name\" parameter"); + } + + SolrZkClient zkClient = coreContainer.getZkController().getZkClient(); + String configPathInZk = ZkConfigManager.CONFIGS_ZKNODE + Path.SEPARATOR + configSetName; + + if (zkClient.exists(configPathInZk, true)) { + throw new SolrException(ErrorCode.BAD_REQUEST, + "The configuration " + configSetName + " already exists in zookeeper"); + } + + Iterator contentStreamsIterator = req.getContentStreams().iterator(); + + if (!contentStreamsIterator.hasNext()) { + throw new SolrException(ErrorCode.BAD_REQUEST, + "No stream found for the config data to be uploaded"); + } + + InputStream inputStream = contentStreamsIterator.next().getStream(); + + // Create a node for the configuration in zookeeper + boolean trusted = getTrusted(req); + zkClient.makePath(configPathInZk, ("{\"trusted\": " + Boolean.toString(trusted) + "}"). + getBytes(StandardCharsets.UTF_8), true); + + ZipInputStream zis = new ZipInputStream(inputStream, StandardCharsets.UTF_8); + ZipEntry zipEntry = null; + while ((zipEntry = zis.getNextEntry()) != null) { + String filePathInZk = configPathInZk + "/" + zipEntry.getName(); + if (zipEntry.isDirectory()) { + zkClient.makePath(filePathInZk, true); + } else { + createZkNodeIfNotExistsAndSetData(zkClient, filePathInZk, + IOUtils.toByteArray(zis)); + } + } + zis.close(); + } + + boolean getTrusted(SolrQueryRequest req) { + AuthenticationPlugin authcPlugin = coreContainer.getAuthenticationPlugin(); + log.info("Trying to upload a configset. authcPlugin: {}, user principal: {}", + authcPlugin, req.getUserPrincipal()); + if (authcPlugin != null && req.getUserPrincipal() != null) { + return true; + } + return false; + } + + private void createZkNodeIfNotExistsAndSetData(SolrZkClient zkClient, + String filePathInZk, byte[] data) throws Exception { + if (!zkClient.exists(filePathInZk, true)) { + zkClient.create(filePathInZk, data, CreateMode.PERSISTENT, true); + } else { + zkClient.setData(filePathInZk, data, true); + } + } + private void handleResponse(String operation, ZkNodeProps m, SolrQueryResponse rsp, long timeout) throws KeeperException, InterruptedException { long time = System.nanoTime(); @@ -226,4 +303,16 @@ public class ConfigSetsHandler extends RequestHandlerBase { public Boolean registerV2() { return Boolean.TRUE; } + + @Override + public Name getPermissionName(AuthorizationContext ctx) { + switch (ctx.getHttpMethod()) { + case "GET": + return Name.CONFIG_READ_PERM; + case "POST": + return Name.CONFIG_EDIT_PERM; + default: + return null; + } + } } diff --git a/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java index 000edeed4d7..a07aff24a20 100644 --- a/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java +++ b/solr/core/src/java/org/apache/solr/handler/loader/XMLLoader.java @@ -42,6 +42,7 @@ import com.google.common.collect.Lists; import org.apache.commons.io.IOUtils; import org.apache.solr.common.EmptyEntityResolver; import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; @@ -134,6 +135,12 @@ public class XMLLoader extends ContentStreamLoader { String tr = req.getParams().get(CommonParams.TR,null); if(tr!=null) { + if (req.getCore().getCoreDescriptor().isConfigSetTrusted() == false) { + throw new SolrException(ErrorCode.UNAUTHORIZED, "The configset for this collection was uploaded without any authentication in place," + + " and this operation is not available for collections with untrusted configsets. To use this feature, re-upload the configset" + + " after enabling authentication and authorization."); + } + final Transformer t = getTransformer(tr,req); final DOMResult result = new DOMResult(); diff --git a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java index 425be38360f..9f797487bf0 100644 --- a/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java +++ b/solr/core/src/java/org/apache/solr/security/PermissionNameProvider.java @@ -40,7 +40,7 @@ public interface PermissionNameProvider { CORE_EDIT_PERM("core-admin-edit", null), READ_PERM("read", "*"), UPDATE_PERM("update", "*"), - CONFIG_EDIT_PERM("config-edit", "*"), + CONFIG_EDIT_PERM("config-edit", unmodifiableSet(new HashSet<>(asList("*", null)))), CONFIG_READ_PERM("config-read", "*"), SCHEMA_READ_PERM("schema-read", "*"), SCHEMA_EDIT_PERM("schema-edit", "*"), diff --git a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java index 1cf8a3fc9a3..cbfb0d5f1fc 100644 --- a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java +++ b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java @@ -72,6 +72,22 @@ public abstract class UpdateHandler implements SolrInfoMBean { } } + /** + * Call the {@link SolrCoreAware#inform(SolrCore)} on all the applicable registered listeners. + */ + public void informEventListeners(SolrCore core) { + for (SolrEventListener listener: commitCallbacks) { + if (listener instanceof SolrCoreAware) { + ((SolrCoreAware) listener).inform(core); + } + } + for (SolrEventListener listener: optimizeCallbacks) { + if (listener instanceof SolrCoreAware) { + ((SolrCoreAware) listener).inform(core); + } + } + } + protected void callPostCommitCallbacks() { for (SolrEventListener listener : commitCallbacks) { listener.postCommit(); diff --git a/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java index 72da7b5d37c..190501d2e4a 100644 --- a/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java +++ b/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java @@ -17,6 +17,7 @@ package org.apache.solr.update.processor; import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.core.SolrCore; @@ -217,6 +218,11 @@ public class StatelessScriptUpdateProcessorFactory extends UpdateRequestProcesso @Override public void inform(SolrCore core) { + if (!core.getCoreDescriptor().isConfigSetTrusted()) { + throw new SolrException(ErrorCode.UNAUTHORIZED, "The configset for this collection was uploaded without any authentication in place," + + " and this operation is not available for collections with untrusted configsets. To use this component, re-upload the configset" + + " after enabling authentication and authorization."); + } resourceLoader = core.getResourceLoader(); // test that our engines & scripts are valid diff --git a/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/managed-schema b/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/managed-schema new file mode 100644 index 00000000000..9e2f9471026 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/managed-schema @@ -0,0 +1,25 @@ + + + + + + + + + + diff --git a/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/solrconfig.xml new file mode 100644 index 00000000000..82d0cc95f92 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/dih-script-transformer/solrconfig.xml @@ -0,0 +1,61 @@ + + + + + + + + + ${solr.data.dir:} + + + + ${tests.luceneMatchVersion:LATEST} + + + + ${solr.commitwithin.softcommit:true} + + + + + + explicit + true + text + + + + + + + + + diff --git a/solr/core/src/test-files/solr/configsets/upload/regular/managed-schema b/solr/core/src/test-files/solr/configsets/upload/regular/managed-schema new file mode 100644 index 00000000000..9e2f9471026 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/regular/managed-schema @@ -0,0 +1,25 @@ + + + + + + + + + + diff --git a/solr/core/src/test-files/solr/configsets/upload/regular/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/regular/solrconfig.xml new file mode 100644 index 00000000000..82d0cc95f92 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/regular/solrconfig.xml @@ -0,0 +1,61 @@ + + + + + + + + + ${solr.data.dir:} + + + + ${tests.luceneMatchVersion:LATEST} + + + + ${solr.commitwithin.softcommit:true} + + + + + + explicit + true + text + + + + + + + + + diff --git a/solr/core/src/test-files/solr/configsets/upload/regular/xslt/xsl-update-handler-test.xsl b/solr/core/src/test-files/solr/configsets/upload/regular/xslt/xsl-update-handler-test.xsl new file mode 100644 index 00000000000..2e7359a62b6 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/regular/xslt/xsl-update-handler-test.xsl @@ -0,0 +1,49 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/managed-schema b/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/managed-schema new file mode 100644 index 00000000000..9e2f9471026 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/managed-schema @@ -0,0 +1,25 @@ + + + + + + + + + + diff --git a/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/solrconfig.xml new file mode 100644 index 00000000000..46838416c41 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/with-run-executable-listener/solrconfig.xml @@ -0,0 +1,69 @@ + + + + + + + + + ${solr.data.dir:} + + + + ${tests.luceneMatchVersion:LATEST} + + + + ${solr.commitwithin.softcommit:true} + + + + /var/opt/resin3/__PORT__/scripts/solr/snapshooter + /var/opt/resin3/__PORT__ + true + arg1 arg2 + MYVAR=val1 + + + + + + explicit + true + text + + + + + + + + + diff --git a/solr/core/src/test-files/solr/configsets/upload/with-script-processor/managed-schema b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/managed-schema new file mode 100644 index 00000000000..9e2f9471026 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/managed-schema @@ -0,0 +1,25 @@ + + + + + + + + + + diff --git a/solr/core/src/test-files/solr/configsets/upload/with-script-processor/missleading.extension.updateprocessor.js.txt b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/missleading.extension.updateprocessor.js.txt new file mode 100644 index 00000000000..984e1d82f10 --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/missleading.extension.updateprocessor.js.txt @@ -0,0 +1,23 @@ +function processAdd(cmd) { + // Integer.valueOf is needed here to get a tru java object, because + // all javascript numbers are floating point (ie: java.lang.Double) + cmd.getSolrInputDocument().addField("script_added_i", + java.lang.Integer.valueOf(42)); + cmd.getSolrInputDocument().addField("script_added_d", 42.3); + +} +function processDelete() { + // NOOP +} +function processCommit() { + // NOOP +} +function processRollback() { + // NOOP +} +function processMergeIndexes() { + // NOOP +} +function finish() { + // NOOP +} diff --git a/solr/core/src/test-files/solr/configsets/upload/with-script-processor/solrconfig.xml b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/solrconfig.xml new file mode 100644 index 00000000000..1c62889d19e --- /dev/null +++ b/solr/core/src/test-files/solr/configsets/upload/with-script-processor/solrconfig.xml @@ -0,0 +1,65 @@ + + + + + + + + + ${solr.data.dir:} + + + + ${tests.luceneMatchVersion:LATEST} + + + + ${solr.commitwithin.softcommit:true} + + + + + + explicit + true + text + + + + + + + javascript + missleading.extension.updateprocessor.js.txt + + + + + + + diff --git a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java index 2027a6da6da..6c20ccc113c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestConfigSetsAPI.java @@ -16,50 +16,96 @@ */ package org.apache.solr.cloud; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET; +import static org.apache.solr.common.params.CommonParams.NAME; +import static org.apache.solr.core.ConfigSetProperties.DEFAULT_FILENAME; + import java.io.ByteArrayInputStream; import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.StringReader; +import java.lang.invoke.MethodHandles; +import java.net.URI; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collection; +import java.util.Deque; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; import java.util.Map; import java.util.Properties; import java.util.Set; +import java.util.zip.ZipEntry; +import java.util.zip.ZipOutputStream; -import com.google.common.collect.ImmutableMap; import org.apache.commons.io.FileUtils; +import org.apache.http.HttpEntity; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.message.BasicHeader; +import org.apache.http.util.EntityUtils; +import org.apache.lucene.util.TestUtil; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.client.solrj.SolrClient; +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.embedded.JettySolrRunner; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.impl.HttpClientUtil; +import org.apache.solr.client.solrj.impl.HttpSolrClient; import org.apache.solr.client.solrj.request.ConfigSetAdminRequest; import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Create; import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.Delete; -import org.apache.solr.client.solrj.request.ConfigSetAdminRequest.List; +import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.client.solrj.response.CollectionAdminResponse; import org.apache.solr.client.solrj.response.ConfigSetAdminResponse; +import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrException.ErrorCode; +import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.ZkConfigManager; +import org.apache.solr.common.cloud.ZkStateReader; +import org.apache.solr.common.params.CollectionParams.CollectionAction; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ConfigSetParams; import org.apache.solr.common.params.ConfigSetParams.ConfigSetAction; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.SolrParams; +import org.apache.solr.common.util.Base64; import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.Utils; import org.apache.solr.core.ConfigSetProperties; +import org.apache.solr.core.TestDynamicLoading; +import org.apache.solr.security.BasicAuthIntegrationTest; +import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.noggit.JSONParser; +import org.noggit.ObjectBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.BASE_CONFIGSET; -import static org.apache.solr.common.params.CommonParams.NAME; -import static org.apache.solr.core.ConfigSetProperties.DEFAULT_FILENAME; +import com.google.common.collect.ImmutableMap; /** * Simple ConfigSets API tests on user errors and simple success cases. */ public class TestConfigSetsAPI extends SolrTestCaseJ4 { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private MiniSolrCloudCluster solrCluster; @Override @@ -231,6 +277,349 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 { } } + @Test + public void testUploadErrors() throws Exception { + final SolrClient solrClient = new HttpSolrClient( + solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString()); + + ByteBuffer emptyData = ByteBuffer.allocate(0); + + // Checking error when no configuration name is specified in request + Map map = postDataAndGetResponse(solrCluster.getSolrClient(), + solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString() + + "/admin/configs?action=UPLOAD&wt=json", emptyData, null, null); + assertNotNull(map); + long statusCode = (long) getObjectByPath(map, false, + Arrays.asList("responseHeader", "status")); + assertEquals(400l, statusCode); + + SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(), + AbstractZkTestCase.TIMEOUT, 45000, null); + + // Create dummy config files in zookeeper + zkClient.makePath("/configs/myconf", true); + zkClient.create("/configs/myconf/firstDummyFile", + "first dummy content".getBytes(StandardCharsets.UTF_8), CreateMode.PERSISTENT, true); + zkClient.create("/configs/myconf/anotherDummyFile", + "second dummy content".getBytes(StandardCharsets.UTF_8), CreateMode.PERSISTENT, true); + + // Checking error when configuration name specified already exists + map = postDataAndGetResponse(solrCluster.getSolrClient(), + solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString() + + "/admin/configs?action=UPLOAD&wt=json&name=myconf", emptyData, null, null); + assertNotNull(map); + statusCode = (long) getObjectByPath(map, false, + Arrays.asList("responseHeader", "status")); + assertEquals(400l, statusCode); + assertTrue("Expected file doesnt exist in zk. It's possibly overwritten", + zkClient.exists("/configs/myconf/firstDummyFile", true)); + assertTrue("Expected file doesnt exist in zk. It's possibly overwritten", + zkClient.exists("/configs/myconf/anotherDummyFile", true)); + + zkClient.close(); + solrClient.close(); + } + + @Test + public void testUpload() throws Exception { + String suffix = "-untrusted"; + uploadConfigSet("regular", suffix, null, null); + // try to create a collection with the uploaded configset + createCollection("newcollection", "regular" + suffix, 1, 1, solrCluster.getSolrClient()); + xsltRequest("newcollection"); + } + + @Test + public void testUploadWithRunExecutableListener() throws Exception { + String suffix = "-untrusted"; + uploadConfigSet("with-run-executable-listener", suffix, null, null); + // try to create a collection with the uploaded configset + CollectionAdminResponse resp = createCollection("newcollection3", "with-run-executable-listener" + suffix, 1, 1, solrCluster.getSolrClient()); + log.info("Client saw errors: "+resp.getErrorMessages()); + assertTrue(resp.getErrorMessages() != null && resp.getErrorMessages().size() > 0); + assertTrue(resp.getErrorMessages().getVal(0). + contains("The configset for this collection was uploaded without any authentication")); + } + + @Test + public void testUploadWithScriptUpdateProcessor() throws Exception { + for (boolean withAuthorization: Arrays.asList(false, true)) { + String suffix; + if (withAuthorization) { + suffix = "-trusted"; + protectConfigsHandler(); + uploadConfigSet("with-script-processor", suffix, "solr", "SolrRocks"); + } else { + suffix = "-untrusted"; + uploadConfigSet("with-script-processor", suffix, null, null); + } + // try to create a collection with the uploaded configset + CollectionAdminResponse resp = createCollection("newcollection2", "with-script-processor"+suffix, + 1, 1, solrCluster.getSolrClient()); + + if (withAuthorization) { + scriptRequest("newcollection2"); + } else { + log.info("Client saw errors: "+resp.getErrorMessages()); + assertTrue(resp.getErrorMessages() != null && resp.getErrorMessages().size() > 0); + assertTrue(resp.getErrorMessages().getVal(0). + contains("The configset for this collection was uploaded without any authentication")); + } + } + } + + protected SolrZkClient zkClient() { + ZkStateReader reader = solrCluster.getSolrClient().getZkStateReader(); + if (reader == null) + solrCluster.getSolrClient().connect(); + return solrCluster.getSolrClient().getZkStateReader().getZkClient(); + } + + private void protectConfigsHandler() throws Exception { + String authcPrefix = "/admin/authentication"; + String authzPrefix = "/admin/authorization"; + + String securityJson = "{\n" + + " 'authentication':{\n" + + " 'class':'solr.BasicAuthPlugin',\n" + + " 'credentials':{'solr':'orwp2Ghgj39lmnrZOTm7Qtre1VqHFDfwAEzr0ApbN3Y= Ju5osoAqOX8iafhWpPP01E5P+sg8tK8tHON7rCYZRRw='}},\n" + + " 'authorization':{\n" + + " 'class':'solr.RuleBasedAuthorizationPlugin',\n" + + " 'user-role':{'solr':'admin'},\n" + + " 'permissions':[{'name':'security-edit','role':'admin'}, {'name':'config-edit','role':'admin'}]}}"; + + HttpClient cl = null; + try { + cl = HttpClientUtil.createClient(null); + JettySolrRunner randomJetty = solrCluster.getRandomJetty(random()); + String baseUrl = randomJetty.getBaseUrl().toString(); + + zkClient().setData("/security.json", securityJson.replaceAll("'", "\"").getBytes(UTF_8), true); + BasicAuthIntegrationTest.verifySecurityStatus(cl, baseUrl + authcPrefix, "authentication/class", "solr.BasicAuthPlugin", 50); + BasicAuthIntegrationTest.verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/class", "solr.RuleBasedAuthorizationPlugin", 50); + } finally { + if (cl != null) { + HttpClientUtil.close(cl); + } + } + Thread.sleep(5000); // TODO: Without a delay, the test fails. Some problem with Authc/Authz framework? + } + + private void uploadConfigSet(String configSetName, String suffix, String username, String password) throws Exception { + // Read zipped sample config + ByteBuffer sampleZippedConfig = TestDynamicLoading + .getFileContent( + createTempZipFile("solr/configsets/upload/"+configSetName), false); + + SolrZkClient zkClient = new SolrZkClient(solrCluster.getZkServer().getZkAddress(), + AbstractZkTestCase.TIMEOUT, 45000, null); + try { + ZkConfigManager configManager = new ZkConfigManager(zkClient); + assertFalse(configManager.configExists(configSetName+suffix)); + + Map map = postDataAndGetResponse(solrCluster.getSolrClient(), + solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/admin/configs?action=UPLOAD&wt=json&name="+configSetName+suffix, + sampleZippedConfig, username, password); + assertNotNull(map); + long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status")); + assertEquals(0l, statusCode); + + assertTrue("managed-schema file should have been uploaded", + zkClient.exists("/configs/"+configSetName+suffix+"/managed-schema", true)); + assertTrue("managed-schema file contents on zookeeper are not exactly same as that of the file uploaded in config", + Arrays.equals(zkClient.getData("/configs/"+configSetName+suffix+"/managed-schema", null, null, true), + readFile("solr/configsets/upload/"+configSetName+"/managed-schema"))); + + assertTrue("solrconfig.xml file should have been uploaded", + zkClient.exists("/configs/"+configSetName+suffix+"/solrconfig.xml", true)); + byte data[] = zkClient.getData("/configs/"+configSetName+suffix, null, null, true); + //assertEquals("{\"trusted\": false}", new String(data, StandardCharsets.UTF_8)); + assertTrue("solrconfig.xml file contents on zookeeper are not exactly same as that of the file uploaded in config", + Arrays.equals(zkClient.getData("/configs/"+configSetName+suffix+"/solrconfig.xml", null, null, true), + readFile("solr/configsets/upload/"+configSetName+"/solrconfig.xml"))); + } finally { + zkClient.close(); + } + } + + /** + * Create a zip file (in the temp directory) containing all the files within the specified directory + * and return the path for the zip file. + */ + private String createTempZipFile(String directoryPath) { + File zipFile = new File(solrCluster.getBaseDir().toFile().getAbsolutePath() + + File.separator + TestUtil.randomSimpleString(random(), 6, 8) + ".zip"); + + File directory = TestDynamicLoading.getFile(directoryPath); + log.info("Directory: "+directory.getAbsolutePath()); + try { + zip (directory, zipFile); + log.info("Zipfile: "+zipFile.getAbsolutePath()); + return zipFile.getAbsolutePath(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static void zip(File directory, File zipfile) throws IOException { + URI base = directory.toURI(); + Deque queue = new LinkedList(); + queue.push(directory); + OutputStream out = new FileOutputStream(zipfile); + ZipOutputStream zout = new ZipOutputStream(out); + try { + while (!queue.isEmpty()) { + directory = queue.pop(); + for (File kid : directory.listFiles()) { + String name = base.relativize(kid.toURI()).getPath(); + if (kid.isDirectory()) { + queue.push(kid); + name = name.endsWith("/") ? name : name + "/"; + zout.putNextEntry(new ZipEntry(name)); + } else { + zout.putNextEntry(new ZipEntry(name)); + + InputStream in = new FileInputStream(kid); + try { + byte[] buffer = new byte[1024]; + while (true) { + int readCount = in.read(buffer); + if (readCount < 0) { + break; + } + zout.write(buffer, 0, readCount); + } + } finally { + in.close(); + } + + zout.closeEntry(); + } + } + } + } finally { + zout.close(); + } + } + + private void xsltRequest(String collection) throws SolrServerException, IOException { + String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString(); + try (HttpSolrClient client = new HttpSolrClient(baseUrl + "/" + collection)) { + String xml = + "" + + " " + + " " + + " " + + " " + + " " + + " " + + " " + + ""; + + SolrQuery query = new SolrQuery(); + query.setQuery( "*:*" );//for anything + query.add("qt","/update"); + query.add(CommonParams.TR, "xsl-update-handler-test.xsl"); + query.add("stream.body", xml); + query.add("commit", "true"); + try { + client.query(query); + fail("This should've returned a 401."); + } catch (SolrException ex) { + assertEquals(ErrorCode.UNAUTHORIZED.code, ex.code()); + } + } + } + + public void scriptRequest(String collection) throws SolrServerException, IOException { + SolrClient client = solrCluster.getSolrClient(); + SolrInputDocument doc = sdoc("id", "4055", "subject", "Solr"); + client.add(collection, doc); + client.commit(collection); + + assertEquals("42", client.query(collection, params("q", "*:*")).getResults().get(0).get("script_added_i")); + } + + protected CollectionAdminResponse createCollection(String collectionName, String confSetName, int numShards, + int replicationFactor, SolrClient client) throws SolrServerException, IOException { + ModifiableSolrParams params = new ModifiableSolrParams(); + params.set("action", CollectionAction.CREATE.toString()); + params.set("collection.configName", confSetName); + params.set("name", collectionName); + params.set("numShards", numShards); + params.set("replicationFactor", replicationFactor); + SolrRequest request = new QueryRequest(params); + request.setPath("/admin/collections"); + + CollectionAdminResponse res = new CollectionAdminResponse(); + res.setResponse(client.request(request)); + return res; + } + + public static Map postDataAndGetResponse(CloudSolrClient cloudClient, + String uri, ByteBuffer bytarr, String username, String password) throws IOException { + HttpPost httpPost = null; + HttpEntity entity; + String response = null; + Map m = null; + + try { + httpPost = new HttpPost(uri); + + if (username != null) { + String userPass = username + ":" + password; + String encoded = Base64.byteArrayToBase64(userPass.getBytes(UTF_8)); + BasicHeader header = new BasicHeader("Authorization", "Basic " + encoded); + httpPost.setHeader(header); + } + + httpPost.setHeader("Content-Type", "application/octet-stream"); + httpPost.setEntity(new ByteArrayEntity(bytarr.array(), bytarr + .arrayOffset(), bytarr.limit())); + entity = cloudClient.getLbClient().getHttpClient().execute(httpPost) + .getEntity(); + try { + response = EntityUtils.toString(entity, StandardCharsets.UTF_8); + m = (Map) ObjectBuilder.getVal(new JSONParser( + new StringReader(response))); + } catch (JSONParser.ParseException e) { + fail(e.getMessage()); + } + } finally { + httpPost.releaseConnection(); + } + return m; + } + + private static Object getObjectByPath(Map root, boolean onlyPrimitive, java.util.List hierarchy) { + Map obj = root; + for (int i = 0; i < hierarchy.size(); i++) { + String s = hierarchy.get(i); + if (i < hierarchy.size() - 1) { + if (!(obj.get(s) instanceof Map)) return null; + obj = (Map) obj.get(s); + if (obj == null) return null; + } else { + Object val = obj.get(s); + if (onlyPrimitive && val instanceof Map) { + return null; + } + return val; + } + } + + return false; + } + + private byte[] readFile(String fname) throws IOException { + byte[] buf = null; + try (FileInputStream fis = new FileInputStream(getFile(fname))) { + buf = new byte[fis.available()]; + fis.read(buf); + } + return buf; + } + @Test public void testDeleteErrors() throws Exception { final String baseUrl = solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString(); @@ -304,7 +693,7 @@ public class TestConfigSetsAPI extends SolrTestCaseJ4 { AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT, null); try { // test empty - List list = new List(); + ConfigSetAdminRequest.List list = new ConfigSetAdminRequest.List(); ConfigSetAdminResponse.List response = list.process(solrClient); Collection actualConfigSets = response.getConfigSets(); assertEquals(0, actualConfigSets.size()); diff --git a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java index b6097ab04d9..7d2f174982f 100644 --- a/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java +++ b/solr/core/src/test/org/apache/solr/core/TestCodecSupport.java @@ -218,7 +218,7 @@ public class TestCodecSupport extends SolrTestCaseJ4 { try { c = new SolrCore(new CoreDescriptor(h.getCoreContainer(), newCoreName, testSolrHome.resolve(newCoreName)), - new ConfigSet("fakeConfigset", config, schema, null)); + new ConfigSet("fakeConfigset", config, schema, null, true)); assertNull(h.getCoreContainer().registerCore(newCoreName, c, false, false)); h.coreName = newCoreName; assertEquals("We are not using the correct core", "solrconfig_codec2.xml", h.getCore().getConfigResource()); diff --git a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java index 306b4b2105d..9bbe09f1c00 100644 --- a/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java +++ b/solr/core/src/test/org/apache/solr/core/TestDynamicLoading.java @@ -25,6 +25,7 @@ import org.apache.solr.util.SimplePostTool; import org.junit.BeforeClass; import org.junit.Test; +import java.io.File; import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; @@ -257,8 +258,16 @@ public class TestDynamicLoading extends AbstractFullDistribZkTestBase { } public static ByteBuffer getFileContent(String f) throws IOException { + return getFileContent(f, true); + } + /** + * @param loadFromClassPath if true, it will look in the classpath to find the file, + * otherwise load from absolute filesystem path. + */ + public static ByteBuffer getFileContent(String f, boolean loadFromClassPath) throws IOException { ByteBuffer jar; - try (FileInputStream fis = new FileInputStream(getFile(f))) { + File file = loadFromClassPath ? getFile(f): new File(f); + try (FileInputStream fis = new FileInputStream(file)) { byte[] buf = new byte[fis.available()]; fis.read(buf); jar = ByteBuffer.wrap(buf); diff --git a/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java b/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java index 49c39ec59ac..fde7e57c1c2 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/ConfigSetParams.java @@ -27,6 +27,7 @@ public interface ConfigSetParams public enum ConfigSetAction { CREATE, + UPLOAD, DELETE, LIST; diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java index e8a0c080472..15895d33b77 100644 --- a/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java +++ b/solr/test-framework/src/java/org/apache/solr/cloud/MiniSolrCloudCluster.java @@ -470,7 +470,11 @@ public class MiniSolrCloudCluster { } } } - + + public Path getBaseDir() { + return baseDir; + } + public CloudSolrClient getSolrClient() { return solrClient; } From 0a689f4d95e981e99ae0e80741e7cf1fa74ff60f Mon Sep 17 00:00:00 2001 From: Steve Rowe Date: Tue, 28 Mar 2017 18:39:28 -0400 Subject: [PATCH 079/563] SOLR-10357: Enable edismax and standard query parsers to handle the option combination sow=false / autoGeneratePhraseQueries=true by setting QueryBuilder.autoGenerateMultiTermSynonymsQuery --- solr/CHANGES.txt | 3 + .../org/apache/solr/parser/QueryParser.java | 11 +- .../org/apache/solr/parser/QueryParser.jj | 11 +- .../solr/search/ExtendedDismaxQParser.java | 8 - .../QueryParserConfigurationException.java | 24 --- .../solr/collection1/conf/synonyms.txt | 1 + .../solr/search/TestExtendedDismaxParser.java | 141 ++++++++++++------ .../solr/search/TestSolrQueryParser.java | 48 ++++-- 8 files changed, 148 insertions(+), 99 deletions(-) delete mode 100644 solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 49300fe1559..9d14e59978c 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -164,6 +164,9 @@ Other Changes * SOLR-10365: Handle a SolrCoreInitializationException while publishing core state during SolrCore creation (Ishan Chattopadhyaya) +* SOLR-10357: Enable edismax and standard query parsers to handle the option combination + sow=false / autoGeneratePhraseQueries="true" by setting QueryBuilder.autoGenerateMultiTermSynonymsQuery. + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/core/src/java/org/apache/solr/parser/QueryParser.java b/solr/core/src/java/org/apache/solr/parser/QueryParser.java index d9a64f4f7c0..e846c6e631b 100644 --- a/solr/core/src/java/org/apache/solr/parser/QueryParser.java +++ b/solr/core/src/java/org/apache/solr/parser/QueryParser.java @@ -13,7 +13,6 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.Query; import org.apache.solr.search.SyntaxError; import org.apache.solr.search.QParser; -import org.apache.solr.search.QueryParserConfigurationException; public class QueryParser extends SolrQueryParserBase implements QueryParserConstants { @@ -54,11 +53,11 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst @Override protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { - if ((getAutoGeneratePhraseQueries() || fieldAutoGenPhraseQueries) && splitOnWhitespace == false) { - throw new QueryParserConfigurationException - ("Field '" + field + "': autoGeneratePhraseQueries == true is disallowed when sow/splitOnWhitespace == false"); - } - return super.newFieldQuery(analyzer, field, queryText, quoted, fieldAutoGenPhraseQueries); + setAutoGenerateMultiTermSynonymsPhraseQuery(fieldAutoGenPhraseQueries || getAutoGeneratePhraseQueries()); + // Don't auto-quote graph-aware field queries + boolean treatAsQuoted = getSplitOnWhitespace() + ? (quoted || fieldAutoGenPhraseQueries || getAutoGeneratePhraseQueries()) : quoted; + return super.newFieldQuery(analyzer, field, queryText, treatAsQuoted, false); } // * Query ::= ( Clause )* diff --git a/solr/core/src/java/org/apache/solr/parser/QueryParser.jj b/solr/core/src/java/org/apache/solr/parser/QueryParser.jj index 5eb4ec7c286..d4d65396eca 100644 --- a/solr/core/src/java/org/apache/solr/parser/QueryParser.jj +++ b/solr/core/src/java/org/apache/solr/parser/QueryParser.jj @@ -37,7 +37,6 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.Query; import org.apache.solr.search.SyntaxError; import org.apache.solr.search.QParser; -import org.apache.solr.search.QueryParserConfigurationException; public class QueryParser extends SolrQueryParserBase { @@ -78,11 +77,11 @@ public class QueryParser extends SolrQueryParserBase { @Override protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { - if ((getAutoGeneratePhraseQueries() || fieldAutoGenPhraseQueries) && splitOnWhitespace == false) { - throw new QueryParserConfigurationException - ("Field '" + field + "': autoGeneratePhraseQueries == true is disallowed when sow/splitOnWhitespace == false"); - } - return super.newFieldQuery(analyzer, field, queryText, quoted, fieldAutoGenPhraseQueries); + setAutoGenerateMultiTermSynonymsPhraseQuery(fieldAutoGenPhraseQueries || getAutoGeneratePhraseQueries()); + // Don't auto-quote graph-aware field queries + boolean treatAsQuoted = getSplitOnWhitespace() + ? (quoted || fieldAutoGenPhraseQueries || getAutoGeneratePhraseQueries()) : quoted; + return super.newFieldQuery(analyzer, field, queryText, treatAsQuoted, false); } } diff --git a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java index a4f9312ac14..9825f72b9f3 100644 --- a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java +++ b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java @@ -310,8 +310,6 @@ public class ExtendedDismaxQParser extends QParser { up.setRemoveStopFilter(true); query = up.parse(mainUserQuery); } - } catch (QueryParserConfigurationException e) { - throw e; // Don't ignore configuration exceptions } catch (Exception e) { // ignore failure and reparse later after escaping reserved chars up.exceptions = false; @@ -1083,10 +1081,6 @@ public class ExtendedDismaxQParser extends QParser { @Override protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { - if ((getAutoGeneratePhraseQueries() || fieldAutoGenPhraseQueries) && getSplitOnWhitespace() == false) { - throw new QueryParserConfigurationException - ("Field '" + field + "': autoGeneratePhraseQueries == true is disallowed when sow/splitOnWhitespace == false"); - } Analyzer actualAnalyzer; if (removeStopFilter) { if (nonStopFilterAnalyzerPerField == null) { @@ -1403,8 +1397,6 @@ public class ExtendedDismaxQParser extends QParser { } return null; - } catch (QueryParserConfigurationException e) { - throw e; // Don't ignore configuration exceptions } catch (Exception e) { // an exception here is due to the field query not being compatible with the input text // for example, passing a string to a numeric field. diff --git a/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java b/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java deleted file mode 100644 index 0dd2a338a2c..00000000000 --- a/solr/core/src/java/org/apache/solr/search/QueryParserConfigurationException.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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.search; - -public class QueryParserConfigurationException extends IllegalArgumentException { - public QueryParserConfigurationException(String message) { - super(message); - } -} diff --git a/solr/core/src/test-files/solr/collection1/conf/synonyms.txt b/solr/core/src/test-files/solr/collection1/conf/synonyms.txt index f64cb724a57..54cf2cc11c6 100644 --- a/solr/core/src/test-files/solr/collection1/conf/synonyms.txt +++ b/solr/core/src/test-files/solr/collection1/conf/synonyms.txt @@ -31,3 +31,4 @@ pixima => pixma # multiword synonyms wi fi => wifi +crow blackbird, grackle \ No newline at end of file diff --git a/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java b/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java index a887fedaeff..c4d8bec6cf7 100644 --- a/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java +++ b/solr/core/src/test/org/apache/solr/search/TestExtendedDismaxParser.java @@ -21,6 +21,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.stream.Stream; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; @@ -34,7 +35,6 @@ import org.apache.solr.common.SolrException; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.request.SolrQueryRequest; -import org.apache.solr.schema.TextField; import org.apache.solr.util.SolrPluginUtils; import org.junit.BeforeClass; import org.junit.Test; @@ -66,7 +66,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { "foo_i", "8" )); assertU(adoc("id", "47", "trait_ss", "Pig", - "text_sw", "line up and fly directly at the enemy death cannons, clogging them with wreckage!")); + "text", "line up and fly directly at the enemy death cannons, clogging them with wreckage!")); assertU(adoc("id", "48", "text_sw", "this has gigabyte potential", "foo_i","100")); assertU(adoc("id", "49", "text_sw", "start the big apple end", "foo_i","-100")); assertU(adoc("id", "50", "text_sw", "start new big city end")); @@ -121,22 +121,22 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { // really just test that exceptions aren't thrown by // single + - - assertJQ(req("defType", "edismax", "q", "-", "df", "text_sw", "sow", sow) + assertJQ(req("defType", "edismax", "q", "-", "sow", sow) , "/response=="); - assertJQ(req("defType", "edismax", "q", "+", "df", "text_sw", "sow", sow) + assertJQ(req("defType", "edismax", "q", "+", "sow", sow) , "/response=="); - assertJQ(req("defType", "edismax", "q", "+ - +", "df", "text_sw", "sow", sow) + assertJQ(req("defType", "edismax", "q", "+ - +", "sow", sow) , "/response=="); - assertJQ(req("defType", "edismax", "q", "- + -", "df", "text_sw", "sow", sow) + assertJQ(req("defType", "edismax", "q", "- + -", "sow", sow) , "/response=="); - assertJQ(req("defType", "edismax", "q", "id:47 +", "df", "text_sw", "sow", sow) + assertJQ(req("defType", "edismax", "q", "id:47 +", "sow", sow) , "/response/numFound==1"); - assertJQ(req("defType", "edismax", "q", "id:47 -", "df", "text_sw", "sow", sow) + assertJQ(req("defType", "edismax", "q", "id:47 -", "sow", sow) , "/response/numFound==1"); Random r = random(); @@ -152,7 +152,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { } String q = sb.toString(); - assertJQ(req("defType", "edismax", "q", q, "df", "text_sw", "sow", sow) + assertJQ(req("defType", "edismax", "q", q, "sow", sow) , "/response=="); } } @@ -264,7 +264,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { , twor ); - assertQ(req("defType", "edismax", "qf", "name title subject text_sw", + assertQ(req("defType", "edismax", "qf", "name title subject text", "q","op"), twor ); assertQ(req("defType", "edismax", @@ -277,29 +277,29 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { "q.op", "OR", "q","Order op"), twor ); - assertQ(req("defType", "edismax", "qf", "name title subject text_sw", + assertQ(req("defType", "edismax", "qf", "name title subject text", "q","Order AND op"), oner ); - assertQ(req("defType", "edismax", "qf", "name title subject text_sw", + assertQ(req("defType", "edismax", "qf", "name title subject text", "q","Order and op"), oner ); - assertQ(req("defType", "edismax", "qf", "name title subject text_sw", + assertQ(req("defType", "edismax", "qf", "name title subject text", "q","+Order op"), oner ); - assertQ(req("defType", "edismax", "qf", "name title subject text_sw", + assertQ(req("defType", "edismax", "qf", "name title subject text", "q","Order OR op"), twor ); - assertQ(req("defType", "edismax", "qf", "name title subject text_sw", + assertQ(req("defType", "edismax", "qf", "name title subject text", "q","Order or op"), twor ); - assertQ(req("defType", "edismax", "qf", "name title subject text_sw", + assertQ(req("defType", "edismax", "qf", "name title subject text", "q","*:*"), allr ); - assertQ(req("defType", "edismax", "qf", "name title subject text_sw", + assertQ(req("defType", "edismax", "qf", "name title subject text", "q","star OR (-star)"), allr ); - assertQ(req("defType", "edismax", "qf", "name title subject text_sw", + assertQ(req("defType", "edismax", "qf", "name title subject text", "q","id:42 OR (-id:42)"), allr ); @@ -551,7 +551,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { "//str[@name='parsedquery_toString'][.='+(id:42)^5.0']"); - assertQ(req("defType","edismax", "uf","-*", "q","cannons", "qf","text_sw"), + assertQ(req("defType","edismax", "uf","-*", "q","cannons", "qf","text"), oner); assertQ(req("defType","edismax", "uf","* -id", "q","42", "qf", "id"), oner); @@ -885,7 +885,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { "*[count(//doc)=3]"); assertQ( "Might be double-escaping a client-escaped colon", - req("q", "text_sw:(theos OR thistokenhasa\\:preescapedcolon OR theou)", "defType", "edismax", "qf", "text_sw"), + req("q", "text_sw:(theos OR thistokenhasa\\:preescapedcolon OR theou)", "defType", "edismax", "qf", "text"), "*[count(//doc)=3]"); } @@ -1047,56 +1047,56 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { // "line up and fly directly at the enemy death cannons, clogging them with wreckage!" assertQ("test default operator with mm (AND + 0% => 0 hits)", req("q", "(line notfound) OR notfound", - "qf", "text_sw", + "qf", "text", "q.op", "AND", "mm", "0%", "defType", "edismax") , "*[count(//doc)=0]"); assertQ("test default operator with mm (OR + 0% => 1 hit)", req("q", "line notfound OR notfound", - "qf", "text_sw", + "qf", "text", "q.op", "OR", "mm", "0%", "defType", "edismax") , "*[count(//doc)=1]"); assertQ("test default operator with mm (OR + 100% => 0 hits)", req("q", "line notfound OR notfound", - "qf", "text_sw", + "qf", "text", "q.op", "OR", "mm", "100%", "defType", "edismax") , "*[count(//doc)=0]"); assertQ("test default operator with mm (OR + 35% => 1 hit)", req("q", "line notfound notfound2 OR notfound", - "qf", "text_sw", + "qf", "text", "q.op", "OR", "mm", "35%", "defType", "edismax") , "*[count(//doc)=1]"); assertQ("test default operator with mm (OR + 75% => 0 hits)", req("q", "line notfound notfound2 OR notfound3", - "qf", "text_sw", + "qf", "text", "q.op", "OR", "mm", "75%", "defType", "edismax") , "*[count(//doc)=0]"); assertQ("test default operator with mm (AND + 0% => 1 hit)", req("q", "(line enemy) OR notfound", - "qf", "text_sw", + "qf", "text", "q.op", "AND", "mm", "0%", "defType", "edismax") , "*[count(//doc)=1]"); assertQ("test default operator with mm (AND + 50% => 1 hit)", req("q", "(line enemy) OR (line notfound) OR (death cannons) OR (death notfound)", - "qf", "text_sw", + "qf", "text", "q.op", "AND", "mm", "50%", "defType", "edismax") , "*[count(//doc)=1]"); assertQ("test default operator with mm (AND + 75% => 0 hits)", req("q", "(line enemy) OR (line notfound) OR (death cannons) OR (death notfound)", - "qf", "text_sw", + "qf", "text", "q.op", "AND", "mm", "75%", "defType", "edismax") @@ -1394,16 +1394,6 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { } - // LUCENE-7533 - public void testSplitOnWhitespace_with_autoGeneratePhraseQueries() throws Exception { - assertTrue(((TextField)h.getCore().getLatestSchema().getField("text").getType()).getAutoGeneratePhraseQueries()); - - try (SolrQueryRequest req = req()) { - final QParser qparser = QParser.getParser("{!edismax sow=false fq=text}blah blah)", req); - expectThrows(IllegalArgumentException.class, qparser::getQuery); - } - } - @Test public void testSplitOnWhitespace_Basic() throws Exception { // The "text_sw" field has synonyms loaded from synonyms.txt @@ -1550,7 +1540,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { , "/response/numFound==0" ); assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi* fi", "sow","false") - , "/response/numFound==2" // matches because wi* matches "wifi" in one doc and "with" in another + , "/response/numFound==1" // matches because wi* matches "wifi" ); assertJQ(req("qf","text_sw title", "defType","edismax", "q","w? fi", "sow","false") , "/response/numFound==0" @@ -1720,7 +1710,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { , "/response/numFound==1" ); assertJQ(req("qf","text_sw title", "defType","edismax", "q","AT* wi fi", "sow","false") - , "/response/numFound==2" + , "/response/numFound==1" ); assertJQ(req("qf","text_sw title", "defType","edismax", "q","AT? wi fi", "sow","false") , "/response/numFound==1" @@ -1750,7 +1740,7 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { , "/response/numFound==1" ); assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi AT*", "sow","false") - , "/response/numFound==2" + , "/response/numFound==1" ); assertJQ(req("qf","text_sw title", "defType","edismax", "q","wi fi AT?", "sow","false") , "/response/numFound==1" @@ -1765,11 +1755,74 @@ public class TestExtendedDismaxParser extends SolrTestCaseJ4 { , "/response/numFound==1" ); } - - + + public void testAutoGeneratePhraseQueries() throws Exception { + ModifiableSolrParams noSowParams = new ModifiableSolrParams(); + ModifiableSolrParams sowFalseParams = new ModifiableSolrParams(); + sowFalseParams.add("sow", "false"); + ModifiableSolrParams sowTrueParams = new ModifiableSolrParams(); + sowTrueParams.add("sow", "true"); + + // From synonyms.txt: + // + // crow blackbird, grackle + + try (SolrQueryRequest req = req(sowFalseParams)) { + QParser qParser = QParser.getParser("text:grackle", "edismax", req); // "text" has autoGeneratePhraseQueries="true" + Query q = qParser.getQuery(); + assertEquals("+(text:\"crow blackbird\" text:grackl)", q.toString()); + } + for (SolrParams params : Arrays.asList(noSowParams, sowTrueParams)) { + try (SolrQueryRequest req = req(params)) { + QParser qParser = QParser.getParser("text:grackle", "edismax", req); + Query q = qParser.getQuery(); + assertEquals("+spanOr([spanNear([text:crow, text:blackbird], 0, true), text:grackl])", q.toString()); + } + } + for (SolrParams params : Arrays.asList(noSowParams, sowTrueParams, sowFalseParams)) { + try (SolrQueryRequest req = req(params)) { + QParser qParser = QParser.getParser("text_sw:grackle", "edismax", req); // "text_sw" doesn't specify autoGeneratePhraseQueries => default false + Query q = qParser.getQuery(); + assertEquals("+((+text_sw:crow +text_sw:blackbird) text_sw:grackl)", q.toString()); + } + } + + Stream.of(noSowParams, sowTrueParams, sowFalseParams).forEach(p->p.add("qf", "text text_sw")); + + try (SolrQueryRequest req = req(sowFalseParams)) { + QParser qParser = QParser.getParser("grackle", "edismax", req); + Query q = qParser.getQuery(); + assertEquals("+((text:\"crow blackbird\" text:grackl)" + + " | ((+text_sw:crow +text_sw:blackbird) text_sw:grackl))", + q.toString()); + + qParser = QParser.getParser("grackle wi fi", "edismax", req); + q = qParser.getQuery(); + assertEquals("+(((text:\"crow blackbird\" text:grackl) text:wifi)" + + " | (((+text_sw:crow +text_sw:blackbird) text_sw:grackl) text_sw:wifi))", + q.toString()); + } + + for (SolrParams params : Arrays.asList(noSowParams, sowTrueParams)) { + try (SolrQueryRequest req = req(params)) { + QParser qParser = QParser.getParser("grackle", "edismax", req); + Query q = qParser.getQuery(); + assertEquals("+(spanOr([spanNear([text:crow, text:blackbird], 0, true), text:grackl])" + + " | ((+text_sw:crow +text_sw:blackbird) text_sw:grackl))", + q.toString()); + + qParser = QParser.getParser("grackle wi fi", "edismax", req); + q = qParser.getQuery(); + assertEquals("+((spanOr([spanNear([text:crow, text:blackbird], 0, true), text:grackl])" + + " | ((+text_sw:crow +text_sw:blackbird) text_sw:grackl)) (text:wi | text_sw:wi) (text:fi | text_sw:fi))", + q.toString()); + } + } + } + private boolean containsClause(Query query, String field, String value, - int boost, boolean fuzzy) { + int boost, boolean fuzzy) { float queryBoost = 1f; if (query instanceof BoostQuery) { diff --git a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java index 92bd6c0e2d6..607f0919e91 100644 --- a/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java +++ b/solr/core/src/test/org/apache/solr/search/TestSolrQueryParser.java @@ -34,11 +34,12 @@ import org.apache.lucene.search.TermInSetQuery; import org.apache.lucene.search.TermQuery; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.params.MapSolrParams; +import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.params.SolrParams; import org.apache.solr.core.SolrInfoMBean; import org.apache.solr.parser.QueryParser; import org.apache.solr.query.FilterQuery; import org.apache.solr.request.SolrQueryRequest; -import org.apache.solr.schema.TextField; import org.junit.BeforeClass; import org.junit.Test; import org.noggit.ObjectBuilder; @@ -573,16 +574,6 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 { req.close(); } - // LUCENE-7533 - public void testSplitOnWhitespace_with_autoGeneratePhraseQueries() throws Exception { - assertTrue(((TextField)h.getCore().getLatestSchema().getField("text").getType()).getAutoGeneratePhraseQueries()); - - try (SolrQueryRequest req = req()) { - final QParser qparser = QParser.getParser("{!lucene sow=false qf=text}blah blah", req); - expectThrows(QueryParserConfigurationException.class, qparser::getQuery); - } - } - @Test public void testSplitOnWhitespace_Basic() throws Exception { // The "syn" field has synonyms loaded from synonyms.txt @@ -969,4 +960,39 @@ public class TestSolrQueryParser extends SolrTestCaseJ4 { , "/response/numFound==1" ); } + + @Test + public void testAutoGeneratePhraseQueries() throws Exception { + ModifiableSolrParams noSowParams = new ModifiableSolrParams(); + ModifiableSolrParams sowFalseParams = new ModifiableSolrParams(); + sowFalseParams.add("sow", "false"); + ModifiableSolrParams sowTrueParams = new ModifiableSolrParams(); + sowTrueParams.add("sow", "true"); + + // From synonyms.txt: + // + // crow blackbird, grackle + // + try (SolrQueryRequest req = req()) { + + QParser qParser = QParser.getParser("text:grackle", req); // "text" has autoGeneratePhraseQueries="true" + qParser.setParams(sowFalseParams); + Query q = qParser.getQuery(); + assertEquals("text:\"crow blackbird\" text:grackl", q.toString()); + + for (SolrParams params : Arrays.asList(noSowParams, sowTrueParams)) { + qParser = QParser.getParser("text:grackle", req); + qParser.setParams(params); + q = qParser.getQuery(); + assertEquals("spanOr([spanNear([text:crow, text:blackbird], 0, true), text:grackl])", q.toString()); + } + + for (SolrParams params : Arrays.asList(noSowParams, sowTrueParams, sowFalseParams)) { + qParser = QParser.getParser("text_sw:grackle", req); // "text_sw" doesn't specify autoGeneratePhraseQueries => default false + qParser.setParams(params); + q = qParser.getQuery(); + assertEquals("(+text_sw:crow +text_sw:blackbird) text_sw:grackl", q.toString()); + } + } + } } \ No newline at end of file From 66bfdcbdbab8f294341232946a30a61898228a34 Mon Sep 17 00:00:00 2001 From: yonik Date: Tue, 28 Mar 2017 19:52:51 -0400 Subject: [PATCH 080/563] SOLR-7452: change terminology from _m missing-bucket to _p partial-bucket for refinement --- .../solr/search/facet/FacetFieldProcessor.java | 6 +++--- .../search/facet/FacetRequestSortedMerger.java | 16 ++++++++-------- .../search/facet/TestJsonFacetRefinement.java | 11 +++++------ 3 files changed, 16 insertions(+), 17 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java index 97d8607aaf4..a29e78d9e6c 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java @@ -530,13 +530,13 @@ abstract class FacetFieldProcessor extends FacetProcessor { protected SimpleOrderedMap refineFacets() throws IOException { List leaves = asList(fcontext.facetInfo.get("_l")); // We have not seen this bucket: do full faceting for this bucket, including all sub-facets List skip = asList(fcontext.facetInfo.get("_s")); // We have seen this bucket, so skip stats on it, and skip sub-facets except for the specified sub-facets that should calculate specified buckets. - List missing = asList(fcontext.facetInfo.get("_m")); // We have not seen this bucket, do full faceting for this bucket, and most sub-facets... but some sub-facets should only visit specified buckets. + List partial = asList(fcontext.facetInfo.get("_p")); // We have not seen this bucket, do full faceting for this bucket, and most sub-facets... but some sub-facets are partial and should only visit specified buckets. // For leaf refinements, we do full faceting for each leaf bucket. Any sub-facets of these buckets will be fully evaluated. Because of this, we should never // encounter leaf refinements that have sub-facets that return partial results. SimpleOrderedMap res = new SimpleOrderedMap<>(); - List bucketList = new ArrayList<>( leaves.size() + skip.size() + missing.size() ); + List bucketList = new ArrayList<>( leaves.size() + skip.size() + partial.size() ); res.add("buckets", bucketList); // TODO: an alternate implementations can fill all accs at once @@ -555,7 +555,7 @@ abstract class FacetFieldProcessor extends FacetProcessor { } // The only difference between skip and missing is the value of "skip" passed to refineBucket - for (List bucketAndFacetInfo : missing) { + for (List bucketAndFacetInfo : partial) { assert bucketAndFacetInfo.size() == 2; Object bucketVal = bucketAndFacetInfo.get(0); Map facetInfo = (Map) bucketAndFacetInfo.get(1); diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java index f55fc0fc180..e05064c4148 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java @@ -161,7 +161,7 @@ abstract class FacetRequestSortedMerger leafBuckets = null; // "_l" missing buckets specified by bucket value only (no need to specify anything further) - ArrayList missingBuckets = null; // "_m" missing buckets that need to specify values for partial facets.. each entry is [bucketval, subs] + ArrayList partialBuckets = null; // "_p" missing buckets that have a partial sub-facet that need to specify those bucket values... each entry is [bucketval, subs] ArrayList skipBuckets = null; // "_s" present buckets that we need to recurse into because children facets have refinement requirements. each entry is [bucketval, subs] for (FacetBucket bucket : bucketList) { @@ -208,12 +208,12 @@ abstract class FacetRequestSortedMerger(); - missingBuckets.add( Arrays.asList(bucket.bucketValue, bucketRefinement) ); + if (partialBuckets==null) partialBuckets = new ArrayList<>(); + partialBuckets.add( Arrays.asList(bucket.bucketValue, bucketRefinement) ); } } - // if we didn't add to "_m" (missing), then we should add to "_l" (leaf missing) + // if we didn't add to "_p" (missing with partial sub-facets), then we should add to "_l" (missing leaf) if (bucketRefinement == null) { if (leafBuckets == null) leafBuckets = new ArrayList<>(); leafBuckets.add(bucket.bucketValue); @@ -231,12 +231,12 @@ abstract class FacetRequestSortedMerger(3); if (leafBuckets != null) refinement.put("_l",leafBuckets); - if (missingBuckets != null) refinement.put("_m", missingBuckets); + if (partialBuckets != null) refinement.put("_p", partialBuckets); if (skipBuckets != null) refinement.put("_s", skipBuckets); } diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java index 5c863471d21..b4b022066ea 100644 --- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java +++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java @@ -36,7 +36,6 @@ import org.noggit.ObjectBuilder; public class TestJsonFacetRefinement extends SolrTestCaseHS { private static SolrInstances servers; // for distributed testing - private static int origTableSize; @BeforeClass public static void beforeTests() throws Exception { @@ -209,13 +208,13 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "}" ); - // for testing missing _m, we need a partial facet within a partial facet + // for testing partial _p, we need a partial facet within a partial facet doTestRefine("{top:{type:terms, field:Afield, refine:true, limit:1, facet:{x : {type:terms, field:X, limit:1, refine:true} } } }", "{top: {buckets:[{val:'A', count:2, x:{buckets:[{val:x1, count:5},{val:x2, count:3}]} } ] } }", "{top: {buckets:[{val:'B', count:1, x:{buckets:[{val:x2, count:4},{val:x3, count:2}]} } ] } }", null, "=={top: {" + - "_m:[ ['A' , {x:{_l:[x1]}} ] ]" + + "_p:[ ['A' , {x:{_l:[x1]}} ] ]" + " } " + "}" ); @@ -329,7 +328,7 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "}" ); - // test that sibling facets and stats are included for _m buckets, but skipped for _s buckets + // test that sibling facets and stats are included for _p buckets, but skipped for _s buckets client.testJQ(params(p, "q", "*:*", "json.facet", "{" + " ab :{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true, facet:{ xy:{type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true}, qq:{query:'*:*'},ww:'sum(${num_d})' }}" + @@ -339,9 +338,9 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "}" ) , "facets=={ count:8" + - ", ab:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _m buckets + ", ab:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _p buckets ", allf:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }] }" + // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets - ", ab2:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _m buckets + ", ab2:{ buckets:[ {val:A, count:4, xy:{buckets:[ {val:X,count:3}]} ,qq:{count:4}, ww:4.0 }] }" + // make sure qq and ww are included for _p buckets ", allf2:{ buckets:[ {count:8, val:all, cat:{buckets:[{val:A,count:4}]} ,qq:{count:8}, ww:2.0 }] }" + // make sure qq and ww are excluded (not calculated again in another phase) for _s buckets "}" ); From 144091ad2957d59f83d59c7fcb1afeda65b0f914 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Wed, 29 Mar 2017 08:09:40 +0700 Subject: [PATCH 081/563] SOLR-10079: TestInPlaceUpdates(Distrib|Standalone) failures --- .../org/apache/solr/util/TestInjection.java | 2 +- .../update/TestInPlaceUpdatesDistrib.java | 22 ++++++++----------- 2 files changed, 10 insertions(+), 14 deletions(-) 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 12992731ac7..6b7b1f86bd0 100644 --- a/solr/core/src/java/org/apache/solr/util/TestInjection.java +++ b/solr/core/src/java/org/apache/solr/util/TestInjection.java @@ -369,7 +369,7 @@ public class TestInjection { Pair pair = parseValue(waitForReplicasInSync); boolean enabled = pair.first(); if (!enabled) return true; - long t = System.currentTimeMillis() - 100; + long t = System.currentTimeMillis() - 200; try { for (int i = 0; i < pair.second(); i++) { if (core.isClosed()) return true; 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 cbd7b0260c5..f9f377cd9e3 100644 --- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java +++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java @@ -201,7 +201,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { assertEquals(2, NONLEADERS.size()); } - final int NUM_RETRIES = 100, WAIT_TIME = 10; + final int NUM_RETRIES = 100, WAIT_TIME = 50; // The following should work: full update to doc 0, in-place update for doc 0, delete doc 0 private void reorderedDBQsSimpleTest() throws Exception { @@ -266,8 +266,6 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { } log.info("reorderedDBQsSimpleTest: This test passed fine..."); - clearIndex(); - commit(); } private void reorderedDBQIndividualReplicaTest() throws Exception { @@ -324,7 +322,8 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { private void docValuesUpdateTest() throws Exception { // number of docs we're testing (0 <= id), index may contain additional random docs (id < 0) - final int numDocs = atLeast(100); + int numDocs = atLeast(100); + if (onlyLeaderIndexes) numDocs = TestUtil.nextInt(random(), 10, 50); log.info("Trying num docs = " + numDocs); final List ids = new ArrayList(numDocs); for (int id = 0; id < numDocs; id++) { @@ -667,8 +666,6 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { } log.info("outOfOrderUpdatesIndividualReplicaTest: This test passed fine..."); - clearIndex(); - commit(); } // The following should work: full update to doc 0, in-place update for doc 0, delete doc 0 @@ -733,8 +730,6 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { } log.info("reorderedDeletesTest: This test passed fine..."); - clearIndex(); - commit(); } /* Test for a situation when a document requiring in-place update cannot be "resurrected" @@ -972,10 +967,11 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { assertEquals("Field: " + fieldName, expected, rsp.getField()); } - private static class AsyncUpdateWithRandomCommit implements Callable { + private class AsyncUpdateWithRandomCommit implements Callable { UpdateRequest update; SolrClient solrClient; final Random rnd; + int commitBound = onlyLeaderIndexes ? 50 : 3; public AsyncUpdateWithRandomCommit (UpdateRequest update, SolrClient solrClient, long seed) { this.update = update; @@ -986,7 +982,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { @Override public UpdateResponse call() throws Exception { UpdateResponse resp = update.process(solrClient); //solrClient.request(update); - if (rnd.nextInt(3) == 0) + if (rnd.nextInt(commitBound) == 0) solrClient.commit(); return resp; } @@ -1113,9 +1109,9 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { * @return the versions of each of the specials document returned when indexing it */ protected List buildRandomIndex(Float initFloat, List specialIds) throws Exception { - + int id = -1; // used for non special docs - final int numPreDocs = rarely() ? TestUtil.nextInt(random(),0,9) : atLeast(10); + final int numPreDocs = rarely() || onlyLeaderIndexes ? TestUtil.nextInt(random(),0,9) : atLeast(10); for (int i = 1; i <= numPreDocs; i++) { addDocAndGetVersion("id", id, "title_s", "title" + id, "id_i", id); id--; @@ -1128,7 +1124,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase { versions.add(addDocAndGetVersion("id", special, "title_s", "title" + special, "id_i", special, "inplace_updatable_float", initFloat)); } - final int numPostDocs = rarely() ? TestUtil.nextInt(random(),0,9) : atLeast(10); + final int numPostDocs = rarely() || onlyLeaderIndexes ? TestUtil.nextInt(random(),0,2) : atLeast(10); for (int i = 1; i <= numPostDocs; i++) { addDocAndGetVersion("id", id, "title_s", "title" + id, "id_i", id); id--; From deddc9b5c8d8c2859469583fa8b956be48efff82 Mon Sep 17 00:00:00 2001 From: Shai Erera Date: Thu, 23 Mar 2017 08:28:05 +0200 Subject: [PATCH 082/563] SOLR-10349: Add totalTermFreq support to TermsComponent TermsComponent only returns docFreq information per requested term. This commit adds a terms.ttf parameter, which if set to true, will return both docFreq and totalTermFreq statistics for each requested term. --- solr/CHANGES.txt | 6 +- .../handler/component/TermsComponent.java | 66 +++++++++++-------- .../DistributedTermsComponentTest.java | 3 +- .../handler/component/TermsComponentTest.java | 38 +++++++++++ .../client/solrj/response/QueryResponse.java | 6 +- .../client/solrj/response/TermsResponse.java | 37 +++++++++-- .../solr/common/params/TermsParams.java | 12 ++-- 7 files changed, 126 insertions(+), 42 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 9d14e59978c..4e63926478d 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -115,17 +115,19 @@ Detailed Change List New Features ---------------------- -* SOLR-9992: Add support for grouping with PointFIelds. (Cao Manh Dat) +* SOLR-9992: Add support for grouping with PointFIelds. (Cao Manh Dat) * SOLR-10046: Add UninvertDocValuesMergePolicyFactory class. (Keith Laban, Christine Poerschke) -* SOLR-9994: Add support for CollapseQParser with PointFields. (Varun Thacker, Cao Manh Dat) +* SOLR-9994: Add support for CollapseQParser with PointFields. (Varun Thacker, Cao Manh Dat) * SOLR-10076: Hide keystore and truststore passwords from /admin/info/* outputs. (Mano Kovacs via Mark Miller) * SOLR-6736: Adding support for uploading zipped configsets using ConfigSets API (Varun Rajput, Ishan Chattopadhyaya, Noble Paul, Anshum Gupta, Gregory Chanan) +* SOLR-10349: Add totalTermFreq support to TermsComponent. (Shai Erera) + Optimizations ---------------------- diff --git a/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java b/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java index e00120c65e1..b05939e511c 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/TermsComponent.java @@ -108,8 +108,9 @@ public class TermsComponent extends SearchComponent { } String termList = params.get(TermsParams.TERMS_LIST); - if(termList != null) { - fetchTerms(rb.req.getSearcher(), fields, termList, termsResult); + if (termList != null) { + boolean includeTotalTermFreq = params.getBool(TermsParams.TERMS_TTF, false); + fetchTerms(rb.req.getSearcher(), fields, termList, includeTotalTermFreq, termsResult); return; } @@ -303,7 +304,7 @@ public class TermsComponent extends SearchComponent { if (th != null) { for (ShardResponse srsp : sreq.responses) { @SuppressWarnings("unchecked") - NamedList> terms = (NamedList>) srsp.getSolrResponse().getResponse().get("terms"); + NamedList> terms = (NamedList>) srsp.getSolrResponse().getResponse().get("terms"); th.parse(terms); @@ -376,7 +377,7 @@ public class TermsComponent extends SearchComponent { } } - public void parse(NamedList> terms) { + public void parse(NamedList> terms) { // exit if there is no terms if (terms == null) { return; @@ -400,6 +401,7 @@ public class TermsComponent extends SearchComponent { if (termmap.containsKey(term)) { TermsResponse.Term oldtc = termmap.get(term); oldtc.addFrequency(tc.getFrequency()); + oldtc.addTotalTermFreq(tc.getTotalTermFreq()); termmap.put(term, oldtc); } else { termmap.put(term, tc); @@ -442,7 +444,7 @@ public class TermsComponent extends SearchComponent { // loop though each field we want terms from for (String key : fieldmap.keySet()) { - NamedList fieldterms = new SimpleOrderedMap<>(); + NamedList fieldterms = new SimpleOrderedMap<>(); TermsResponse.Term[] data = null; if (sort) { data = getCountSorted(fieldmap.get(key)); @@ -450,11 +452,19 @@ public class TermsComponent extends SearchComponent { data = getLexSorted(fieldmap.get(key)); } + boolean includeTotalTermFreq = params.getBool(TermsParams.TERMS_TTF, false); // loop though each term until we hit limit int cnt = 0; for (TermsResponse.Term tc : data) { if (tc.getFrequency() >= freqmin && tc.getFrequency() <= freqmax) { - fieldterms.add(tc.getTerm(), num(tc.getFrequency())); + if (includeTotalTermFreq) { + NamedList termStats = new SimpleOrderedMap<>(); + termStats.add("docFreq", tc.getFrequency()); + termStats.add("totalTermFreq", tc.getTotalTermFreq()); + fieldterms.add(tc.getTerm(), termStats); + } else { + fieldterms.add(tc.getTerm(), num(tc.getFrequency())); + } cnt++; } @@ -508,10 +518,9 @@ public class TermsComponent extends SearchComponent { private void fetchTerms(SolrIndexSearcher indexSearcher, String[] fields, String termList, + boolean includeTotalTermFreq, NamedList result) throws IOException { - NamedList termsMap = new SimpleOrderedMap(); - List leaves = indexSearcher.getTopReaderContext().leaves(); String field = fields[0]; FieldType fieldType = indexSearcher.getSchema().getField(field).getType(); String[] splitTerms = termList.split(","); @@ -521,35 +530,43 @@ public class TermsComponent extends SearchComponent { } Term[] terms = new Term[splitTerms.length]; - TermContext[] termContexts = new TermContext[terms.length]; for(int i=0; i termStats = new SimpleOrderedMap<>(); + termStats.add("docFreq", (long) docFreq); + termStats.add("totalTermFreq", totalTermFreq); + termsMap.add(outTerm, termStats); + } } } result.add(field, termsMap); } - private void collectTermContext(IndexReader reader, - List leaves, TermContext[] contextArray, - Term[] queryTerms) throws IOException { + private void collectTermContext(IndexReaderContext topReaderContext, TermContext[] contextArray, Term[] queryTerms) + throws IOException { TermsEnum termsEnum = null; - for (LeafReaderContext context : leaves) { + for (LeafReaderContext context : topReaderContext.leaves()) { final Fields fields = context.reader().fields(); for (int i = 0; i < queryTerms.length; i++) { Term term = queryTerms[i]; - TermContext termContext = contextArray[i]; final Terms terms = fields.terms(term.field()); if (terms == null) { // field does not exist @@ -559,18 +576,15 @@ public class TermsComponent extends SearchComponent { assert termsEnum != null; if (termsEnum == TermsEnum.EMPTY) continue; + + TermContext termContext = contextArray[i]; if (termsEnum.seekExact(term.bytes())) { if (termContext == null) { - contextArray[i] = new TermContext(reader.getContext(), - termsEnum.termState(), context.ord, termsEnum.docFreq(), - termsEnum.totalTermFreq()); - } else { - termContext.register(termsEnum.termState(), context.ord, - termsEnum.docFreq(), termsEnum.totalTermFreq()); + termContext = new TermContext(topReaderContext); + contextArray[i] = termContext; } - + termContext.accumulateStatistics(termsEnum.docFreq(), termsEnum.totalTermFreq()); } - } } } diff --git a/solr/core/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java index 951cd88cb7b..9c90efb03e7 100644 --- a/solr/core/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java +++ b/solr/core/src/test/org/apache/solr/handler/component/DistributedTermsComponentTest.java @@ -52,7 +52,6 @@ public class DistributedTermsComponentTest extends BaseDistributedSearchTestCase query("qt", "/terms", "shards.qt", "/terms", "terms", "true", "terms.fl", "b_t", "terms.list", "snake, zebra, ant, bad"); query("qt", "/terms", "shards.qt", "/terms", "terms", "true", "terms.fl", "foo_i", "terms.list", "2, 3, 1"); query("qt", "/terms", "shards.qt", "/terms", "terms", "true", "terms.fl", "foo_i", "terms.stats", "true","terms.list", "2, 3, 1"); - - + query("qt", "/terms", "shards.qt", "/terms", "terms", "true", "terms.fl", "b_t", "terms.list", "snake, zebra", "terms.ttf", "true"); } } diff --git a/solr/core/src/test/org/apache/solr/handler/component/TermsComponentTest.java b/solr/core/src/test/org/apache/solr/handler/component/TermsComponentTest.java index 177881a41a4..7fb5e120260 100644 --- a/solr/core/src/test/org/apache/solr/handler/component/TermsComponentTest.java +++ b/solr/core/src/test/org/apache/solr/handler/component/TermsComponentTest.java @@ -18,6 +18,7 @@ package org.apache.solr.handler.component; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.TermsParams; +import org.apache.solr.request.SolrQueryRequest; import org.junit.BeforeClass; import org.junit.Test; @@ -313,4 +314,41 @@ public class TermsComponentTest extends SolrTestCaseJ4 { ,"count(//lst[@name='standardfilt']/*)=3" ); } + + @Test + public void testDocFreqAndTotalTermFreq() throws Exception { + SolrQueryRequest req = req( + "indent","true", + "qt", "/terms", + "terms", "true", + "terms.fl", "standardfilt", + "terms.ttf", "true", + "terms.list", "snake,spider,shark,ddddd"); + assertQ(req, + "count(//lst[@name='standardfilt']/*)=4", + "//lst[@name='standardfilt']/lst[@name='ddddd']/long[@name='docFreq'][.='4']", + "//lst[@name='standardfilt']/lst[@name='ddddd']/long[@name='totalTermFreq'][.='4']", + "//lst[@name='standardfilt']/lst[@name='shark']/long[@name='docFreq'][.='2']", + "//lst[@name='standardfilt']/lst[@name='shark']/long[@name='totalTermFreq'][.='2']", + "//lst[@name='standardfilt']/lst[@name='snake']/long[@name='docFreq'][.='3']", + "//lst[@name='standardfilt']/lst[@name='snake']/long[@name='totalTermFreq'][.='3']", + "//lst[@name='standardfilt']/lst[@name='spider']/long[@name='docFreq'][.='1']", + "//lst[@name='standardfilt']/lst[@name='spider']/long[@name='totalTermFreq'][.='1']"); + } + + @Test + public void testDocFreqAndTotalTermFreqForNonExistingTerm() throws Exception { + SolrQueryRequest req = req( + "indent","true", + "qt", "/terms", + "terms", "true", + "terms.fl", "standardfilt", + "terms.ttf", "true", + "terms.list", "boo,snake"); + assertQ(req, + "count(//lst[@name='standardfilt']/*)=1", + "//lst[@name='standardfilt']/lst[@name='snake']/long[@name='docFreq'][.='3']", + "//lst[@name='standardfilt']/lst[@name='snake']/long[@name='totalTermFreq'][.='3']"); + } + } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java index eb595aaad1c..4e7800505b1 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/QueryResponse.java @@ -50,7 +50,7 @@ public class QueryResponse extends SolrResponseBase private List> _clusterInfo = null; private Map> _suggestInfo = null; private NamedList _statsInfo = null; - private NamedList> _termsInfo = null; + private NamedList> _termsInfo = null; private NamedList _moreLikeThisInfo = null; private String _cursorMarkNext = null; @@ -166,7 +166,7 @@ public class QueryResponse extends SolrResponseBase extractStatsInfo( _statsInfo ); } else if ( "terms".equals( n ) ) { - _termsInfo = (NamedList>) res.getVal( i ); + _termsInfo = (NamedList>) res.getVal( i ); extractTermsInfo( _termsInfo ); } else if ( "moreLikeThis".equals( n ) ) { @@ -191,7 +191,7 @@ public class QueryResponse extends SolrResponseBase _suggestResponse = new SuggesterResponse(suggestInfo); } - private void extractTermsInfo(NamedList> termsInfo) { + private void extractTermsInfo(NamedList> termsInfo) { _termsResponse = new TermsResponse(termsInfo); } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/TermsResponse.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/TermsResponse.java index e3fb0612564..b4ee55333cc 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/response/TermsResponse.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/TermsResponse.java @@ -28,17 +28,26 @@ import java.util.Map; public class TermsResponse { private Map> termMap = new HashMap<>(); - public TermsResponse(NamedList> termsInfo) { + public TermsResponse(NamedList> termsInfo) { for (int i = 0; i < termsInfo.size(); i++) { String fieldName = termsInfo.getName(i); List itemList = new ArrayList<>(); - NamedList items = termsInfo.getVal(i); + NamedList items = termsInfo.getVal(i); for (int j = 0; j < items.size(); j++) { - Term t = new Term(items.getName(j), items.getVal(j).longValue()); + String term = items.getName(j); + Object val = items.getVal(j); + Term t; + if (val instanceof NamedList) { + @SuppressWarnings("unchecked") + NamedList termStats = (NamedList) val; + t = new Term(term, termStats.get("docFreq").longValue(), termStats.get("totalTermFreq").longValue()); + } else { + t = new Term(term, ((Number) val).longValue()); + } itemList.add(t); } - + termMap.put(fieldName, itemList); } } @@ -59,10 +68,16 @@ public class TermsResponse { public static class Term { private String term; private long frequency; + private long totalTermFreq; public Term(String term, long frequency) { + this(term, frequency, 0); + } + + public Term(String term, long frequency, long totalTermFreq) { this.term = term; this.frequency = frequency; + this.totalTermFreq = totalTermFreq; } public String getTerm() { @@ -80,9 +95,21 @@ public class TermsResponse { public void setFrequency(long frequency) { this.frequency = frequency; } - + public void addFrequency(long frequency) { this.frequency += frequency; } + + public long getTotalTermFreq() { + return totalTermFreq; + } + + public void setTotalTermFreq(long totalTermFreq) { + this.totalTermFreq = totalTermFreq; + } + + public void addTotalTermFreq(long totalTermFreq) { + this.totalTermFreq += totalTermFreq; + } } } diff --git a/solr/solrj/src/java/org/apache/solr/common/params/TermsParams.java b/solr/solrj/src/java/org/apache/solr/common/params/TermsParams.java index 497584667d2..9f96a80a7f6 100644 --- a/solr/solrj/src/java/org/apache/solr/common/params/TermsParams.java +++ b/solr/solrj/src/java/org/apache/solr/common/params/TermsParams.java @@ -42,16 +42,19 @@ public interface TermsParams { /** * Optional. The list of terms to be retrieved. - * */ public static final String TERMS_LIST = TERMS_PREFIX + "list"; /** - * Optional. The list of terms to be retrieved. - * + * Optional. If true, also returns index-level statistics, such as numDocs. */ public static final String TERMS_STATS = TERMS_PREFIX + "stats"; + /** + * Optional. If true, also returns terms' total term frequency. + */ + public static final String TERMS_TTF = TERMS_PREFIX + "ttf"; + /** * Optional. The lower bound term to start at. The TermEnum will start at the next term after this term in the dictionary. * @@ -107,10 +110,11 @@ public interface TermsParams { } } - /** + /** * Optional. The minimum value of docFreq to be returned. 1 by default */ public static final String TERMS_MINCOUNT = TERMS_PREFIX + "mincount"; + /** * Optional. The maximum value of docFreq to be returned. -1 by default means no boundary */ From 9b7c8d5606cdd88725c2779389bfb20e1d01cb49 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Wed, 29 Mar 2017 13:52:51 +0700 Subject: [PATCH 083/563] SOLR-9993: Add support for ExpandComponent with PointFields --- solr/CHANGES.txt | 2 + .../handler/component/ExpandComponent.java | 75 +++++++++++++------ .../component/TestExpandComponent.java | 2 - 3 files changed, 53 insertions(+), 26 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 4e63926478d..80c2aa0a2fd 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -128,6 +128,8 @@ New Features * SOLR-10349: Add totalTermFreq support to TermsComponent. (Shai Erera) +* SOLR-9993: Add support for ExpandComponent with PointFields. (Cao Manh Dat) + Optimizations ---------------------- diff --git a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java index f6ecd8d888f..656ac7113f0 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java +++ b/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java @@ -73,7 +73,12 @@ import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.core.PluginInfo; import org.apache.solr.core.SolrCore; import org.apache.solr.request.SolrQueryRequest; +import org.apache.solr.schema.DoublePointField; import org.apache.solr.schema.FieldType; +import org.apache.solr.schema.FloatPointField; +import org.apache.solr.schema.IntPointField; +import org.apache.solr.schema.LongPointField; +import org.apache.solr.schema.SchemaField; import org.apache.solr.schema.StrField; import org.apache.solr.schema.TrieDoubleField; import org.apache.solr.schema.TrieFloatField; @@ -209,7 +214,8 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia SolrIndexSearcher searcher = req.getSearcher(); LeafReader reader = searcher.getSlowAtomicReader(); - FieldType fieldType = searcher.getSchema().getField(field).getType(); + SchemaField schemaField = searcher.getSchema().getField(field); + FieldType fieldType = schemaField.getType(); SortedDocValues values = null; long nullValue = 0; @@ -228,17 +234,18 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia //Get the nullValue for the numeric collapse field String defaultValue = searcher.getSchema().getField(field).getDefaultValue(); if(defaultValue != null) { - if(fieldType instanceof TrieIntField || fieldType instanceof TrieLongField) { + if(fieldType instanceof TrieIntField || fieldType instanceof TrieLongField || + fieldType instanceof IntPointField || fieldType instanceof LongPointField) { nullValue = Long.parseLong(defaultValue); - } else if(fieldType instanceof TrieFloatField){ + } else if(fieldType instanceof TrieFloatField || fieldType instanceof FloatPointField){ nullValue = Float.floatToIntBits(Float.parseFloat(defaultValue)); - } else if(fieldType instanceof TrieDoubleField){ + } else if(fieldType instanceof TrieDoubleField || fieldType instanceof DoublePointField){ nullValue = Double.doubleToLongBits(Double.parseDouble(defaultValue)); } } else { - if(fieldType instanceof TrieFloatField){ + if(fieldType instanceof TrieFloatField || fieldType instanceof FloatPointField){ nullValue = Float.floatToIntBits(0.0f); - } else if(fieldType instanceof TrieDoubleField){ + } else if(fieldType instanceof TrieDoubleField || fieldType instanceof DoublePointField){ nullValue = Double.doubleToLongBits(0.0f); } } @@ -369,7 +376,11 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia } if(count > 0 && count < 200) { - groupQuery = getGroupQuery(field, fieldType, count, groupSet); + if (fieldType.isPointField()) { + groupQuery = getPointGroupQuery(schemaField, count, groupSet); + } else { + groupQuery = getGroupQuery(field, fieldType, count, groupSet); + } } } @@ -442,13 +453,7 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia String group = charsRef.toString(); outMap.add(group, slice); } else { - if(fieldType instanceof TrieIntField || fieldType instanceof TrieLongField ) { - outMap.add(Long.toString(groupValue), slice); - } else if(fieldType instanceof TrieFloatField) { - outMap.add(Float.toString(Float.intBitsToFloat((int) groupValue)), slice); - } else if(fieldType instanceof TrieDoubleField) { - outMap.add(Double.toString(Double.longBitsToDouble(groupValue)), slice); - } + outMap.add(numericToString(fieldType, groupValue), slice); } } } @@ -703,18 +708,10 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia BytesRefBuilder term = new BytesRefBuilder(); Iterator it = groupSet.iterator(); int index = -1; - String stringVal = null; + while (it.hasNext()) { LongCursor cursor = it.next(); - if(ft instanceof TrieIntField || ft instanceof TrieLongField) { - stringVal = Long.toString(cursor.value); - } else { - if(ft instanceof TrieFloatField) { - stringVal = Float.toString(Float.intBitsToFloat((int)cursor.value)); - } else { - stringVal = Double.toString(Double.longBitsToDouble(cursor.value)); - } - } + String stringVal = numericToString(ft, cursor.value); ft.readableToIndexed(stringVal, term); bytesRefs[++index] = term.toBytesRef(); } @@ -722,6 +719,36 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia return new SolrConstantScoreQuery(new QueryWrapperFilter(new TermInSetQuery(fname, bytesRefs))); } + private Query getPointGroupQuery(SchemaField sf, + int size, + LongHashSet groupSet) { + + Iterator it = groupSet.iterator(); + List values = new ArrayList<>(size); + FieldType ft = sf.getType(); + while (it.hasNext()) { + LongCursor cursor = it.next(); + values.add(numericToString(ft, cursor.value)); + } + + return new SolrConstantScoreQuery(new QueryWrapperFilter(sf.getType().getSetQuery(null, sf, values))); + } + + private String numericToString(FieldType fieldType, long val) { + if (fieldType.getNumberType() != null) { + switch (fieldType.getNumberType()) { + case INTEGER: + case LONG: + return Long.toString(val); + case FLOAT: + return Float.toString(Float.intBitsToFloat((int)val)); + case DOUBLE: + return Double.toString(Double.longBitsToDouble(val)); + } + } + throw new IllegalArgumentException("FieldType must be INT,LONG,FLOAT,DOUBLE found " + fieldType); + } + private Query getGroupQuery(String fname, int size, IntObjectHashMap ordBytes) throws Exception { diff --git a/solr/core/src/test/org/apache/solr/handler/component/TestExpandComponent.java b/solr/core/src/test/org/apache/solr/handler/component/TestExpandComponent.java index 7baa5a94d7e..d1906d55060 100644 --- a/solr/core/src/test/org/apache/solr/handler/component/TestExpandComponent.java +++ b/solr/core/src/test/org/apache/solr/handler/component/TestExpandComponent.java @@ -21,14 +21,12 @@ import java.util.Collections; import java.util.List; import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.SolrTestCaseJ4.SuppressPointFields; import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.search.CollapsingQParserPlugin; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -@SuppressPointFields public class TestExpandComponent extends SolrTestCaseJ4 { @BeforeClass From a6f27d3e1e457f9bc3bd4b9afcacfc37e9a87e80 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jan=20H=C3=B8ydahl?= Date: Wed, 29 Mar 2017 10:51:34 +0200 Subject: [PATCH 084/563] SOLR-10147: Admin UI -> Cloud -> Graph: Impossible to see shard state --- solr/CHANGES.txt | 2 ++ solr/webapp/web/css/angular/cloud.css | 24 +++++++++++++++++++ .../web/js/angular/controllers/cloud.js | 19 ++++++++++----- solr/webapp/web/partials/cloud.html | 1 + 4 files changed, 40 insertions(+), 6 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 80c2aa0a2fd..3692bd50505 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -171,6 +171,8 @@ Other Changes * SOLR-10357: Enable edismax and standard query parsers to handle the option combination sow=false / autoGeneratePhraseQueries="true" by setting QueryBuilder.autoGenerateMultiTermSynonymsQuery. +* SOLR-10147: Admin UI -> Cloud -> Graph: Impossible to see shard state (Amrit Sarkar, janhoy) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/webapp/web/css/angular/cloud.css b/solr/webapp/web/css/angular/cloud.css index 4017c221683..80e53e8137c 100644 --- a/solr/webapp/web/css/angular/cloud.css +++ b/solr/webapp/web/css/angular/cloud.css @@ -331,11 +331,13 @@ limitations under the License. { color: #C43C35; stroke: #C43C35; + font-style: italic; } #content #graph-content .node.recovery_failed text { fill: #C43C35; + font-style: italic; } #content #cloud #legend .down, @@ -357,11 +359,13 @@ limitations under the License. { color: #d5dd00; stroke: #d5dd00; + font-style: italic; } #content #graph-content .node.recovering text { fill: #d5dd00; + font-style: italic; } #content #cloud #legend .active, @@ -394,6 +398,12 @@ limitations under the License. stroke: #c0c0c0; } +#content #cloud #legend .leader, +#content #graph-content .leader text +{ + font-weight: bold; +} + #content #graph-content .node.lvl-0 circle { stroke: #fff; @@ -410,3 +420,17 @@ limitations under the License. padding-top: 15px; padding-bottom: 15px; } + +#content #cloud #legend .shard-inactive, +#content #cloud #legend .shard-inactive li, +#content #cloud #legend .shard-inactive li text, +#content #graph-content .shard-inactive text +{ + text-decoration: line-through; +} +#content #cloud #legend .shard-inactive circle, +#content #graph-content .shard-inactive circle, +#content #graph-content .link.shard-inactive +{ + stroke: #e9e9e9; +} \ No newline at end of file diff --git a/solr/webapp/web/js/angular/controllers/cloud.js b/solr/webapp/web/js/angular/controllers/cloud.js index 2d0dae0860b..aeaf5d3d89a 100644 --- a/solr/webapp/web/js/angular/controllers/cloud.js +++ b/solr/webapp/web/js/angular/controllers/cloud.js @@ -139,6 +139,8 @@ var graphSubController = function ($scope, Zookeeper, isRadial) { for (var c in state) { var shards = []; for (var s in state[c].shards) { + var shard_status = state[c].shards[s].state; + shard_status = shard_status == 'inactive' ? 'shard-inactive' : shard_status; var nodes = []; for (var n in state[c].shards[s].replicas) { leaf_count++; @@ -160,17 +162,19 @@ var graphSubController = function ($scope, Zookeeper, isRadial) { $scope.helperData.port.push(uri_parts.port); $scope.helperData.pathname.push(uri_parts.pathname); - var status = replica.state; + var replica_status = replica.state; if (!live_nodes[replica.node_name]) { - status = 'gone'; + replica_status = 'gone'; + } else if(shard_status=='shard-inactive') { + replica_status += ' ' + shard_status; } var node = { name: uri, data: { type: 'node', - state: status, + state: replica_status, leader: 'true' === replica.leader, uri: uri_parts } @@ -179,9 +183,10 @@ var graphSubController = function ($scope, Zookeeper, isRadial) { } var shard = { - name: s, + name: shard_status == "shard-inactive" ? s + ' (inactive)' : s, data: { - type: 'shard' + type: 'shard', + state: shard_status }, children: nodes }; @@ -280,7 +285,9 @@ solrAdminApp.directive('graph', function(Constants) { } if (d.data && d.data.state) { - classes.push(d.data.state); + if(!(d.data.type=='shard' && d.data.state=='active')){ + classes.push(d.data.state); + } } return classes.join(' '); diff --git a/solr/webapp/web/partials/cloud.html b/solr/webapp/web/partials/cloud.html index 415e123e73d..ea1c6a0d670 100644 --- a/solr/webapp/web/partials/cloud.html +++ b/solr/webapp/web/partials/cloud.html @@ -59,6 +59,7 @@ limitations under the License.
  • Recovering
  • Down
  • Recovery Failed
  • +
  • Inactive
  • Gone
  • From 1ace1740d96859009dacc41bce1941f1fb31e497 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Wed, 29 Mar 2017 08:48:26 -0400 Subject: [PATCH 085/563] remove dead code --- lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java | 1 - 1 file changed, 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 5ab16c248d3..c5d9ddcf887 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -414,7 +414,6 @@ public class IndexSearcher { throw new IllegalArgumentException("after.doc exceeds the number of documents in the reader: after.doc=" + after.doc + " limit=" + limit); } - numHits = Math.min(numHits, limit); final int cappedNumHits = Math.min(numHits, limit); From 30f7914c3b8ed990fcc0812f10de21722e96469f Mon Sep 17 00:00:00 2001 From: Andrzej Bialecki Date: Wed, 29 Mar 2017 14:42:20 +0200 Subject: [PATCH 086/563] SOLR-10362 Be more specific when catching this exception. --- .../src/java/org/apache/solr/util/stats/MetricUtils.java | 8 ++++++-- .../test/org/apache/solr/util/stats/MetricUtilsTest.java | 8 ++++---- 2 files changed, 10 insertions(+), 6 deletions(-) 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 9809070de6a..491932d1a8d 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 @@ -215,8 +215,12 @@ public class MetricUtils { try { consumer.accept(n, convertGauge(gauge, compact)); } catch (InternalError ie) { - LOG.warn("Error converting gauge '" + n + "', possible JDK bug: SOLR-10362", ie); - consumer.accept(n, null); + if (n.startsWith("memory.") && ie.getMessage().contains("Memory Pool not found")) { + LOG.warn("Error converting gauge '" + n + "', possible JDK bug: SOLR-10362", ie); + consumer.accept(n, null); + } else { + throw ie; + } } } else if (metric instanceof Meter) { Meter meter = (Meter) metric; 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 fe9815711df..aa02de5cdf1 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 @@ -81,8 +81,8 @@ public class MetricUtilsTest extends SolrTestCaseJ4 { am.set("bar", 2); Gauge gauge = () -> "foobar"; registry.register("gauge", gauge); - Gauge error = () -> {throw new InternalError("expected error");}; - registry.register("expected.error", error); + Gauge error = () -> {throw new InternalError("Memory Pool not found error");}; + registry.register("memory.expected.error", error); MetricUtils.toMaps(registry, Collections.singletonList(MetricFilter.ALL), MetricFilter.ALL, false, false, false, (k, o) -> { Map v = (Map)o; @@ -108,7 +108,7 @@ public class MetricUtilsTest extends SolrTestCaseJ4 { update = (Map)values.get("bar"); assertEquals(2, update.get("value")); assertEquals(2, update.get("updateCount")); - } else if (k.startsWith("expected.error")) { + } else if (k.startsWith("memory.expected.error")) { assertNull(v); } }); @@ -147,7 +147,7 @@ public class MetricUtilsTest extends SolrTestCaseJ4 { update = (Map)values.get("bar"); assertEquals(2, update.get("value")); assertEquals(2, update.get("updateCount")); - } else if (k.startsWith("expected.error")) { + } else if (k.startsWith("memory.expected.error")) { assertNull(o); } else { Map v = (Map)o; From efdb04d06c9d37b543ab0469c65f3474c34d455a Mon Sep 17 00:00:00 2001 From: Ishan Chattopadhyaya Date: Wed, 29 Mar 2017 19:22:02 +0530 Subject: [PATCH 087/563] SOLR-10352: Fixing available entropy warning limit to 300 --- solr/bin/solr | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/solr/bin/solr b/solr/bin/solr index 27ec0542ab7..903309bc83e 100755 --- a/solr/bin/solr +++ b/solr/bin/solr @@ -1771,7 +1771,7 @@ function launch_solr() { 1>"$SOLR_LOGS_DIR/solr-$SOLR_PORT-console.log" 2>&1 & echo $! > "$SOLR_PID_DIR/solr-$SOLR_PORT.pid" # check if /proc/sys/kernel/random/entropy_avail exists then check output of cat /proc/sys/kernel/random/entropy_avail to see if less than 300 - if [[ -f /proc/sys/kernel/random/entropy_avail ]] && (( `cat /proc/sys/kernel/random/entropy_avail` < 30000)); then + if [[ -f /proc/sys/kernel/random/entropy_avail ]] && (( `cat /proc/sys/kernel/random/entropy_avail` < 300)); then echo "Warning: Available entropy is low. As a result, use of the UUIDField, SSL, or any other features that require" echo "RNG might not work properly. To check for the amount of available entropy, use 'cat /proc/sys/kernel/random/entropy_avail'." echo "" From edcdc3052ba95840593ace32d6d9a7a6e4ebe7ea Mon Sep 17 00:00:00 2001 From: Erick Erickson Date: Wed, 29 Mar 2017 21:13:40 -0700 Subject: [PATCH 088/563] SOLR-10387: zkTransfer normalizes destination path incorrectly if source is a windows directory --- solr/CHANGES.txt | 3 ++ .../apache/solr/cloud/SolrCLIZkUtilsTest.java | 13 +++--- .../solr/common/cloud/ZkMaintenanceUtils.java | 41 ++++++++++--------- 3 files changed, 32 insertions(+), 25 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 3692bd50505..840de480058 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -148,6 +148,9 @@ Bug Fixes * SOLR-10369: bin\solr.cmd delete and healthcheck now works again; fixed continuation chars ^ (Luis Goes via janhoy) +* SOLR-10387: zkTransfer normalizes destination path incorrectly if source is a windows directory + (gopikannan venugopalsamy, Erick Erickson) + Other Changes ---------------------- diff --git a/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java b/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java index 35ba1d4584c..776075e7b61 100644 --- a/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/SolrCLIZkUtilsTest.java @@ -18,6 +18,7 @@ package org.apache.solr.cloud; import java.io.ByteArrayOutputStream; +import java.io.File; import java.io.IOException; import java.io.PrintStream; import java.nio.charset.Charset; @@ -229,9 +230,9 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase { // NOTE: really can't test copying to '.' because the test framework doesn't allow altering the source tree // and at least IntelliJ's CWD is in the source tree. - // copy to local ending in '/' + // copy to local ending in separator //src and cp3 and cp4 are valid - String localSlash = tmp.normalize() + "/cpToLocal/"; + String localSlash = tmp.normalize() + File.separator +"cpToLocal" + File.separator; args = new String[]{ "-src", "zk:/cp3/schema.xml", "-dst", localSlash, @@ -246,7 +247,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase { // copy to ZK ending in '/'. //src and cp3 are valid args = new String[]{ - "-src", "file:" + srcPathCheck.normalize().toAbsolutePath().toString() + "/solrconfig.xml", + "-src", "file:" + srcPathCheck.normalize().toAbsolutePath().toString() + File.separator + "solrconfig.xml", "-dst", "zk:/powerup/", "-recurse", "false", "-zkHost", zkAddr, @@ -259,7 +260,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase { // copy individual file up //src and cp3 are valid args = new String[]{ - "-src", "file:" + srcPathCheck.normalize().toAbsolutePath().toString() + "/solrconfig.xml", + "-src", "file:" + srcPathCheck.normalize().toAbsolutePath().toString() + File.separator + "solrconfig.xml", "-dst", "zk:/copyUpFile.xml", "-recurse", "false", "-zkHost", zkAddr, @@ -272,7 +273,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase { // copy individual file down //src and cp3 are valid - String localNamed = tmp.normalize().toString() + "/localnamed/renamed.txt"; + String localNamed = tmp.normalize().toString() + File.separator + "localnamed" + File.separator + "renamed.txt"; args = new String[]{ "-src", "zk:/cp4/solrconfig.xml", "-dst", "file:" + localNamed, @@ -404,7 +405,7 @@ public class SolrCLIZkUtilsTest extends SolrCloudTestCase { // Files are in mv2 // Now fail if we specify "file:". Everything should still be in /mv2 args = new String[]{ - "-src", "file:/mv2", + "-src", "file:" + File.separator + "mv2", "-dst", "/mv3", "-zkHost", zkAddr, }; diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java index f569ae3bfd4..dfdc8cf2fd3 100644 --- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java +++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkMaintenanceUtils.java @@ -119,33 +119,36 @@ public class ZkMaintenanceUtils { throw new SolrServerException("Local path " + Paths.get(src).toAbsolutePath() + " is a directory and recurse is false"); } } - if (srcIsZk == false && dstIsZk == false) { - throw new SolrServerException("At least one of the source and dest parameters must be prefixed with 'zk:' "); - } + if (dstIsZk && dst.length() == 0) { dst = "/"; // for consistency, one can copy from zk: and send to zk:/ } - dst = normalizeDest(src, dst); + dst = normalizeDest(src, dst, srcIsZk, dstIsZk); + // ZK -> ZK copy. if (srcIsZk && dstIsZk) { traverseZkTree(zkClient, src, VISIT_ORDER.VISIT_PRE, new ZkCopier(zkClient, src, dst)); return; } + + //local -> ZK copy if (dstIsZk) { uploadToZK(zkClient, Paths.get(src), dst, null); return; } - // Copying individual files from ZK requires special handling since downloadFromZK assumes it's a directory. + // Copying individual files from ZK requires special handling since downloadFromZK assumes the node has children. // This is kind of a weak test for the notion of "directory" on Zookeeper. + // ZK -> local copy where ZK is a parent node if (zkClient.getChildren(src, null, true).size() > 0) { downloadFromZK(zkClient, src, Paths.get(dst)); return; } + // Single file ZK -> local copy where ZK is a leaf node if (Files.isDirectory(Paths.get(dst))) { - if (dst.endsWith("/") == false) dst += "/"; - dst = normalizeDest(src, dst); + if (dst.endsWith(File.separator) == false) dst += File.separator; + dst = normalizeDest(src, dst, srcIsZk, dstIsZk); } byte[] data = zkClient.getData(src, null, null, true); Path filename = Paths.get(dst); @@ -154,31 +157,32 @@ public class ZkMaintenanceUtils { Files.write(filename, data); } - - private static String normalizeDest(String srcName, String dstName) { + // If the dest ends with a separator, it's a directory or non-leaf znode, so return the + // last element of the src to appended to the dstName. + private static String normalizeDest(String srcName, String dstName, boolean srcIsZk, boolean dstIsZk) { // Special handling for "." if (dstName.equals(".")) { return Paths.get(".").normalize().toAbsolutePath().toString(); } - // Pull the last element of the src path and add it to the dst if the src does NOT end in a slash - // If the source ends in a slash, do not append the last segment to the dest - - if (dstName.endsWith("/")) { // Dest is a directory. - int pos = srcName.lastIndexOf("/"); + String dstSeparator = (dstIsZk) ? "/" : File.separator; + String srcSeparator = (srcIsZk) ? "/" : File.separator; + + if (dstName.endsWith(dstSeparator)) { // Dest is a directory or non-leaf znode, append last element of the src path. + int pos = srcName.lastIndexOf(srcSeparator); if (pos < 0) { dstName += srcName; } else { dstName += srcName.substring(pos + 1); } } - + log.info("copying from '{}' to '{}'", srcName, dstName); return dstName; } public static void moveZnode(SolrZkClient zkClient, String src, String dst) throws SolrServerException, KeeperException, InterruptedException { - String destName = normalizeDest(src, dst); + String destName = normalizeDest(src, dst, true, true); // Special handling if the source has no children, i.e. copying just a single file. if (zkClient.getChildren(src, null, true).size() == 0) { @@ -384,12 +388,11 @@ public class ZkMaintenanceUtils { } } - // Take into account Windows file separaters when making a Znode's name. + // Take into account Windows file separators when making a Znode's name. public static String createZkNodeName(String zkRoot, Path root, Path file) { String relativePath = root.relativize(file).toString(); // Windows shenanigans - String separator = root.getFileSystem().getSeparator(); - if ("\\".equals(separator)) + if ("\\".equals(File.separator)) relativePath = relativePath.replaceAll("\\\\", "/"); // It's possible that the relative path and file are the same, in which case // adding the bare slash is A Bad Idea unless it's a non-leaf data node From edafcbad14482f3cd2f072fdca0c89600e72885d Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 30 Mar 2017 15:11:52 +0200 Subject: [PATCH 089/563] LUCENE-7755: Join queries should not reference IndexReaders. --- lucene/CHANGES.txt | 7 +++++++ .../apache/lucene/index/IndexReaderContext.java | 10 +++++++++- .../lucene/search/join/GlobalOrdinalsQuery.java | 16 ++++++++++------ .../join/GlobalOrdinalsWithScoreQuery.java | 16 ++++++++++------ .../org/apache/lucene/search/join/JoinUtil.java | 8 +++----- 5 files changed, 39 insertions(+), 18 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 92f01a9e80f..da643ff5b82 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -93,6 +93,13 @@ Other * LUCENE-7743: Never call new String(String). (Daniel Jelinski via Adrien Grand) +======================= Lucene 6.5.1 ======================= + +Bug Fixes + +* LUCENE-7755: Fixed join queries to not reference IndexReaders, as it could + cause leaks if they are cached. (Adrien Grand) + ======================= Lucene 6.5.0 ======================= API Changes diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexReaderContext.java b/lucene/core/src/java/org/apache/lucene/index/IndexReaderContext.java index dada3ff7263..bca7a140c18 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexReaderContext.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexReaderContext.java @@ -46,7 +46,15 @@ public abstract class IndexReaderContext { this.ordInParent = ordInParent; this.isTopLevel = parent==null; } - + + /** Expert: Return an {@link Object} that uniquely identifies this context. + * The returned object does neither reference this {@link IndexReaderContext} + * nor the wrapped {@link IndexReader}. + * @lucene.experimental */ + public Object id() { + return identity; + } + /** Returns the {@link IndexReader}, this context represents. */ public abstract IndexReader reader(); diff --git a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java index 93edcc073d8..5aaca1af153 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsQuery.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.Set; import org.apache.lucene.index.DocValues; -import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.MultiDocValues; import org.apache.lucene.index.SortedDocValues; @@ -48,19 +48,23 @@ final class GlobalOrdinalsQuery extends Query { // just for hashcode and equals: private final Query fromQuery; - private final IndexReader indexReader; + // id of the context rather than the context itself in order not to hold references to index readers + private final Object indexReaderContextId; - GlobalOrdinalsQuery(LongBitSet foundOrds, String joinField, MultiDocValues.OrdinalMap globalOrds, Query toQuery, Query fromQuery, IndexReader indexReader) { + GlobalOrdinalsQuery(LongBitSet foundOrds, String joinField, MultiDocValues.OrdinalMap globalOrds, Query toQuery, Query fromQuery, IndexReaderContext context) { this.foundOrds = foundOrds; this.joinField = joinField; this.globalOrds = globalOrds; this.toQuery = toQuery; this.fromQuery = fromQuery; - this.indexReader = indexReader; + this.indexReaderContextId = context.id(); } @Override public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + if (searcher.getTopReaderContext().id() != indexReaderContextId) { + throw new IllegalStateException("Creating the weight against a different index reader than this query has been built for."); + } return new W(this, toQuery.createWeight(searcher, false, 1f), boost); } @@ -74,7 +78,7 @@ final class GlobalOrdinalsQuery extends Query { return fromQuery.equals(other.fromQuery) && joinField.equals(other.joinField) && toQuery.equals(other.toQuery) && - indexReader.equals(other.indexReader); + indexReaderContextId.equals(other.indexReaderContextId); } @Override @@ -83,7 +87,7 @@ final class GlobalOrdinalsQuery extends Query { result = 31 * result + joinField.hashCode(); result = 31 * result + toQuery.hashCode(); result = 31 * result + fromQuery.hashCode(); - result = 31 * result + indexReader.hashCode(); + result = 31 * result + indexReaderContextId.hashCode(); return result; } diff --git a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java index 0aedf353f85..5e614ea41f3 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/GlobalOrdinalsWithScoreQuery.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.util.Set; import org.apache.lucene.index.DocValues; -import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.MultiDocValues; import org.apache.lucene.index.SortedDocValues; @@ -48,9 +48,10 @@ final class GlobalOrdinalsWithScoreQuery extends Query { private final Query fromQuery; private final int min; private final int max; - private final IndexReader indexReader; + // id of the context rather than the context itself in order not to hold references to index readers + private final Object indexReaderContextId; - GlobalOrdinalsWithScoreQuery(GlobalOrdinalsWithScoreCollector collector, String joinField, MultiDocValues.OrdinalMap globalOrds, Query toQuery, Query fromQuery, int min, int max, IndexReader indexReader) { + GlobalOrdinalsWithScoreQuery(GlobalOrdinalsWithScoreCollector collector, String joinField, MultiDocValues.OrdinalMap globalOrds, Query toQuery, Query fromQuery, int min, int max, IndexReaderContext context) { this.collector = collector; this.joinField = joinField; this.globalOrds = globalOrds; @@ -58,11 +59,14 @@ final class GlobalOrdinalsWithScoreQuery extends Query { this.fromQuery = fromQuery; this.min = min; this.max = max; - this.indexReader = indexReader; + this.indexReaderContextId = context.id(); } @Override public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + if (searcher.getTopReaderContext().id() != indexReaderContextId) { + throw new IllegalStateException("Creating the weight against a different index reader than this query has been built for."); + } return new W(this, toQuery.createWeight(searcher, false, 1f)); } @@ -78,7 +82,7 @@ final class GlobalOrdinalsWithScoreQuery extends Query { joinField.equals(other.joinField) && fromQuery.equals(other.fromQuery) && toQuery.equals(other.toQuery) && - indexReader.equals(other.indexReader); + indexReaderContextId.equals(other.indexReaderContextId); } @Override @@ -89,7 +93,7 @@ final class GlobalOrdinalsWithScoreQuery extends Query { result = 31 * result + fromQuery.hashCode(); result = 31 * result + min; result = 31 * result + max; - result = 31 * result + indexReader.hashCode(); + result = 31 * result + indexReaderContextId.hashCode(); return result; } diff --git a/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java b/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java index bfc1f9bf3ec..537b2244aee 100644 --- a/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java +++ b/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java @@ -32,7 +32,6 @@ import org.apache.lucene.document.LongPoint; import org.apache.lucene.index.BinaryDocValues; import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValuesType; -import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.MultiDocValues; @@ -467,8 +466,7 @@ public final class JoinUtil { MultiDocValues.OrdinalMap ordinalMap, int min, int max) throws IOException { - IndexReader indexReader = searcher.getIndexReader(); - int numSegments = indexReader.leaves().size(); + int numSegments = searcher.getIndexReader().leaves().size(); final long valueCount; if (numSegments == 0) { return new MatchNoDocsQuery("JoinUtil.createJoinQuery with no segments"); @@ -509,7 +507,7 @@ public final class JoinUtil { if (min <= 0 && max == Integer.MAX_VALUE) { GlobalOrdinalsCollector globalOrdinalsCollector = new GlobalOrdinalsCollector(joinField, ordinalMap, valueCount); searcher.search(rewrittenFromQuery, globalOrdinalsCollector); - return new GlobalOrdinalsQuery(globalOrdinalsCollector.getCollectorOrdinals(), joinField, ordinalMap, rewrittenToQuery, rewrittenFromQuery, indexReader); + return new GlobalOrdinalsQuery(globalOrdinalsCollector.getCollectorOrdinals(), joinField, ordinalMap, rewrittenToQuery, rewrittenFromQuery, searcher.getTopReaderContext()); } else { globalOrdinalsWithScoreCollector = new GlobalOrdinalsWithScoreCollector.NoScore(joinField, ordinalMap, valueCount, min, max); break; @@ -518,7 +516,7 @@ public final class JoinUtil { throw new IllegalArgumentException(String.format(Locale.ROOT, "Score mode %s isn't supported.", scoreMode)); } searcher.search(rewrittenFromQuery, globalOrdinalsWithScoreCollector); - return new GlobalOrdinalsWithScoreQuery(globalOrdinalsWithScoreCollector, joinField, ordinalMap, rewrittenToQuery, rewrittenFromQuery, min, max, indexReader); + return new GlobalOrdinalsWithScoreQuery(globalOrdinalsWithScoreCollector, joinField, ordinalMap, rewrittenToQuery, rewrittenFromQuery, min, max, searcher.getTopReaderContext()); } } From 6c2155c02434bfae2ff5aa62c9ffe57318063626 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Thu, 30 Mar 2017 17:34:28 +0100 Subject: [PATCH 090/563] SOLR-10351: Add analyze Stream Evaluator to support streaming NLP --- .../apache/solr/handler/AnalyzeEvaluator.java | 111 +++++++++++++++ .../apache/solr/handler/StreamHandler.java | 1 + .../solrj/io/eval/BooleanEvaluator.java | 9 +- .../solrj/io/eval/ConditionalEvaluator.java | 6 + .../client/solrj/io/eval/NumberEvaluator.java | 6 + .../client/solrj/io/eval/SimpleEvaluator.java | 7 + .../client/solrj/io/eval/StreamEvaluator.java | 2 + .../io/stream/CartesianProductStream.java | 6 +- .../client/solrj/io/stream/SelectStream.java | 6 + .../solrj/io/stream/StreamExpressionTest.java | 133 +++++++++++++++++- 10 files changed, 278 insertions(+), 9 deletions(-) create mode 100644 solr/core/src/java/org/apache/solr/handler/AnalyzeEvaluator.java diff --git a/solr/core/src/java/org/apache/solr/handler/AnalyzeEvaluator.java b/solr/core/src/java/org/apache/solr/handler/AnalyzeEvaluator.java new file mode 100644 index 00000000000..207f4046590 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/handler/AnalyzeEvaluator.java @@ -0,0 +1,111 @@ +/* + * 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; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.StreamContext; +import org.apache.solr.client.solrj.io.stream.expr.Explanation; +import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.apache.solr.client.solrj.io.eval.*; + +import org.apache.solr.common.SolrException; +import org.apache.lucene.analysis.*; +import org.apache.solr.core.SolrCore; + +public class AnalyzeEvaluator extends SimpleEvaluator { + private static final long serialVersionUID = 1L; + + private String fieldName; + private String analyzerField; + private Analyzer analyzer; + + public AnalyzeEvaluator(String _fieldName, String _analyzerField) { + init(_fieldName, _analyzerField); + } + + public AnalyzeEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + String _fieldName = factory.getValueOperand(expression, 0); + String _analyzerField = factory.getValueOperand(expression, 1); + init(_fieldName, _analyzerField); + } + + public void setStreamContext(StreamContext context) { + Object solrCoreObj = context.get("solr-core"); + if (solrCoreObj == null || !(solrCoreObj instanceof SolrCore) ) { + throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "StreamContext must have SolrCore in solr-core key"); + } + SolrCore solrCore = (SolrCore) solrCoreObj; + + analyzer = solrCore.getLatestSchema().getFieldType(analyzerField).getIndexAnalyzer(); + } + + private void init(String fieldName, String analyzerField) { + this.fieldName = fieldName; + if(analyzerField == null) { + this.analyzerField = fieldName; + } else { + this.analyzerField = analyzerField; + } + } + + @Override + public Object evaluate(Tuple tuple) throws IOException { + String value = tuple.getString(fieldName); + if(value == null) { + return null; + } + + TokenStream tokenStream = analyzer.tokenStream(analyzerField, value); + CharTermAttribute termAtt = tokenStream.getAttribute(CharTermAttribute.class); + tokenStream.reset(); + List tokens = new ArrayList(); + while (tokenStream.incrementToken()) { + tokens.add(termAtt.toString()); + } + + tokenStream.end(); + tokenStream.close(); + + return tokens; + } + + @Override + public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException { + return new StreamExpressionValue(fieldName); + } + + @Override + public Explanation toExplanation(StreamFactory factory) throws IOException { + return new Explanation(nodeId.toString()) + .withExpressionType(ExpressionType.EVALUATOR) + .withImplementingClass(getClass().getName()) + .withExpression(toExpression(factory).toString()); + } + +} diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java index 5d10664debf..3ede7320f89 100644 --- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java @@ -209,6 +209,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, .withFunctionName("log", NaturalLogEvaluator.class) // Conditional Stream Evaluators .withFunctionName("if", IfThenElseEvaluator.class) + .withFunctionName("analyze", AnalyzeEvaluator.class) ; // This pulls all the overrides and additions from the config diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java index bf21f1d3a1f..f02f1fac65e 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java @@ -24,11 +24,13 @@ import java.util.ArrayList; import java.util.List; import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; public abstract class BooleanEvaluator extends ComplexEvaluator { protected static final long serialVersionUID = 1L; + protected StreamContext streamContext; public BooleanEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ super(expression, factory); @@ -45,7 +47,12 @@ public abstract class BooleanEvaluator extends ComplexEvaluator { return results; } - + + public void setStreamContext(StreamContext streamContext) { + this.streamContext = streamContext; + } + + public interface Checker { default boolean isNullAllowed(){ return false; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java index 499e2f856f0..61265441f8d 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java @@ -24,11 +24,13 @@ import java.util.ArrayList; import java.util.List; import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; public abstract class ConditionalEvaluator extends ComplexEvaluator { protected static final long serialVersionUID = 1L; + protected StreamContext streamContext; public ConditionalEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ super(expression, factory); @@ -42,6 +44,10 @@ public abstract class ConditionalEvaluator extends ComplexEvaluator { return results; } + + public void setStreamContext(StreamContext streamContext) { + this.streamContext = streamContext; + } public interface Checker { default boolean isNullAllowed(){ diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java index f4491fd45ff..283c7b196e8 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java @@ -26,11 +26,13 @@ import java.util.List; import java.util.Locale; import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; public abstract class NumberEvaluator extends ComplexEvaluator { protected static final long serialVersionUID = 1L; + protected StreamContext streamContext; public NumberEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ super(expression, factory); @@ -38,6 +40,10 @@ public abstract class NumberEvaluator extends ComplexEvaluator { // restrict result to a Number public abstract Number evaluate(Tuple tuple) throws IOException; + + public void setStreamContext(StreamContext context) { + this.streamContext = context; + } public List evaluateAll(final Tuple tuple) throws IOException { // evaluate each and confirm they are all either null or numeric diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SimpleEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SimpleEvaluator.java index 79d1799282e..5ee17150106 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SimpleEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SimpleEvaluator.java @@ -21,9 +21,16 @@ package org.apache.solr.client.solrj.io.eval; import java.util.UUID; +import org.apache.solr.client.solrj.io.stream.StreamContext; + public abstract class SimpleEvaluator implements StreamEvaluator { private static final long serialVersionUID = 1L; protected UUID nodeId = UUID.randomUUID(); + protected StreamContext streamContext; + + public void setStreamContext(StreamContext streamContext) { + this.streamContext = streamContext; + } } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/StreamEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/StreamEvaluator.java index 6bc4d50d570..1774c46e286 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/StreamEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/StreamEvaluator.java @@ -23,8 +23,10 @@ import java.io.IOException; import java.io.Serializable; import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.client.solrj.io.stream.expr.Expressible; public interface StreamEvaluator extends Expressible, Serializable { Object evaluate(final Tuple tuple) throws IOException; + void setStreamContext(StreamContext streamContext); } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CartesianProductStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CartesianProductStream.java index feb10c797a8..6514ae496cc 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CartesianProductStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/CartesianProductStream.java @@ -49,7 +49,7 @@ public class CartesianProductStream extends TupleStream implements Expressible { private List evaluators; private StreamComparator orderBy; - // Used to contain the sorted queue of generated tuples + // Used to contain the sorted queue of generated tuples private LinkedList generatedTuples; public CartesianProductStream(StreamExpression expression,StreamFactory factory) throws IOException { @@ -59,7 +59,6 @@ public class CartesianProductStream extends TupleStream implements Expressible { List streamExpressions = factory.getExpressionOperandsRepresentingTypes(expression, Expressible.class, TupleStream.class); List evaluateAsExpressions = factory.getOperandsOfType(expression, StreamExpressionValue.class); StreamExpressionNamedParameter orderByExpression = factory.getNamedOperand(expression, "productSort"); - // validate expression contains only what we want. if(expression.getParameters().size() != streamExpressions.size() + evaluateAsExpressions.size() + (null == orderByExpression ? 0 : 1)){ throw new IOException(String.format(Locale.ROOT,"Invalid %s expression %s - unknown operands found", functionName, expression)); @@ -259,6 +258,9 @@ public class CartesianProductStream extends TupleStream implements Expressible { public void setStreamContext(StreamContext context) { this.stream.setStreamContext(context); + for(NamedEvaluator evaluator : evaluators) { + evaluator.getEvaluator().setStreamContext(context); + } } public List children() { diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java index eed818280ff..c0cbc176efd 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/SelectStream.java @@ -22,6 +22,7 @@ import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Set; import org.apache.solr.client.solrj.io.Tuple; import org.apache.solr.client.solrj.io.comp.StreamComparator; @@ -213,6 +214,11 @@ public class SelectStream extends TupleStream implements Expressible { public void setStreamContext(StreamContext context) { this.stream.setStreamContext(context); + Set evaluators = selectedEvaluators.keySet(); + + for(StreamEvaluator evaluator : evaluators) { + evaluator.setStreamContext(context); + } } public List children() { diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java index 6c96025e640..18ddb931560 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java @@ -61,6 +61,7 @@ import org.apache.solr.cloud.AbstractDistribZkTestBase; import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.handler.AnalyzeEvaluator; import org.junit.Assume; import org.junit.Before; import org.junit.BeforeClass; @@ -379,7 +380,7 @@ public class StreamExpressionTest extends SolrCloudTestCase { stream = factory.constructStream("sort(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i asc\")"); tuples = getTuples(stream); assert(tuples.size() == 6); - assertOrder(tuples, 0,1,5,2,3,4); + assertOrder(tuples, 0, 1, 5, 2, 3, 4); // Basic test desc stream = factory.constructStream("sort(search(" + COLLECTIONORALIAS + ", q=*:*, fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc\"), by=\"a_i desc\")"); @@ -1908,7 +1909,7 @@ public class StreamExpressionTest extends SolrCloudTestCase { stream = new InnerJoinStream(expression, factory); tuples = getTuples(stream); assert(tuples.size() == 8); - assertOrder(tuples, 1,1,15,15,3,4,5,7); + assertOrder(tuples, 1, 1, 15, 15, 3, 4, 5, 7); // Basic desc expression = StreamExpressionParser.parse("innerJoin(" @@ -1922,9 +1923,9 @@ public class StreamExpressionTest extends SolrCloudTestCase { // Results in both searches, no join matches expression = StreamExpressionParser.parse("innerJoin(" - + "search(" + COLLECTIONORALIAS + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\")," - + "search(" + COLLECTIONORALIAS + ", q=\"side_s:right\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\", aliases=\"id=right.id, join1_i=right.join1_i, join2_s=right.join2_s, ident_s=right.ident_s\")," - + "on=\"ident_s=right.ident_s\")"); + + "search(" + COLLECTIONORALIAS + ", q=\"side_s:left\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\")," + + "search(" + COLLECTIONORALIAS + ", q=\"side_s:right\", fl=\"id,join1_i,join2_s,ident_s\", sort=\"ident_s asc\", aliases=\"id=right.id, join1_i=right.join1_i, join2_s=right.join2_s, ident_s=right.ident_s\")," + + "on=\"ident_s=right.ident_s\")"); stream = new InnerJoinStream(expression, factory); tuples = getTuples(stream); assert(tuples.size() == 0); @@ -1938,7 +1939,7 @@ public class StreamExpressionTest extends SolrCloudTestCase { tuples = getTuples(stream); assert(tuples.size() == 8); - assertOrder(tuples, 1,1,15,15,3,4,5,7); + assertOrder(tuples, 1, 1, 15, 15, 3, 4, 5, 7); } @@ -4347,6 +4348,126 @@ public class StreamExpressionTest extends SolrCloudTestCase { CollectionAdminRequest.deleteCollection("checkpointCollection").process(cluster.getSolrClient()); } + @Test + public void testAnalyzeEvaluator() throws Exception { + + UpdateRequest updateRequest = new UpdateRequest(); + updateRequest.add(id, "1", "test_t", "l b c d c"); + updateRequest.commit(cluster.getSolrClient(), COLLECTIONORALIAS); + + + SolrClientCache cache = new SolrClientCache(); + try { + + String expr = "cartesianProduct(search("+COLLECTIONORALIAS+", q=\"*:*\", fl=\"id, test_t\", sort=\"id desc\"), analyze(test_t, test_t) as test_t)"; + ModifiableSolrParams paramsLoc = new ModifiableSolrParams(); + paramsLoc.set("expr", expr); + paramsLoc.set("qt", "/stream"); + String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS; + + SolrStream solrStream = new SolrStream(url, paramsLoc); + + StreamContext context = new StreamContext(); + solrStream.setStreamContext(context); + List tuples = getTuples(solrStream); + assertTrue(tuples.size() == 5); + + Tuple t = tuples.get(0); + assertTrue(t.getString("test_t").equals("l")); + assertTrue(t.getString("id").equals("1")); + + t = tuples.get(1); + assertTrue(t.getString("test_t").equals("b")); + assertTrue(t.getString("id").equals("1")); + + + t = tuples.get(2); + assertTrue(t.getString("test_t").equals("c")); + assertTrue(t.getString("id").equals("1")); + + + t = tuples.get(3); + assertTrue(t.getString("test_t").equals("d")); + assertTrue(t.getString("id").equals("1")); + + t = tuples.get(4); + assertTrue(t.getString("test_t").equals("c")); + assertTrue(t.getString("id").equals("1")); + + + //Try with single param + expr = "cartesianProduct(search("+COLLECTIONORALIAS+", q=\"*:*\", fl=\"id, test_t\", sort=\"id desc\"), analyze(test_t) as test_t)"; + paramsLoc = new ModifiableSolrParams(); + paramsLoc.set("expr", expr); + paramsLoc.set("qt", "/stream"); + + solrStream = new SolrStream(url, paramsLoc); + + context = new StreamContext(); + solrStream.setStreamContext(context); + tuples = getTuples(solrStream); + assertTrue(tuples.size() == 5); + + t = tuples.get(0); + assertTrue(t.getString("test_t").equals("l")); + assertTrue(t.getString("id").equals("1")); + + t = tuples.get(1); + assertTrue(t.getString("test_t").equals("b")); + assertTrue(t.getString("id").equals("1")); + + + t = tuples.get(2); + assertTrue(t.getString("test_t").equals("c")); + assertTrue(t.getString("id").equals("1")); + + + t = tuples.get(3); + assertTrue(t.getString("test_t").equals("d")); + assertTrue(t.getString("id").equals("1")); + + t = tuples.get(4); + assertTrue(t.getString("test_t").equals("c")); + assertTrue(t.getString("id").equals("1")); + + + //Try with null in the test_t field + expr = "cartesianProduct(search("+COLLECTIONORALIAS+", q=\"*:*\", fl=\"id\", sort=\"id desc\"), analyze(test_t, test_t) as test_t)"; + paramsLoc = new ModifiableSolrParams(); + paramsLoc.set("expr", expr); + paramsLoc.set("qt", "/stream"); + + solrStream = new SolrStream(url, paramsLoc); + + context = new StreamContext(); + solrStream.setStreamContext(context); + tuples = getTuples(solrStream); + assertTrue(tuples.size() == 1); + + //Test annotating tuple + expr = "select(search("+COLLECTIONORALIAS+", q=\"*:*\", fl=\"id, test_t\", sort=\"id desc\"), analyze(test_t, test_t) as test1_t)"; + paramsLoc = new ModifiableSolrParams(); + paramsLoc.set("expr", expr); + paramsLoc.set("qt", "/stream"); + + solrStream = new SolrStream(url, paramsLoc); + + context = new StreamContext(); + solrStream.setStreamContext(context); + tuples = getTuples(solrStream); + assertTrue(tuples.size() == 1); + List l = (List)tuples.get(0).get("test1_t"); + assertTrue(l.get(0).equals("l")); + assertTrue(l.get(1).equals("b")); + assertTrue(l.get(2).equals("c")); + assertTrue(l.get(3).equals("d")); + assertTrue(l.get(4).equals("c")); + } finally { + cache.close(); + } + } + + @Test public void testExecutorStream() throws Exception { CollectionAdminRequest.createCollection("workQueue", "conf", 2, 1).process(cluster.getSolrClient()); From bdd0c7e32087f534de04657fb3ef1b3afa93cc68 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Thu, 30 Mar 2017 17:52:16 +0100 Subject: [PATCH 091/563] SOLR-10351: Fix pre-commit --- .../apache/solr/client/solrj/io/stream/StreamExpressionTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java index 18ddb931560..f153a1be05d 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java @@ -61,7 +61,6 @@ import org.apache.solr.cloud.AbstractDistribZkTestBase; import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; -import org.apache.solr.handler.AnalyzeEvaluator; import org.junit.Assume; import org.junit.Before; import org.junit.BeforeClass; From 0445f8200e0630e1bb8b7117f200529ed1259747 Mon Sep 17 00:00:00 2001 From: markrmiller Date: Fri, 31 Mar 2017 10:53:20 -0400 Subject: [PATCH 092/563] SOLR-10338: Configure SecureRandom non blocking for tests. --- dev-tools/idea/.idea/workspace.xml | 2 +- dev-tools/maven/pom.xml.template | 1 + lucene/common-build.xml | 3 ++ solr/CHANGES.txt | 2 + .../test/SecureRandomAlgorithmTesterApp.java | 41 +++++++++++++++++++ .../java/org/apache/solr/SolrTestCaseJ4.java | 9 ++++ 6 files changed, 57 insertions(+), 1 deletion(-) create mode 100644 solr/core/src/test/SecureRandomAlgorithmTesterApp.java diff --git a/dev-tools/idea/.idea/workspace.xml b/dev-tools/idea/.idea/workspace.xml index 0ca7f0c2b70..e22108f04b4 100644 --- a/dev-tools/idea/.idea/workspace.xml +++ b/dev-tools/idea/.idea/workspace.xml @@ -2,7 +2,7 @@ - diff --git a/dev-tools/maven/pom.xml.template b/dev-tools/maven/pom.xml.template index cd8d6b8db77..6b7f915e22d 100644 --- a/dev-tools/maven/pom.xml.template +++ b/dev-tools/maven/pom.xml.template @@ -277,6 +277,7 @@ ${tests.postingsformat} ${tests.timezone} ${tests.verbose} + file:/dev/./urandom diff --git a/lucene/common-build.xml b/lucene/common-build.xml index 327a01d8486..aee7899d1f6 100644 --- a/lucene/common-build.xml +++ b/lucene/common-build.xml @@ -1062,6 +1062,9 @@ + + + diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 840de480058..097ee2b0690 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -176,6 +176,8 @@ Other Changes * SOLR-10147: Admin UI -> Cloud -> Graph: Impossible to see shard state (Amrit Sarkar, janhoy) +* SOLR-10338: Configure SecureRandom non blocking for tests. (Mihaly Toth, hossman, Ishan Chattopadhyaya, via Mark Miller) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/core/src/test/SecureRandomAlgorithmTesterApp.java b/solr/core/src/test/SecureRandomAlgorithmTesterApp.java new file mode 100644 index 00000000000..44f79e903c8 --- /dev/null +++ b/solr/core/src/test/SecureRandomAlgorithmTesterApp.java @@ -0,0 +1,41 @@ +/* + * 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. + */ + +import java.security.NoSuchAlgorithmException; +import java.security.SecureRandom; + +public class SecureRandomAlgorithmTesterApp { + public static void main(String[] args) throws NoSuchAlgorithmException { + String algorithm = args[0]; + String method = args[1]; + int amount = Integer.valueOf(args[2]); + SecureRandom secureRandom; + if(algorithm.equals("default")) + secureRandom = new SecureRandom(); + else + secureRandom = SecureRandom.getInstance(algorithm); + System.out.println("Algorithm:" + secureRandom.getAlgorithm()); + switch(method) { + case "seed": secureRandom.generateSeed(amount); break; + case "bytes": secureRandom.nextBytes(new byte[amount]); break; + case "long": secureRandom.nextLong(); break; + case "int": secureRandom.nextInt(); break; + default: throw new IllegalArgumentException("Not supported random function: " + method); + } + System.out.println("SecureRandom function invoked"); + } +} diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java index f82ccc6e975..be8e96db41e 100644 --- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java +++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java @@ -39,6 +39,7 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.security.SecureRandom; import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; @@ -2431,6 +2432,14 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase { } } + @BeforeClass + public static void assertNonBlockingRandomGeneratorAvailable() { + if(Boolean.parseBoolean(System.getProperty("test.solr.allow.any.securerandom","false"))) + return; + // Use -Djava.security.egd=file:/dev/./urandom VM option if you hit this + assertEquals("SHA1PRNG", new SecureRandom().getAlgorithm()); + } + @AfterClass public static void unchooseMPForMP() { System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY); From 23c6ea274ea3e288a916acc160c2ca6c63d3c4dd Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 31 Mar 2017 16:11:19 +0200 Subject: [PATCH 093/563] LUCENE-7761: Fixed comment in ReqExclScorer. --- lucene/CHANGES.txt | 3 +++ .../core/src/java/org/apache/lucene/search/ReqExclScorer.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index da643ff5b82..c8a8deb3b03 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -93,6 +93,9 @@ Other * LUCENE-7743: Never call new String(String). (Daniel Jelinski via Adrien Grand) +* LUCENE-7761: Fixed comment in ReqExclScorer. + (Pablo Pita Leira via Adrien Grand) + ======================= Lucene 6.5.1 ======================= Bug Fixes diff --git a/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java b/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java index 5ce6f5edee7..50a321b9236 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java @@ -154,7 +154,7 @@ class ReqExclScorer extends Scorer { } }; } else { - // reqTwoPhaseIterator is MORE costly than exclTwoPhaseIterator, check it first + // reqTwoPhaseIterator is MORE costly than exclTwoPhaseIterator, check it last return new TwoPhaseIterator(reqApproximation) { @Override From e80643e5a77297ba7ee29c7297e37af6ffb73ac2 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 31 Mar 2017 16:22:45 +0200 Subject: [PATCH 094/563] LUCENE-7753: Make fields static when possible. --- lucene/CHANGES.txt | 3 + .../byTask/feeds/EnwikiContentSourceTest.java | 4 +- .../apache/lucene/index/TestCustomNorms.java | 14 +- .../org/apache/lucene/index/TestNorms.java | 12 +- .../index/TestSameTokenSamePosition.java | 4 +- .../lucene/search/TestAutomatonQuery.java | 2 +- .../search/TestAutomatonQueryUnicode.java | 2 +- .../TestEarlyTerminatingSortingCollector.java | 12 +- .../apache/lucene/search/TestRegexpQuery.java | 2 +- .../lucene/search/TestSearchWithThreads.java | 2 +- .../apache/lucene/store/TestRAMDirectory.java | 26 ++-- .../grouping/DistinctValuesCollectorTest.java | 48 +++---- .../LengthGoalBreakIteratorTest.java | 26 ++-- .../vectorhighlight/AbstractTestCase.java | 6 +- .../surround/parser/QueryParser.java | 41 +++--- .../surround/parser/QueryParser.jj | 45 +++--- .../surround/query/FieldsQuery.java | 4 +- .../lucene/spatial3d/geom/GeoBBoxTest.java | 2 +- .../mockrandom/MockRandomPostingsFormat.java | 2 +- .../request/AnalyticsContentHandler.java | 44 +++--- .../handler/dataimport/TestDocBuilder2.java | 48 +++---- .../TestHierarchicalDocBuilder.java | 16 +-- .../org/apache/solr/cloud/DistributedMap.java | 10 +- .../apache/solr/cloud/OverseerTaskQueue.java | 6 +- .../org/apache/solr/schema/CurrencyField.java | 2 +- .../org/apache/solr/update/CommitTracker.java | 6 +- .../org/apache/solr/util/SimplePostTool.java | 2 +- .../org/apache/solr/cloud/OverseerTest.java | 134 +++++++++--------- .../cloud/SegmentTerminateEarlyTestState.java | 78 +++++----- .../apache/solr/cloud/SolrXmlInZkTest.java | 2 +- .../apache/solr/cloud/TestSegmentSorting.java | 2 +- .../solr/core/OpenCloseCoreStressTest.java | 6 +- .../solr/schema/SpatialRPTFieldTypeTest.java | 12 +- .../org/apache/solr/search/TestRTGBase.java | 2 +- .../apache/solr/search/TestRealTimeGet.java | 4 +- .../solr/search/TestReloadDeadlock.java | 2 +- .../apache/solr/search/TestStressLucene.java | 8 +- .../solr/search/TestStressRecovery.java | 4 +- .../apache/solr/search/TestStressReorder.java | 4 +- .../solr/search/TestStressUserVersions.java | 6 +- .../solr/search/TestStressVersions.java | 4 +- .../solr/common/util/TestJavaBinCodec.java | 4 +- 42 files changed, 326 insertions(+), 337 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index c8a8deb3b03..e0827e78e90 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -80,6 +80,9 @@ Other * LUCENE-7681: MemoryIndex uses new DocValues API (Alan Woodward) +* LUCENE-7753: Make fields static when possible. + (Daniel Jelinski via Adrien Grand) + ======================= Lucene 6.6.0 ======================= Other diff --git a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSourceTest.java b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSourceTest.java index 9e76700153a..d5533c27417 100644 --- a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSourceTest.java +++ b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSourceTest.java @@ -61,7 +61,7 @@ public class EnwikiContentSourceTest extends LuceneTestCase { }); } - private final String PAGE1 = + private static final String PAGE1 = " \r\n" + " Title1\r\n" + " 0\r\n" + @@ -80,7 +80,7 @@ public class EnwikiContentSourceTest extends LuceneTestCase { " \r\n" + " \r\n"; - private final String PAGE2 = + private static final String PAGE2 = " \r\n" + " Title2\r\n" + " 0\r\n" + diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java b/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java index f1931400a2e..a8111921ff6 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestCustomNorms.java @@ -38,8 +38,8 @@ import org.apache.lucene.util.TestUtil; * */ public class TestCustomNorms extends LuceneTestCase { - final String floatTestField = "normsTestFloat"; - final String exceptionTestField = "normsTestExcp"; + static final String FLOAT_TEST_FIELD = "normsTestFloat"; + static final String EXCEPTION_TEST_FIELD = "normsTestExcp"; public void testFloatNorms() throws IOException { @@ -57,11 +57,11 @@ public class TestCustomNorms extends LuceneTestCase { Document doc = docs.nextDoc(); int boost = TestUtil.nextInt(random(), 1, 10); String value = IntStream.range(0, boost).mapToObj(k -> Integer.toString(boost)).collect(Collectors.joining(" ")); - Field f = new TextField(floatTestField, value, Field.Store.YES); + Field f = new TextField(FLOAT_TEST_FIELD, value, Field.Store.YES); doc.add(f); writer.addDocument(doc); - doc.removeField(floatTestField); + doc.removeField(FLOAT_TEST_FIELD); if (rarely()) { writer.commit(); } @@ -69,11 +69,11 @@ public class TestCustomNorms extends LuceneTestCase { writer.commit(); writer.close(); DirectoryReader open = DirectoryReader.open(dir); - NumericDocValues norms = MultiDocValues.getNormValues(open, floatTestField); + NumericDocValues norms = MultiDocValues.getNormValues(open, FLOAT_TEST_FIELD); assertNotNull(norms); for (int i = 0; i < open.maxDoc(); i++) { Document document = open.document(i); - int expected = Integer.parseInt(document.get(floatTestField).split(" ")[0]); + int expected = Integer.parseInt(document.get(FLOAT_TEST_FIELD).split(" ")[0]); assertEquals(i, norms.nextDoc()); assertEquals(expected, norms.longValue()); } @@ -87,7 +87,7 @@ public class TestCustomNorms extends LuceneTestCase { @Override public Similarity get(String field) { - if (floatTestField.equals(field)) { + if (FLOAT_TEST_FIELD.equals(field)) { return new FloatEncodingBoostSimilarity(); } else { return delegate; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java index 16ce61daf9d..64c064996be 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestNorms.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestNorms.java @@ -48,7 +48,7 @@ import org.apache.lucene.util.TestUtil; @SuppressCodecs({ "Memory", "Direct", "SimpleText" }) @Slow public class TestNorms extends LuceneTestCase { - final String byteTestField = "normsTestByte"; + static final String BYTE_TEST_FIELD = "normsTestByte"; static class CustomNormEncodingSimilarity extends TFIDFSimilarity { @@ -115,11 +115,11 @@ public class TestNorms extends LuceneTestCase { Directory dir = newFSDirectory(createTempDir("TestNorms.testMaxByteNorms")); buildIndex(dir); DirectoryReader open = DirectoryReader.open(dir); - NumericDocValues normValues = MultiDocValues.getNormValues(open, byteTestField); + NumericDocValues normValues = MultiDocValues.getNormValues(open, BYTE_TEST_FIELD); assertNotNull(normValues); for (int i = 0; i < open.maxDoc(); i++) { Document document = open.document(i); - int expected = Integer.parseInt(document.get(byteTestField).split(" ")[0]); + int expected = Integer.parseInt(document.get(BYTE_TEST_FIELD).split(" ")[0]); assertEquals(i, normValues.nextDoc()); assertEquals(expected, normValues.longValue()); } @@ -143,10 +143,10 @@ public class TestNorms extends LuceneTestCase { Document doc = docs.nextDoc(); int boost = TestUtil.nextInt(random, 1, 255); String value = IntStream.range(0, boost).mapToObj(k -> Integer.toString(boost)).collect(Collectors.joining(" ")); - Field f = new TextField(byteTestField, value, Field.Store.YES); + Field f = new TextField(BYTE_TEST_FIELD, value, Field.Store.YES); doc.add(f); writer.addDocument(doc); - doc.removeField(byteTestField); + doc.removeField(BYTE_TEST_FIELD); if (rarely()) { writer.commit(); } @@ -162,7 +162,7 @@ public class TestNorms extends LuceneTestCase { @Override public Similarity get(String field) { - if (byteTestField.equals(field)) { + if (BYTE_TEST_FIELD.equals(field)) { return new ByteEncodingBoostSimilarity(); } else { return delegate; diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSameTokenSamePosition.java b/lucene/core/src/test/org/apache/lucene/index/TestSameTokenSamePosition.java index 4bb23181286..5d3c0911c7d 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSameTokenSamePosition.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSameTokenSamePosition.java @@ -64,7 +64,7 @@ final class BugReproTokenStream extends TokenStream { private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class); private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class); private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class); - private final int tokenCount = 4; + private static final int TOKEN_COUNT = 4; private int nextTokenIndex = 0; private final String terms[] = new String[]{"six", "six", "drunken", "drunken"}; private final int starts[] = new int[]{0, 0, 4, 4}; @@ -73,7 +73,7 @@ final class BugReproTokenStream extends TokenStream { @Override public boolean incrementToken() { - if (nextTokenIndex < tokenCount) { + if (nextTokenIndex < TOKEN_COUNT) { termAtt.setEmpty().append(terms[nextTokenIndex]); offsetAtt.setOffset(starts[nextTokenIndex], ends[nextTokenIndex]); posIncAtt.setPositionIncrement(incs[nextTokenIndex]); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java index d4c865fac01..3da69f08219 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQuery.java @@ -49,7 +49,7 @@ public class TestAutomatonQuery extends LuceneTestCase { private IndexReader reader; private IndexSearcher searcher; - private final String FN = "field"; + private static final String FN = "field"; @Override public void setUp() throws Exception { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java b/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java index 7b103645aac..2af30985325 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestAutomatonQueryUnicode.java @@ -39,7 +39,7 @@ public class TestAutomatonQueryUnicode extends LuceneTestCase { private IndexSearcher searcher; private Directory directory; - private final String FN = "field"; + private static final String FN = "field"; @Override public void setUp() throws Exception { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java index 6d699e8a1c1..217834cd319 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestEarlyTerminatingSortingCollector.java @@ -35,14 +35,6 @@ import org.apache.lucene.index.QueryTimeout; import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.SerialMergeScheduler; import org.apache.lucene.index.Term; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.MatchAllDocsQuery; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; -import org.apache.lucene.search.TermQuery; -import org.apache.lucene.search.TopFieldCollector; import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; @@ -57,7 +49,7 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase { private final Sort sort = new Sort(new SortField("ndv1", SortField.Type.LONG)); private RandomIndexWriter iw; private IndexReader reader; - private final int forceMergeMaxSegmentCount = 5; + private static final int FORCE_MERGE_MAX_SEGMENT_COUNT = 5; private Document randomDocument() { final Document doc = new Document(); @@ -107,7 +99,7 @@ public class TestEarlyTerminatingSortingCollector extends LuceneTestCase { iw.forceMerge(1); } else if (random().nextBoolean()) { - iw.forceMerge(forceMergeMaxSegmentCount); + iw.forceMerge(FORCE_MERGE_MAX_SEGMENT_COUNT); } reader = iw.getReader(); } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java index b1cbe1e1f7c..505b26ff914 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestRegexpQuery.java @@ -42,7 +42,7 @@ public class TestRegexpQuery extends LuceneTestCase { private IndexSearcher searcher; private IndexReader reader; private Directory directory; - private final String FN = "field"; + private static final String FN = "field"; @Override public void setUp() throws Exception { diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearchWithThreads.java b/lucene/core/src/test/org/apache/lucene/search/TestSearchWithThreads.java index 95c0b8ed6ea..0008dc71022 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestSearchWithThreads.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestSearchWithThreads.java @@ -32,7 +32,7 @@ import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; @SuppressCodecs({ "SimpleText", "Memory", "Direct" }) public class TestSearchWithThreads extends LuceneTestCase { int NUM_DOCS; - final int NUM_SEARCH_THREADS = 5; + static final int NUM_SEARCH_THREADS = 5; int RUN_TIME_MSEC; @Override diff --git a/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java index d4e8bfc2242..f0f2d460737 100644 --- a/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java +++ b/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java @@ -49,7 +49,7 @@ public class TestRAMDirectory extends BaseDirectoryTestCase { } // add enough document so that the index will be larger than RAMDirectory.READ_BUFFER_SIZE - private final int docsToAdd = 500; + private static final int DOCS_TO_ADD = 500; private Path buildIndex() throws IOException { Path path = createTempDir("buildIndex"); @@ -59,12 +59,12 @@ public class TestRAMDirectory extends BaseDirectoryTestCase { new MockAnalyzer(random())).setOpenMode(OpenMode.CREATE)); // add some documents Document doc = null; - for (int i = 0; i < docsToAdd; i++) { + for (int i = 0; i < DOCS_TO_ADD; i++) { doc = new Document(); doc.add(newStringField("content", English.intToEnglish(i).trim(), Field.Store.YES)); writer.addDocument(doc); } - assertEquals(docsToAdd, writer.maxDoc()); + assertEquals(DOCS_TO_ADD, writer.maxDoc()); writer.close(); dir.close(); @@ -100,13 +100,13 @@ public class TestRAMDirectory extends BaseDirectoryTestCase { // open reader to test document count IndexReader reader = DirectoryReader.open(ramDir); - assertEquals(docsToAdd, reader.numDocs()); + assertEquals(DOCS_TO_ADD, reader.numDocs()); // open search zo check if all doc's are there IndexSearcher searcher = newSearcher(reader); // search for all documents - for (int i = 0; i < docsToAdd; i++) { + for (int i = 0; i < DOCS_TO_ADD; i++) { Document doc = searcher.doc(i); assertTrue(doc.getField("content") != null); } @@ -115,8 +115,8 @@ public class TestRAMDirectory extends BaseDirectoryTestCase { reader.close(); } - private final int numThreads = 10; - private final int docsPerThread = 40; + private static final int NUM_THREADS = 10; + private static final int DOCS_PER_THREAD = 40; public void testRAMDirectorySize() throws IOException, InterruptedException { @@ -132,15 +132,15 @@ public class TestRAMDirectory extends BaseDirectoryTestCase { assertEquals(ramDir.sizeInBytes(), ramDir.getRecomputedSizeInBytes()); - Thread[] threads = new Thread[numThreads]; - for (int i=0; i> firstCollector = createRandomFirstPassCollector(new Sort(), groupField, 10); + FirstPassGroupingCollector> firstCollector = createRandomFirstPassCollector(new Sort(), GROUP_FIELD, 10); indexSearcher.search(new TermQuery(new Term("content", "random")), firstCollector); DistinctValuesCollector> distinctValuesCollector - = createDistinctCountCollector(firstCollector, groupField, countField); + = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD); indexSearcher.search(new TermQuery(new Term("content", "random")), distinctValuesCollector); List>> gcs = distinctValuesCollector.getGroups(); @@ -178,9 +178,9 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase { compare("1", countValues.get(0)); // === Search for content:some - firstCollector = createRandomFirstPassCollector(new Sort(), groupField, 10); + firstCollector = createRandomFirstPassCollector(new Sort(), GROUP_FIELD, 10); indexSearcher.search(new TermQuery(new Term("content", "some")), firstCollector); - distinctValuesCollector = createDistinctCountCollector(firstCollector, groupField, countField); + distinctValuesCollector = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD); indexSearcher.search(new TermQuery(new Term("content", "some")), distinctValuesCollector); gcs = distinctValuesCollector.getGroups(); @@ -205,9 +205,9 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase { compare("1", countValues.get(0)); // === Search for content:blob - firstCollector = createRandomFirstPassCollector(new Sort(), groupField, 10); + firstCollector = createRandomFirstPassCollector(new Sort(), GROUP_FIELD, 10); indexSearcher.search(new TermQuery(new Term("content", "blob")), firstCollector); - distinctValuesCollector = createDistinctCountCollector(firstCollector, groupField, countField); + distinctValuesCollector = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD); indexSearcher.search(new TermQuery(new Term("content", "blob")), distinctValuesCollector); gcs = distinctValuesCollector.getGroups(); @@ -242,10 +242,10 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase { List>> expectedResult = createExpectedResult(context, term, groupSort, topN); - FirstPassGroupingCollector> firstCollector = createRandomFirstPassCollector(groupSort, groupField, topN); + FirstPassGroupingCollector> firstCollector = createRandomFirstPassCollector(groupSort, GROUP_FIELD, topN); searcher.search(new TermQuery(new Term("content", term)), firstCollector); DistinctValuesCollector> distinctValuesCollector - = createDistinctCountCollector(firstCollector, groupField, countField); + = createDistinctCountCollector(firstCollector, GROUP_FIELD, COUNT_FIELD); searcher.search(new TermQuery(new Term("content", term)), distinctValuesCollector); @SuppressWarnings("unchecked") List>> actualResult = distinctValuesCollector.getGroups(); @@ -440,10 +440,10 @@ public class DistinctValuesCollectorTest extends AbstractGroupingTestCase { doc.add(new StringField("id", String.format(Locale.ROOT, "%09d", i), Field.Store.YES)); doc.add(new SortedDocValuesField("id", new BytesRef(String.format(Locale.ROOT, "%09d", i)))); if (groupValue != null) { - addField(doc, groupField, groupValue); + addField(doc, GROUP_FIELD, groupValue); } if (countValue != null) { - addField(doc, countField, countValue); + addField(doc, COUNT_FIELD, countValue); } doc.add(new TextField("content", content, Field.Store.YES)); w.addDocument(doc); diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java index 42d2bf6b112..4dd30e2bd16 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/uhighlight/LengthGoalBreakIteratorTest.java @@ -39,7 +39,7 @@ public class LengthGoalBreakIteratorTest extends LuceneTestCase { // We do a '.' BreakIterator and test varying the length goal. // 0 1 // 01234567890123456789 - final String content = "Aa bb. Cc dd. Ee ff"; + static final String CONTENT = "Aa bb. Cc dd. Ee ff"; public void testTargetLen() throws IOException { // "goal" means target length goal to find closest break @@ -47,22 +47,22 @@ public class LengthGoalBreakIteratorTest extends LuceneTestCase { // at first word: Query query = query("aa"); assertEquals("almost two sent", - "Aa bb.", highlightClosestToLen(content, query, 9)); + "Aa bb.", highlightClosestToLen(CONTENT, query, 9)); assertEquals( "barely two sent", - "Aa bb. Cc dd.", highlightClosestToLen(content, query, 10)); + "Aa bb. Cc dd.", highlightClosestToLen(CONTENT, query, 10)); assertEquals("long goal", - "Aa bb. Cc dd. Ee ff", highlightClosestToLen(content, query, 17 + random().nextInt(20))); + "Aa bb. Cc dd. Ee ff", highlightClosestToLen(CONTENT, query, 17 + random().nextInt(20))); // at some word not at start of passage query = query("dd"); assertEquals("short goal", - " Cc dd.", highlightClosestToLen(content, query, random().nextInt(5))); + " Cc dd.", highlightClosestToLen(CONTENT, query, random().nextInt(5))); assertEquals("almost two sent", - " Cc dd.", highlightClosestToLen(content, query, 10)); + " Cc dd.", highlightClosestToLen(CONTENT, query, 10)); assertEquals("barely two sent", - " Cc dd. Ee ff", highlightClosestToLen(content, query, 11)); + " Cc dd. Ee ff", highlightClosestToLen(CONTENT, query, 11)); assertEquals("long goal", - " Cc dd. Ee ff", highlightClosestToLen(content, query, 12 + random().nextInt(20))); + " Cc dd. Ee ff", highlightClosestToLen(CONTENT, query, 12 + random().nextInt(20))); } public void testMinLen() throws IOException { @@ -70,19 +70,19 @@ public class LengthGoalBreakIteratorTest extends LuceneTestCase { Query query = query("dd"); assertEquals("almost two sent", - " Cc dd.", highlightMinLen(content, query, 6)); + " Cc dd.", highlightMinLen(CONTENT, query, 6)); assertEquals("barely two sent", - " Cc dd. Ee ff", highlightMinLen(content, query, 7)); + " Cc dd. Ee ff", highlightMinLen(CONTENT, query, 7)); } public void testDefaultSummaryTargetLen() throws IOException { Query query = query("zz"); assertEquals("Aa bb.", - highlightClosestToLen(content, query, random().nextInt(10))); // < 10 + highlightClosestToLen(CONTENT, query, random().nextInt(10))); // < 10 assertEquals("Aa bb. Cc dd.", - highlightClosestToLen(content, query, 10 + 6)); // cusp of adding 3rd sentence + highlightClosestToLen(CONTENT, query, 10 + 6)); // cusp of adding 3rd sentence assertEquals("Aa bb. Cc dd. Ee ff", - highlightClosestToLen(content, query, 17 + random().nextInt(20))); // >= 14 + highlightClosestToLen(CONTENT, query, 17 + random().nextInt(20))); // >= 14 } private Query query(String qStr) { diff --git a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/AbstractTestCase.java b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/AbstractTestCase.java index be75e64c3ce..24242196f63 100644 --- a/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/AbstractTestCase.java +++ b/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/AbstractTestCase.java @@ -47,9 +47,9 @@ import org.apache.lucene.util.LuceneTestCase; public abstract class AbstractTestCase extends LuceneTestCase { - protected final String F = "f"; - protected final String F1 = "f1"; - protected final String F2 = "f2"; + protected static final String F = "f"; + protected static final String F1 = "f1"; + protected static final String F2 = "f2"; protected Directory dir; protected Analyzer analyzerW; protected Analyzer analyzerB; 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 f0f4b34f8f1..e4135844ae2 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 @@ -42,18 +42,15 @@ import org.apache.lucene.queryparser.surround.query.SrndTruncQuery; */ public class QueryParser implements QueryParserConstants { - final int minimumPrefixLength = 3; - final int minimumCharsInTrunc = 3; - final String truncationErrorMessage = "Too unrestrictive truncation: "; - final String boostErrorMessage = "Cannot handle boost value: "; + static final int MINIMUM_PREFIX_LENGTH = 3; + static final int MINIMUM_CHARS_IN_TRUNC = 3; + static final String TRUNCATION_ERROR_MESSAGE = "Too unrestrictive truncation: "; + static final String BOOST_ERROR_MESSAGE = "Cannot handle boost value: "; /* CHECKME: These should be the same as for the tokenizer. How? */ - final char truncator = '*'; - final char anyChar = '?'; - final char quote = '"'; - final char fieldOperator = ':'; - final char comma = ','; /* prefix list separator */ - final char carat = '^'; /* weight operator */ + static final char TRUNCATOR = '*'; + static final char ANY_CHAR = '?'; + static final char FIELD_OPERATOR = ':'; static public SrndQuery parse(String query) throws ParseException { QueryParser parser = new QueryParser(); @@ -78,7 +75,7 @@ public class QueryParser implements QueryParserConstants { /* FIXME: check acceptable subquery: at least one subquery should not be * a fields query. */ - return new FieldsQuery(q, fieldNames, fieldOperator); + return new FieldsQuery(q, fieldNames, FIELD_OPERATOR); } protected SrndQuery getOrQuery(List queries, boolean infix, Token orToken) { @@ -128,12 +125,12 @@ public class QueryParser implements QueryParserConstants { } protected boolean allowedSuffix(String suffixed) { - return (suffixed.length() - 1) >= minimumPrefixLength; + return (suffixed.length() - 1) >= MINIMUM_PREFIX_LENGTH; } protected SrndQuery getPrefixQuery( String prefix, boolean quoted) { - return new SrndPrefixQuery(prefix, quoted, truncator); + return new SrndPrefixQuery(prefix, quoted, TRUNCATOR); } protected boolean allowedTruncation(String truncated) { @@ -141,15 +138,15 @@ public class QueryParser implements QueryParserConstants { int nrNormalChars = 0; for (int i = 0; i < truncated.length(); i++) { char c = truncated.charAt(i); - if ((c != truncator) && (c != anyChar)) { + if ((c != TRUNCATOR) && (c != ANY_CHAR)) { nrNormalChars++; } } - return nrNormalChars >= minimumCharsInTrunc; + return nrNormalChars >= MINIMUM_CHARS_IN_TRUNC; } protected SrndQuery getTruncQuery(String truncated) { - return new SrndTruncQuery(truncated, truncator, anyChar); + return new SrndTruncQuery(truncated, TRUNCATOR, ANY_CHAR); } final public SrndQuery TopSrndQuery() throws ParseException { @@ -437,7 +434,7 @@ public class QueryParser implements QueryParserConstants { term = jj_consume_token(SUFFIXTERM); /* ending in * */ if (! allowedSuffix(term.image)) { - {if (true) throw new ParseException(truncationErrorMessage + term.image);} + {if (true) throw new ParseException(TRUNCATION_ERROR_MESSAGE + term.image);} } {if (true) return getPrefixQuery(term.image.substring(0, term.image.length()-1), false /* not quoted */);} break; @@ -445,15 +442,15 @@ public class QueryParser implements QueryParserConstants { term = jj_consume_token(TRUNCTERM); /* with at least one * or ? */ if (! allowedTruncation(term.image)) { - {if (true) throw new ParseException(truncationErrorMessage + term.image);} + {if (true) throw new ParseException(TRUNCATION_ERROR_MESSAGE + term.image);} } {if (true) return getTruncQuery(term.image);} break; case TRUNCQUOTED: term = jj_consume_token(TRUNCQUOTED); /* eg. "9b-b,m"* */ - if ((term.image.length() - 3) < minimumPrefixLength) { - {if (true) throw new ParseException(truncationErrorMessage + term.image);} + if ((term.image.length() - 3) < MINIMUM_PREFIX_LENGTH) { + {if (true) throw new ParseException(TRUNCATION_ERROR_MESSAGE + term.image);} } {if (true) return getPrefixQuery(term.image.substring(1, term.image.length()-2), true /* quoted */);} break; @@ -483,10 +480,10 @@ public class QueryParser implements QueryParserConstants { try { f = Float.parseFloat(weight.image); } catch (Exception floatExc) { - {if (true) throw new ParseException(boostErrorMessage + weight.image + " (" + floatExc + ")");} + {if (true) throw new ParseException(BOOST_ERROR_MESSAGE + weight.image + " (" + floatExc + ")");} } if (f <= 0.0) { - {if (true) throw new ParseException(boostErrorMessage + weight.image);} + {if (true) throw new ParseException(BOOST_ERROR_MESSAGE + weight.image);} } q.setWeight(f * q.getWeight()); /* left associative, fwiw */ 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 857cca16df7..787ed16bde4 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 @@ -73,19 +73,16 @@ import org.apache.lucene.queryparser.surround.query.SrndTruncQuery; */ public class QueryParser { - final int minimumPrefixLength = 3; - final int minimumCharsInTrunc = 3; - final String truncationErrorMessage = "Too unrestrictive truncation: "; - final String boostErrorMessage = "Cannot handle boost value: "; - + static final int MINIMUM_PREFIX_LENGTH = 3; + static final int MINIMUM_CHARS_IN_TRUNC = 3; + static final String TRUNCATION_ERROR_MESSAGE = "Too unrestrictive truncation: "; + static final String BOOST_ERROR_MESSAGE = "Cannot handle boost value: "; + /* CHECKME: These should be the same as for the tokenizer. How? */ - final char truncator = '*'; - final char anyChar = '?'; - final char quote = '"'; - final char fieldOperator = ':'; - final char comma = ','; /* prefix list separator */ - final char carat = '^'; /* weight operator */ - + static final char TRUNCATOR = '*'; + static final char ANY_CHAR = '?'; + static final char FIELD_OPERATOR = ':'; + static public SrndQuery parse(String query) throws ParseException { QueryParser parser = new QueryParser(); return parser.parse2(query); @@ -109,7 +106,7 @@ public class QueryParser { /* FIXME: check acceptable subquery: at least one subquery should not be * a fields query. */ - return new FieldsQuery(q, fieldNames, fieldOperator); + return new FieldsQuery(q, fieldNames, FIELD_OPERATOR); } protected SrndQuery getOrQuery(List queries, boolean infix, Token orToken) { @@ -159,12 +156,12 @@ public class QueryParser { } protected boolean allowedSuffix(String suffixed) { - return (suffixed.length() - 1) >= minimumPrefixLength; + return (suffixed.length() - 1) >= MINIMUM_PREFIX_LENGTH; } protected SrndQuery getPrefixQuery( String prefix, boolean quoted) { - return new SrndPrefixQuery(prefix, quoted, truncator); + return new SrndPrefixQuery(prefix, quoted, TRUNCATOR); } protected boolean allowedTruncation(String truncated) { @@ -172,15 +169,15 @@ public class QueryParser { int nrNormalChars = 0; for (int i = 0; i < truncated.length(); i++) { char c = truncated.charAt(i); - if ((c != truncator) && (c != anyChar)) { + if ((c != TRUNCATOR) && (c != ANY_CHAR)) { nrNormalChars++; } } - return nrNormalChars >= minimumCharsInTrunc; + return nrNormalChars >= MINIMUM_CHARS_IN_TRUNC; } protected SrndQuery getTruncQuery(String truncated) { - return new SrndTruncQuery(truncated, truncator, anyChar); + return new SrndTruncQuery(truncated, TRUNCATOR, ANY_CHAR); } } @@ -432,21 +429,21 @@ SrndQuery SimpleTerm() : { | term= { /* ending in * */ if (! allowedSuffix(term.image)) { - throw new ParseException(truncationErrorMessage + term.image); + throw new ParseException(TRUNCATION_ERROR_MESSAGE + term.image); } return getPrefixQuery(term.image.substring(0, term.image.length()-1), false /* not quoted */); } | term= { /* with at least one * or ? */ if (! allowedTruncation(term.image)) { - throw new ParseException(truncationErrorMessage + term.image); + throw new ParseException(TRUNCATION_ERROR_MESSAGE + term.image); } return getTruncQuery(term.image); } | term= { /* eg. "9b-b,m"* */ - if ((term.image.length() - 3) < minimumPrefixLength) { - throw new ParseException(truncationErrorMessage + term.image); + if ((term.image.length() - 3) < MINIMUM_PREFIX_LENGTH) { + throw new ParseException(TRUNCATION_ERROR_MESSAGE + term.image); } return getPrefixQuery(term.image.substring(1, term.image.length()-2), true /* quoted */); } @@ -462,10 +459,10 @@ void OptionalWeights(SrndQuery q) : { try { f = Float.parseFloat(weight.image); } catch (Exception floatExc) { - throw new ParseException(boostErrorMessage + weight.image + " (" + floatExc + ")"); + throw new ParseException(BOOST_ERROR_MESSAGE + weight.image + " (" + floatExc + ")"); } if (f <= 0.0) { - throw new ParseException(boostErrorMessage + weight.image); + throw new ParseException(BOOST_ERROR_MESSAGE + weight.image); } q.setWeight(f * q.getWeight()); /* left associative, fwiw */ } diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/query/FieldsQuery.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/query/FieldsQuery.java index 23bb0952696..4d933b7b0a4 100644 --- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/query/FieldsQuery.java +++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/query/FieldsQuery.java @@ -28,7 +28,7 @@ public class FieldsQuery extends SrndQuery { /* mostly untested */ private SrndQuery q; private List fieldNames; private final char fieldOp; - private final String OrOperatorName = "OR"; /* for expanded queries, not normally visible */ + private static final String OR_OPERATOR_NAME = "OR"; /* for expanded queries, not normally visible */ public FieldsQuery(SrndQuery q, List fieldNames, char fieldOp) { this.q = q; @@ -61,7 +61,7 @@ public class FieldsQuery extends SrndQuery { /* mostly untested */ } OrQuery oq = new OrQuery(queries, true /* infix OR for field names */, - OrOperatorName); + OR_OPERATOR_NAME); // System.out.println(getClass().toString() + ", fields expanded: " + oq.toString()); /* needs testing */ return oq.makeLuceneQueryField(null, qf); } diff --git a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoBBoxTest.java b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoBBoxTest.java index 7530c7406e9..401d8513031 100755 --- a/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoBBoxTest.java +++ b/lucene/spatial3d/src/test/org/apache/lucene/spatial3d/geom/GeoBBoxTest.java @@ -27,7 +27,7 @@ import static org.junit.Assert.assertTrue; public class GeoBBoxTest { - protected final double DEGREES_TO_RADIANS = Math.PI / 180.0; + protected static final double DEGREES_TO_RADIANS = Math.PI / 180.0; @Test public void testBBoxDegenerate() { diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java index 78bf299c1ac..6f57a2e1569 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java +++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java @@ -60,7 +60,7 @@ import org.apache.lucene.util.TestUtil; public final class MockRandomPostingsFormat extends PostingsFormat { private final Random seedRandom; - private final String SEED_EXT = "sd"; + private static final String SEED_EXT = "sd"; public MockRandomPostingsFormat() { // This ctor should *only* be used at read-time: get NPE if you use it! diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsContentHandler.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsContentHandler.java index 31b0576a7d2..b93a59e9fb9 100644 --- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsContentHandler.java +++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/request/AnalyticsContentHandler.java @@ -35,34 +35,34 @@ import org.xml.sax.SAXException; */ public class AnalyticsContentHandler implements ContentHandler { // XML Element/Attribute Name Constants - public final String ANALYTICS_REQUEST_ENVELOPE="analyticsRequestEnvelope"; + public static final String ANALYTICS_REQUEST_ENVELOPE="analyticsRequestEnvelope"; - public final String ANALYTICS_REQUEST="analyticsRequest"; - public final String NAME="name"; + public static final String ANALYTICS_REQUEST="analyticsRequest"; + public static final String NAME="name"; - public final String STATISTIC="statistic"; - public final String EXPRESSION="expression"; + public static final String STATISTIC="statistic"; + public static final String EXPRESSION="expression"; - public final String FIELD_FACET="fieldFacet"; - public final String FIELD="field"; - public final String SHOW_MISSING="showMissing"; - public final String LIMIT="limit"; - public final String MIN_COUNT="minCount"; + public static final String FIELD_FACET="fieldFacet"; + public static final String FIELD="field"; + public static final String SHOW_MISSING="showMissing"; + public static final String LIMIT="limit"; + public static final String MIN_COUNT="minCount"; - public final String SORT_SPECIFICATION="sortSpecification"; - public final String STAT_NAME="statName"; - public final String DIRECTION="direction"; + public static final String SORT_SPECIFICATION="sortSpecification"; + public static final String STAT_NAME="statName"; + public static final String DIRECTION="direction"; - public final String RANGE_FACET="rangeFacet"; - public final String START="start"; - public final String END="end"; - public final String GAP="gap"; - public final String INCLUDE_BOUNDARY="includeBoundary"; - public final String OTHER_RANGE="otherRange"; - public final String HARD_END="hardend"; + public static final String RANGE_FACET="rangeFacet"; + public static final String START="start"; + public static final String END="end"; + public static final String GAP="gap"; + public static final String INCLUDE_BOUNDARY="includeBoundary"; + public static final String OTHER_RANGE="otherRange"; + public static final String HARD_END="hardend"; - public final String QUERY_FACET="queryFacet"; - public final String QUERY="query"; + public static final String QUERY_FACET="queryFacet"; + public static final String QUERY="query"; // Default Values public static final int DEFAULT_FACET_LIMIT = -1; diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java index 32a0d4ad3de..27865cd5e34 100644 --- a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java @@ -66,7 +66,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows.add(createMap("id", "1", "desC", "one")); MockDataSource.setIterator("select * from x", rows.iterator()); - runFullImport(dataConfigWithCaseInsensitiveFields); + runFullImport(DATA_CONFIG_WITH_CASE_INSENSITIVE_FIELDS); assertQ(req("id:1"), "//*[@numFound='1']"); assertTrue("Start event listener was not called", StartEventListener.executed); @@ -81,7 +81,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows.add(createMap("id", "1", "FORCE_ERROR", "true")); MockDataSource.setIterator("select * from x", rows.iterator()); - runFullImport(dataConfigWithErrorHandler); + runFullImport(DATA_CONFIG_WITH_ERROR_HANDLER); assertTrue("Error event listener was not called", ErrorEventListener.executed); assertTrue(ErrorEventListener.lastException.getMessage().contains("ForcedException")); @@ -94,7 +94,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows.add(createMap("id", "1", "desc", "one")); MockDataSource.setIterator("select * from x", rows.iterator()); - runFullImport(dataConfigWithDynamicTransformer); + runFullImport(DATA_CONFIG_WITH_DYNAMIC_TRANSFORMER); assertQ(req("id:1"), "//*[@numFound='1']"); assertQ(req("dynamic_s:test"), "//*[@numFound='1']"); @@ -110,7 +110,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { LocalSolrQueryRequest request = lrf.makeRequest("command", "full-import", "debug", "on", "clean", "true", "commit", "true", "category", "search", - "dataConfig", requestParamAsVariable); + "dataConfig", REQUEST_PARAM_AS_VARIABLE); h.query("/dataimport", request); assertQ(req("desc:ApacheSolr"), "//*[@numFound='1']"); } @@ -124,7 +124,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { LocalSolrQueryRequest request = lrf.makeRequest("command", "full-import", "debug", "on", "clean", "true", "commit", "true", - "dataConfig", dataConfigWithDynamicFieldNames); + "dataConfig", DATA_CONFIG_WITH_DYNAMIC_FIELD_NAMES); h.query("/dataimport", request); assertQ(req("id:101"), "//*[@numFound='1']", "//*[@name='101_s']"); } @@ -139,7 +139,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { LocalSolrQueryRequest request = lrf.makeRequest("command", "full-import", "debug", "on", "clean", "true", "commit", "true", "mypk", "id", "text", "desc", - "dataConfig", dataConfigWithTemplatizedFieldNames); + "dataConfig", DATA_CONFIG_WITH_TEMPLATIZED_FIELD_NAMES); h.query("/dataimport", request); assertQ(req("id:101"), "//*[@numFound='1']"); } @@ -162,7 +162,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows.add(createMap("id", "2", "desc", "two", DocBuilder.SKIP_DOC, "true")); MockDataSource.setIterator("select * from x", rows.iterator()); - runFullImport(dataConfigWithDynamicTransformer); + runFullImport(DATA_CONFIG_WITH_DYNAMIC_TRANSFORMER); assertQ(req("id:1"), "//*[@numFound='1']"); assertQ(req("id:2"), "//*[@numFound='0']"); @@ -176,7 +176,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows.add(createMap("id", "2", "desc", "two", DocBuilder.SKIP_ROW, "true")); MockDataSource.setIterator("select * from x", rows.iterator()); - runFullImport(dataConfigWithDynamicTransformer); + runFullImport(DATA_CONFIG_WITH_DYNAMIC_TRANSFORMER); assertQ(req("id:1"), "//*[@numFound='1']"); assertQ(req("id:2"), "//*[@numFound='0']"); @@ -196,7 +196,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows.add(createMap("name_s", "xyz", DocBuilder.SKIP_ROW, "true")); MockDataSource.setIterator("4", rows.iterator()); - runFullImport(dataConfigWithTwoEntities); + runFullImport(DATA_CONFIG_WITH_TWO_ENTITIES); assertQ(req("id:3"), "//*[@numFound='1']"); assertQ(req("id:4"), "//*[@numFound='1']"); assertQ(req("name_s:abcd"), "//*[@numFound='1']"); @@ -211,7 +211,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows.add(createMap("id", "2", "desc", "two", "$stopTransform", "true")); MockDataSource.setIterator("select * from x", rows.iterator()); - runFullImport(dataConfigForSkipTransform); + runFullImport(DATA_CONFIG_FOR_SKIP_TRANSFORM); assertQ(req("id:1"), "//*[@numFound='1']"); assertQ(req("id:2"), "//*[@numFound='1']"); @@ -227,7 +227,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows.add(createMap("id", "3", "desc", "two", DocBuilder.DELETE_DOC_BY_ID, "2")); MockDataSource.setIterator("select * from x", rows.iterator()); - runFullImport(dataConfigForSkipTransform); + runFullImport(DATA_CONFIG_FOR_SKIP_TRANSFORM); assertQ(req("id:1"), "//*[@numFound='1']"); assertQ(req("id:2"), "//*[@numFound='0']"); @@ -243,7 +243,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows.add(createMap("id", "3", "desc", "two", DocBuilder.DELETE_DOC_BY_QUERY, "desc:one")); MockDataSource.setIterator("select * from x", rows.iterator()); - runFullImport(dataConfigForSkipTransform); + runFullImport(DATA_CONFIG_FOR_SKIP_TRANSFORM); assertQ(req("id:1"), "//*[@numFound='0']"); assertQ(req("id:2"), "//*[@numFound='0']"); @@ -256,7 +256,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { rows = new ArrayList(); rows.add(createMap(DocBuilder.DELETE_DOC_BY_ID, "3")); MockDataSource.setIterator("select * from x", rows.iterator()); - runFullImport(dataConfigForSkipTransform, createMap("clean","false")); + runFullImport(DATA_CONFIG_FOR_SKIP_TRANSFORM, createMap("clean","false")); assertQ(req("id:3"), "//*[@numFound='0']"); assertTrue("Update request processor processDelete was not called", TestUpdateRequestProcessor.processDeleteCalled); @@ -274,12 +274,12 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { createFile(tmpdir, "a.xml", "a.xml".getBytes(StandardCharsets.UTF_8), true); createFile(tmpdir, "b.xml", "b.xml".getBytes(StandardCharsets.UTF_8), true); createFile(tmpdir, "c.props", "c.props".getBytes(StandardCharsets.UTF_8), true); - runFullImport(dataConfigFileList, params); + runFullImport(DATA_CONFIG_FILE_LIST, params); assertQ(req("*:*"), "//*[@numFound='3']"); // Add a new file after a full index is done createFile(tmpdir, "t.xml", "t.xml".getBytes(StandardCharsets.UTF_8), false); - runFullImport(dataConfigFileList, params); + runFullImport(DATA_CONFIG_FILE_LIST, params); // we should find only 1 because by default clean=true is passed // and this particular import should find only one file t.xml assertQ(req("*:*"), "//*[@numFound='1']"); @@ -342,7 +342,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { } } - private final String requestParamAsVariable = "\n" + + private static final String REQUEST_PARAM_AS_VARIABLE = "\n" + " \n" + " \n" + " \n" + @@ -352,7 +352,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { " \n" + ""; - private final String dataConfigWithDynamicTransformer = " \n" + + private static final String DATA_CONFIG_WITH_DYNAMIC_TRANSFORMER = " \n" + " \n" + " \n" + @@ -362,7 +362,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { " \n" + ""; - private final String dataConfigForSkipTransform = " \n" + + private static final String DATA_CONFIG_FOR_SKIP_TRANSFORM = " \n" + " \n" + " \n" + @@ -373,7 +373,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { " \n" + ""; - private final String dataConfigWithTwoEntities = "\n" + + private static final String DATA_CONFIG_WITH_TWO_ENTITIES = "\n" + " \n" + " " + " \n" + @@ -385,7 +385,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { " \n" + ""; - private final String dataConfigWithCaseInsensitiveFields = " \n" + + private static final String DATA_CONFIG_WITH_CASE_INSENSITIVE_FIELDS = " \n" + " \n" + " \n" + " \n" + @@ -394,7 +394,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { " \n" + ""; - private final String dataConfigWithErrorHandler = " \n" + + private static final String DATA_CONFIG_WITH_ERROR_HANDLER = " \n" + " \n" + " \n" + " \n" + @@ -403,7 +403,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { " \n" + ""; - private final String dataConfigWithTemplatizedFieldNames = "\n" + + private static final String DATA_CONFIG_WITH_TEMPLATIZED_FIELD_NAMES = "\n" + " \n" + " \n" + " \n" + @@ -412,7 +412,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { " \n" + ""; - private final String dataConfigWithDynamicFieldNames = "\n" + + private static final String DATA_CONFIG_WITH_DYNAMIC_FIELD_NAMES = "\n" + " \n" + " \n" + " \n" + @@ -421,7 +421,7 @@ public class TestDocBuilder2 extends AbstractDataImportHandlerTestCase { " \n" + ""; - private final String dataConfigFileList = "\n" + + private static final String DATA_CONFIG_FILE_LIST = "\n" + "\t\n" + "\t\t childData = createMockedIterator(childName, parentData, holder); String subChildren = createChildren(childName, currentLevel + 1, maxLevel, childData, holder); - String child = StrUtils.formatString(childEntityTemplate, childName, select, fields, subChildren); + String child = StrUtils.formatString(CHILD_ENTITY_TEMPLATE, childName, select, fields, subChildren); builder.append(child); builder.append('\n'); } @@ -414,7 +414,7 @@ public class TestHierarchicalDocBuilder extends AbstractDataImportHandlerTestCas return builder.toString(); } - private final String threeLevelHierarchyConfig = "\n" + + private static final String THREE_LEVEL_HIERARCHY_CONFIG = "\n" + " \n" + " \n" + " \n" + @@ -436,7 +436,7 @@ public class TestHierarchicalDocBuilder extends AbstractDataImportHandlerTestCas ""; /** {0} is rootEntity block **/ - private final String dataConfigTemplate = "\n\n {0}"; + private static final String DATA_CONFIG_TEMPLATE = "\n\n {0}"; /** * {0} - entityName, @@ -444,7 +444,7 @@ public class TestHierarchicalDocBuilder extends AbstractDataImportHandlerTestCas * {2} - fieldsList * {3} - childEntitiesList **/ - private final String rootEntityTemplate = "\n{2} {3}\n\n"; + private static final String ROOT_ENTITY_TEMPLATE = "\n{2} {3}\n\n"; /** * {0} - entityName, @@ -452,7 +452,7 @@ public class TestHierarchicalDocBuilder extends AbstractDataImportHandlerTestCas * {2} - fieldsList * {3} - childEntitiesList **/ - private final String childEntityTemplate = "\n {2} {3} \n"; + private static final String CHILD_ENTITY_TEMPLATE = "\n {2} {3} \n"; private BitSetProducer createParentFilter(String type) { BooleanQuery.Builder parentQuery = new BooleanQuery.Builder(); diff --git a/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java b/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java index 6e3ae1e304d..7518208fb34 100644 --- a/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java +++ b/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java @@ -36,7 +36,7 @@ public class DistributedMap { protected SolrZkClient zookeeper; - protected final String prefix = "mn-"; + protected static final String PREFIX = "mn-"; public DistributedMap(SolrZkClient zookeeper, String dir) { this.dir = dir; @@ -56,15 +56,15 @@ public class DistributedMap { public void put(String trackingId, byte[] data) throws KeeperException, InterruptedException { - zookeeper.makePath(dir + "/" + prefix + trackingId, data, CreateMode.PERSISTENT, null, false, true); + zookeeper.makePath(dir + "/" + PREFIX + trackingId, data, CreateMode.PERSISTENT, null, false, true); } public byte[] get(String trackingId) throws KeeperException, InterruptedException { - return zookeeper.getData(dir + "/" + prefix + trackingId, null, null, true); + return zookeeper.getData(dir + "/" + PREFIX + trackingId, null, null, true); } public boolean contains(String trackingId) throws KeeperException, InterruptedException { - return zookeeper.exists(dir + "/" + prefix + trackingId, true); + return zookeeper.exists(dir + "/" + PREFIX + trackingId, true); } public int size() throws KeeperException, InterruptedException { @@ -80,7 +80,7 @@ public class DistributedMap { */ public boolean remove(String trackingId) throws KeeperException, InterruptedException { try { - zookeeper.delete(dir + "/" + prefix + trackingId, -1, true); + zookeeper.delete(dir + "/" + PREFIX + trackingId, -1, true); } catch (KeeperException.NoNodeException e) { return false; } diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java index e031303ad78..92e34cfe498 100644 --- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java +++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java @@ -42,7 +42,7 @@ import org.slf4j.LoggerFactory; public class OverseerTaskQueue extends DistributedQueue { private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private final String response_prefix = "qnr-" ; + private static final String RESPONSE_PREFIX = "qnr-" ; public OverseerTaskQueue(SolrZkClient zookeeper, String dir) { this(zookeeper, dir, new Overseer.Stats()); @@ -88,7 +88,7 @@ public class OverseerTaskQueue extends DistributedQueue { Timer.Context time = stats.time(dir + "_remove_event"); try { String path = event.getId(); - String responsePath = dir + "/" + response_prefix + String responsePath = dir + "/" + RESPONSE_PREFIX + path.substring(path.lastIndexOf("-") + 1); if (zookeeper.exists(responsePath, true)) { zookeeper.setData(responsePath, event.getBytes(), true); @@ -217,7 +217,7 @@ public class OverseerTaskQueue extends DistributedQueue { String createResponseNode() throws KeeperException, InterruptedException { return createData( - dir + "/" + response_prefix, + dir + "/" + RESPONSE_PREFIX, null, CreateMode.EPHEMERAL_SEQUENTIAL); } diff --git a/solr/core/src/java/org/apache/solr/schema/CurrencyField.java b/solr/core/src/java/org/apache/solr/schema/CurrencyField.java index 9e994cfcaed..7b27c3f46c8 100644 --- a/solr/core/src/java/org/apache/solr/schema/CurrencyField.java +++ b/solr/core/src/java/org/apache/solr/schema/CurrencyField.java @@ -504,7 +504,7 @@ public class CurrencyField extends FieldType implements SchemaAware, ResourceLoa final FunctionValues currencies = currencyValues.getValues(context, reader); return new FunctionValues() { - private final int MAX_CURRENCIES_TO_CACHE = 256; + private static final int MAX_CURRENCIES_TO_CACHE = 256; private final int[] fractionDigitCache = new int[MAX_CURRENCIES_TO_CACHE]; private final String[] currencyOrdToCurrencyCache = new String[MAX_CURRENCIES_TO_CACHE]; private final double[] exchangeRateCache = new double[MAX_CURRENCIES_TO_CACHE]; 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 9c09ebeb3fd..6cf75044bec 100644 --- a/solr/core/src/java/org/apache/solr/update/CommitTracker.java +++ b/solr/core/src/java/org/apache/solr/update/CommitTracker.java @@ -48,7 +48,7 @@ public final class CommitTracker implements Runnable { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); // scheduler delay for maxDoc-triggered autocommits - public final int DOC_COMMIT_DELAY_MS = 1; + public static final int DOC_COMMIT_DELAY_MS = 1; // settings, not final so we can change them in testing private int docsUpperBound; @@ -66,7 +66,7 @@ public final class CommitTracker implements Runnable { private final boolean softCommit; private boolean openSearcher; - private final boolean waitSearcher = true; + private static final boolean WAIT_SEARCHER = true; private String name; @@ -205,7 +205,7 @@ public final class CommitTracker implements Runnable { try { CommitUpdateCommand command = new CommitUpdateCommand(req, false); command.openSearcher = openSearcher; - command.waitSearcher = waitSearcher; + command.waitSearcher = WAIT_SEARCHER; command.softCommit = softCommit; if (core.getCoreDescriptor().getCloudDescriptor() != null && core.getCoreDescriptor().getCloudDescriptor().isLeader() 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 e64208936f5..6e18cf938a3 100644 --- a/solr/core/src/java/org/apache/solr/util/SimplePostTool.java +++ b/solr/core/src/java/org/apache/solr/util/SimplePostTool.java @@ -1105,7 +1105,7 @@ public class SimplePostTool { // class PageFetcher { Map> robotsCache; - final String DISALLOW = "Disallow:"; + static final String DISALLOW = "Disallow:"; public PageFetcher() { robotsCache = new HashMap<>(); diff --git a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java index dccc2c6b064..b0721a2b8e0 100644 --- a/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/OverseerTest.java @@ -81,8 +81,8 @@ public class OverseerTest extends SolrTestCaseJ4 { private List readers = new ArrayList<>(); private List httpShardHandlerFactorys = new ArrayList<>(); private List updateShardHandlers = new ArrayList<>(); - - final private String collection = SolrTestCaseJ4.DEFAULT_TEST_COLLECTION_NAME; + + private static final String COLLECTION = SolrTestCaseJ4.DEFAULT_TEST_COLLECTION_NAME; public static class MockZKController{ @@ -271,17 +271,17 @@ public class OverseerTest extends SolrTestCaseJ4 { final int numShards=6; for (int i = 0; i < numShards; i++) { - assertNotNull("shard got no id?", zkController.publishState(collection, "core" + (i+1), "node" + (i+1), Replica.State.ACTIVE, 3)); + assertNotNull("shard got no id?", zkController.publishState(COLLECTION, "core" + (i+1), "node" + (i+1), Replica.State.ACTIVE, 3)); } - final Map rmap = reader.getClusterState().getSlice(collection, "shard1").getReplicasMap(); + final Map rmap = reader.getClusterState().getSlice(COLLECTION, "shard1").getReplicasMap(); assertEquals(rmap.toString(), 2, rmap.size()); - assertEquals(rmap.toString(), 2, reader.getClusterState().getSlice(collection, "shard2").getReplicasMap().size()); - assertEquals(rmap.toString(), 2, reader.getClusterState().getSlice(collection, "shard3").getReplicasMap().size()); + assertEquals(rmap.toString(), 2, reader.getClusterState().getSlice(COLLECTION, "shard2").getReplicasMap().size()); + assertEquals(rmap.toString(), 2, reader.getClusterState().getSlice(COLLECTION, "shard3").getReplicasMap().size()); //make sure leaders are in cloud state - assertNotNull(reader.getLeaderUrl(collection, "shard1", 15000)); - assertNotNull(reader.getLeaderUrl(collection, "shard2", 15000)); - assertNotNull(reader.getLeaderUrl(collection, "shard3", 15000)); + assertNotNull(reader.getLeaderUrl(COLLECTION, "shard1", 15000)); + assertNotNull(reader.getLeaderUrl(COLLECTION, "shard2", 15000)); + assertNotNull(reader.getLeaderUrl(COLLECTION, "shard3", 15000)); } finally { close(zkClient); @@ -321,17 +321,17 @@ public class OverseerTest extends SolrTestCaseJ4 { final int numShards=3; for (int i = 0; i < numShards; i++) { - assertNotNull("shard got no id?", zkController.publishState(collection, "core" + (i+1), "node" + (i+1), Replica.State.ACTIVE, 3)); + assertNotNull("shard got no id?", zkController.publishState(COLLECTION, "core" + (i+1), "node" + (i+1), Replica.State.ACTIVE, 3)); } - assertEquals(1, reader.getClusterState().getSlice(collection, "shard1").getReplicasMap().size()); - assertEquals(1, reader.getClusterState().getSlice(collection, "shard2").getReplicasMap().size()); - assertEquals(1, reader.getClusterState().getSlice(collection, "shard3").getReplicasMap().size()); + assertEquals(1, reader.getClusterState().getSlice(COLLECTION, "shard1").getReplicasMap().size()); + assertEquals(1, reader.getClusterState().getSlice(COLLECTION, "shard2").getReplicasMap().size()); + assertEquals(1, reader.getClusterState().getSlice(COLLECTION, "shard3").getReplicasMap().size()); //make sure leaders are in cloud state - assertNotNull(reader.getLeaderUrl(collection, "shard1", 15000)); - assertNotNull(reader.getLeaderUrl(collection, "shard2", 15000)); - assertNotNull(reader.getLeaderUrl(collection, "shard3", 15000)); + assertNotNull(reader.getLeaderUrl(COLLECTION, "shard1", 15000)); + assertNotNull(reader.getLeaderUrl(COLLECTION, "shard2", 15000)); + assertNotNull(reader.getLeaderUrl(COLLECTION, "shard3", 15000)); // publish a bad queue item String emptyCollectionName = ""; @@ -408,7 +408,7 @@ public class OverseerTest extends SolrTestCaseJ4 { final String coreName = "core" + slot; try { - ids[slot] = controllers[slot % nodeCount].publishState(collection, coreName, "node" + slot, Replica.State.ACTIVE, sliceCount); + ids[slot] = controllers[slot % nodeCount].publishState(COLLECTION, coreName, "node" + slot, Replica.State.ACTIVE, sliceCount); } catch (Throwable e) { e.printStackTrace(); fail("register threw exception:" + e.getClass()); @@ -429,7 +429,7 @@ public class OverseerTest extends SolrTestCaseJ4 { for (int i = 0; i < 40; i++) { cloudStateSliceCount = 0; ClusterState state = reader.getClusterState(); - final Map slices = state.getSlicesMap(collection); + final Map slices = state.getSlicesMap(COLLECTION); if (slices != null) { for (String name : slices.keySet()) { cloudStateSliceCount += slices.get(name).getReplicasMap().size(); @@ -483,7 +483,7 @@ public class OverseerTest extends SolrTestCaseJ4 { //make sure leaders are in cloud state for (int i = 0; i < sliceCount; i++) { - assertNotNull(reader.getLeaderUrl(collection, "shard" + (i + 1), 15000)); + assertNotNull(reader.getLeaderUrl(COLLECTION, "shard" + (i + 1), 15000)); } } finally { @@ -549,23 +549,23 @@ public class OverseerTest extends SolrTestCaseJ4 { ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(), ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr", ZkStateReader.NODE_NAME_PROP, "node1", - ZkStateReader.COLLECTION_PROP, collection, + ZkStateReader.COLLECTION_PROP, COLLECTION, ZkStateReader.CORE_NAME_PROP, "core1", ZkStateReader.ROLES_PROP, "", ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString()); q.offer(Utils.toJSON(m)); - waitForCollections(reader, collection); + waitForCollections(reader, COLLECTION); assertSame(reader.getClusterState().toString(), Replica.State.RECOVERING, - reader.getClusterState().getSlice(collection, "shard1").getReplica("core_node1").getState()); + reader.getClusterState().getSlice(COLLECTION, "shard1").getReplica("core_node1").getState()); //publish node state (active) m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(), ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr", ZkStateReader.NODE_NAME_PROP, "node1", - ZkStateReader.COLLECTION_PROP, collection, + ZkStateReader.COLLECTION_PROP, COLLECTION, ZkStateReader.CORE_NAME_PROP, "core1", ZkStateReader.ROLES_PROP, "", ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString()); @@ -634,25 +634,25 @@ public class OverseerTest extends SolrTestCaseJ4 { overseerClient = electNewOverseer(server.getZkAddress()); Thread.sleep(1000); - mockController.publishState(collection, core, core_node, + mockController.publishState(COLLECTION, core, core_node, Replica.State.RECOVERING, numShards); - waitForCollections(reader, collection); - verifyReplicaStatus(reader, collection, "shard1", "core_node1", Replica.State.RECOVERING); + waitForCollections(reader, COLLECTION); + verifyReplicaStatus(reader, COLLECTION, "shard1", "core_node1", Replica.State.RECOVERING); int version = getClusterStateVersion(zkClient); - mockController.publishState(collection, core, core_node, Replica.State.ACTIVE, + mockController.publishState(COLLECTION, core, core_node, Replica.State.ACTIVE, numShards); while (version == getClusterStateVersion(zkClient)); - verifyReplicaStatus(reader, collection, "shard1", "core_node1", Replica.State.ACTIVE); + verifyReplicaStatus(reader, COLLECTION, "shard1", "core_node1", Replica.State.ACTIVE); version = getClusterStateVersion(zkClient); overseerClient.close(); Thread.sleep(1000); // wait for overseer to get killed - mockController.publishState(collection, core, core_node, + mockController.publishState(COLLECTION, core, core_node, Replica.State.RECOVERING, numShards); version = getClusterStateVersion(zkClient); @@ -660,20 +660,20 @@ public class OverseerTest extends SolrTestCaseJ4 { while (version == getClusterStateVersion(zkClient)); - verifyReplicaStatus(reader, collection, "shard1", "core_node1", Replica.State.RECOVERING); + verifyReplicaStatus(reader, COLLECTION, "shard1", "core_node1", Replica.State.RECOVERING); assertEquals("Live nodes count does not match", 1, reader .getClusterState().getLiveNodes().size()); assertEquals(shard+" replica count does not match", 1, reader.getClusterState() - .getSlice(collection, shard).getReplicasMap().size()); + .getSlice(COLLECTION, shard).getReplicasMap().size()); version = getClusterStateVersion(zkClient); - mockController.publishState(collection, core, core_node, null, numShards); + mockController.publishState(COLLECTION, core, core_node, null, numShards); while (version == getClusterStateVersion(zkClient)); Thread.sleep(500); - assertTrue(collection+" should remain after removal of the last core", // as of SOLR-5209 core removal does not cascade to remove the slice and collection - reader.getClusterState().hasCollection(collection)); + assertTrue(COLLECTION +" should remain after removal of the last core", // as of SOLR-5209 core removal does not cascade to remove the slice and collection + reader.getClusterState().hasCollection(COLLECTION)); assertTrue(core_node+" should be gone after publishing the null state", - null == reader.getClusterState().getCollection(collection).getReplica(core_node)); + null == reader.getClusterState().getCollection(COLLECTION).getReplica(core_node)); } finally { close(mockController); close(overseerClient); @@ -723,7 +723,7 @@ public class OverseerTest extends SolrTestCaseJ4 { overseerElector.setup(ec); overseerElector.joinElection(ec, false); - mockController.publishState(collection, "core1", "core_node1", Replica.State.ACTIVE, 1); + mockController.publishState(COLLECTION, "core1", "core_node1", Replica.State.ACTIVE, 1); assertNotNull(overseer.getStats()); assertTrue((overseer.getStats().getSuccessCount(OverseerAction.STATE.toLower())) > 0); @@ -819,19 +819,19 @@ public class OverseerTest extends SolrTestCaseJ4 { for (int i = 0; i < atLeast(4); i++) { killCounter.incrementAndGet(); //for each round allow 1 kill mockController = new MockZKController(server.getZkAddress(), "node1"); - mockController.publishState(collection, "core1", "node1", Replica.State.ACTIVE,1); + mockController.publishState(COLLECTION, "core1", "node1", Replica.State.ACTIVE,1); if(mockController2!=null) { mockController2.close(); mockController2 = null; } - mockController.publishState(collection, "core1", "node1",Replica.State.RECOVERING,1); + mockController.publishState(COLLECTION, "core1", "node1",Replica.State.RECOVERING,1); mockController2 = new MockZKController(server.getZkAddress(), "node2"); - mockController.publishState(collection, "core1", "node1", Replica.State.ACTIVE,1); - verifyShardLeader(reader, collection, "shard1", "core1"); - mockController2.publishState(collection, "core4", "node2", Replica.State.ACTIVE ,1); + mockController.publishState(COLLECTION, "core1", "node1", Replica.State.ACTIVE,1); + verifyShardLeader(reader, COLLECTION, "shard1", "core1"); + mockController2.publishState(COLLECTION, "core4", "node2", Replica.State.ACTIVE ,1); mockController.close(); mockController = null; - verifyShardLeader(reader, collection, "shard1", "core4"); + verifyShardLeader(reader, COLLECTION, "shard1", "core4"); } } finally { if (killer != null) { @@ -874,18 +874,18 @@ public class OverseerTest extends SolrTestCaseJ4 { overseerClient = electNewOverseer(server.getZkAddress()); - mockController.publishState(collection, "core1", "core_node1", Replica.State.RECOVERING, 1); + mockController.publishState(COLLECTION, "core1", "core_node1", Replica.State.RECOVERING, 1); waitForCollections(reader, "collection1"); - verifyReplicaStatus(reader, collection, "shard1", "core_node1", Replica.State.RECOVERING); + verifyReplicaStatus(reader, COLLECTION, "shard1", "core_node1", Replica.State.RECOVERING); mockController.close(); int version = getClusterStateVersion(controllerClient); mockController = new MockZKController(server.getZkAddress(), "node1"); - mockController.publishState(collection, "core1", "core_node1", Replica.State.RECOVERING, 1); + mockController.publishState(COLLECTION, "core1", "core_node1", Replica.State.RECOVERING, 1); while (version == reader.getClusterState().getZkClusterStateVersion()) { Thread.sleep(100); @@ -940,11 +940,11 @@ public class OverseerTest extends SolrTestCaseJ4 { overseerClient = electNewOverseer(server.getZkAddress()); - mockController.publishState(collection, "core1", "node1", Replica.State.RECOVERING, 12); + mockController.publishState(COLLECTION, "core1", "node1", Replica.State.RECOVERING, 12); - waitForCollections(reader, collection); + waitForCollections(reader, COLLECTION); - assertEquals("Slicecount does not match", 12, reader.getClusterState().getSlices(collection).size()); + assertEquals("Slicecount does not match", 12, reader.getClusterState().getSlices(COLLECTION).size()); } finally { close(overseerClient); @@ -1117,7 +1117,7 @@ public class OverseerTest extends SolrTestCaseJ4 { ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr", ZkStateReader.NODE_NAME_PROP, "node1", ZkStateReader.SHARD_ID_PROP, "s1", - ZkStateReader.COLLECTION_PROP, collection, + ZkStateReader.COLLECTION_PROP, COLLECTION, ZkStateReader.CORE_NAME_PROP, "core1", ZkStateReader.ROLES_PROP, "", ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString()); @@ -1126,7 +1126,7 @@ public class OverseerTest extends SolrTestCaseJ4 { ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr", ZkStateReader.NODE_NAME_PROP, "node1", ZkStateReader.SHARD_ID_PROP, "s1", - ZkStateReader.COLLECTION_PROP, collection, + ZkStateReader.COLLECTION_PROP, COLLECTION, ZkStateReader.CORE_NAME_PROP, "core2", ZkStateReader.ROLES_PROP, "", ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString()); @@ -1140,19 +1140,19 @@ public class OverseerTest extends SolrTestCaseJ4 { ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr", ZkStateReader.NODE_NAME_PROP, "node1", ZkStateReader.SHARD_ID_PROP, "s1", - ZkStateReader.COLLECTION_PROP, collection, + ZkStateReader.COLLECTION_PROP, COLLECTION, ZkStateReader.CORE_NAME_PROP, "core3", ZkStateReader.ROLES_PROP, "", ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString()); queue.offer(Utils.toJSON(m)); for(int i=0;i<100;i++) { - Slice s = reader.getClusterState().getSlice(collection, "s1"); + Slice s = reader.getClusterState().getSlice(COLLECTION, "s1"); if(s!=null && s.getReplicasMap().size()==3) break; Thread.sleep(100); } - assertNotNull(reader.getClusterState().getSlice(collection, "s1")); - assertEquals(3, reader.getClusterState().getSlice(collection, "s1").getReplicasMap().size()); + assertNotNull(reader.getClusterState().getSlice(COLLECTION, "s1")); + assertEquals(3, reader.getClusterState().getSlice(COLLECTION, "s1").getReplicasMap().size()); } finally { close(overseerClient); close(zkClient); @@ -1340,14 +1340,14 @@ public class OverseerTest extends SolrTestCaseJ4 { { final Integer maxShardsPerNode = numReplicas * numShards; ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.CREATE.toLower(), - "name", collection, + "name", COLLECTION, ZkStateReader.NUM_SHARDS_PROP, numShards.toString(), ZkStateReader.REPLICATION_FACTOR, "1", ZkStateReader.MAX_SHARDS_PER_NODE, maxShardsPerNode.toString() ); q.offer(Utils.toJSON(m)); } - waitForCollections(zkStateReader, collection); + waitForCollections(zkStateReader, COLLECTION); // create nodes with state recovering for (int rr = 1; rr <= numReplicas; ++rr) { @@ -1357,7 +1357,7 @@ public class OverseerTest extends SolrTestCaseJ4 { ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr", ZkStateReader.SHARD_ID_PROP, "shard"+ss, ZkStateReader.NODE_NAME_PROP, "node"+N, - ZkStateReader.COLLECTION_PROP, collection, + ZkStateReader.COLLECTION_PROP, COLLECTION, ZkStateReader.CORE_NAME_PROP, "core"+N, ZkStateReader.ROLES_PROP, "", ZkStateReader.STATE_PROP, Replica.State.RECOVERING.toString()); @@ -1369,7 +1369,7 @@ public class OverseerTest extends SolrTestCaseJ4 { for (int rr = 1; rr <= numReplicas; ++rr) { for (int ss = 1; ss <= numShards; ++ss) { final int N = (numReplicas-rr)*numShards + ss; - verifyReplicaStatus(zkStateReader, collection, "shard"+ss, "core_node"+N, Replica.State.RECOVERING); + verifyReplicaStatus(zkStateReader, COLLECTION, "shard"+ss, "core_node"+N, Replica.State.RECOVERING); } } @@ -1380,7 +1380,7 @@ public class OverseerTest extends SolrTestCaseJ4 { ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.STATE.toLower(), ZkStateReader.BASE_URL_PROP, "http://127.0.0.1/solr", ZkStateReader.NODE_NAME_PROP, "node"+N, - ZkStateReader.COLLECTION_PROP, collection, + ZkStateReader.COLLECTION_PROP, COLLECTION, ZkStateReader.CORE_NAME_PROP, "core"+N, ZkStateReader.ROLES_PROP, "", ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString()); @@ -1392,7 +1392,7 @@ public class OverseerTest extends SolrTestCaseJ4 { for (int rr = 1; rr <= numReplicas; ++rr) { for (int ss = 1; ss <= numShards; ++ss) { final int N = (numReplicas-rr)*numShards + ss; - verifyReplicaStatus(zkStateReader, collection, "shard"+ss, "core_node"+N, Replica.State.ACTIVE); + verifyReplicaStatus(zkStateReader, COLLECTION, "shard"+ss, "core_node"+N, Replica.State.ACTIVE); } } @@ -1401,7 +1401,7 @@ public class OverseerTest extends SolrTestCaseJ4 { for (int ss = 1; ss <= numShards; ++ss) { final int N = (numReplicas-rr)*numShards + ss; ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(), - ZkStateReader.COLLECTION_PROP, collection, + ZkStateReader.COLLECTION_PROP, COLLECTION, ZkStateReader.CORE_NODE_NAME_PROP, "core_node"+N); q.offer(Utils.toJSON(m)); @@ -1409,23 +1409,23 @@ public class OverseerTest extends SolrTestCaseJ4 { { int iterationsLeft = 100; while (iterationsLeft-- > 0) { - final Slice slice = zkStateReader.getClusterState().getSlice(collection, "shard"+ss); + final Slice slice = zkStateReader.getClusterState().getSlice(COLLECTION, "shard"+ss); if (null == slice || null == slice.getReplicasMap().get("core_node"+N)) { break; } - if (VERBOSE) log.info("still seeing {} shard{} core_node{}, rechecking in 50ms ({} iterations left)", collection, ss, N, iterationsLeft); + if (VERBOSE) log.info("still seeing {} shard{} core_node{}, rechecking in 50ms ({} iterations left)", COLLECTION, ss, N, iterationsLeft); Thread.sleep(50); } } - final DocCollection docCollection = zkStateReader.getClusterState().getCollection(collection); - assertTrue("found no "+collection, (null != docCollection)); + final DocCollection docCollection = zkStateReader.getClusterState().getCollection(COLLECTION); + assertTrue("found no "+ COLLECTION, (null != docCollection)); final Slice slice = docCollection.getSlice("shard"+ss); - assertTrue("found no "+collection+" shard"+ss+" slice after removal of replica "+rr+" of "+numReplicas, (null != slice)); + assertTrue("found no "+ COLLECTION +" shard"+ss+" slice after removal of replica "+rr+" of "+numReplicas, (null != slice)); final Collection replicas = slice.getReplicas(); - assertEquals("wrong number of "+collection+" shard"+ss+" replicas left, replicas="+replicas, numReplicas-rr, replicas.size()); + assertEquals("wrong number of "+ COLLECTION +" shard"+ss+" replicas left, replicas="+replicas, numReplicas-rr, replicas.size()); } } diff --git a/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java b/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java index 3fe12ed8bf8..9824e3da4ac 100644 --- a/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java +++ b/solr/core/src/test/org/apache/solr/cloud/SegmentTerminateEarlyTestState.java @@ -34,14 +34,14 @@ import org.apache.solr.response.SolrQueryResponse; class SegmentTerminateEarlyTestState { - final String keyField = "id"; + static final String KEY_FIELD = "id"; // for historic reasons, this is refered to as a "timestamp" field, but in actuallity is just an int // value representing a number of "minutes" between 0-60. // aka: I decided not to rename a million things while refactoring this test - public static final String timestampField = "timestamp_i_dvo"; - public static final String oddField = "odd_l1"; // - public static final String quadField = "quad_l1"; // + public static final String TIMESTAMP_FIELD = "timestamp_i_dvo"; + public static final String ODD_FIELD = "odd_l1"; // + public static final String QUAD_FIELD = "quad_l1"; // final Set minTimestampDocKeys = new HashSet<>(); final Set maxTimestampDocKeys = new HashSet<>(); @@ -63,7 +63,7 @@ class SegmentTerminateEarlyTestState { ++numDocs; final Integer docKey = new Integer(numDocs); SolrInputDocument doc = new SolrInputDocument(); - doc.setField(keyField, ""+docKey); + doc.setField(KEY_FIELD, ""+docKey); final int MM = rand.nextInt(60); // minutes if (minTimestampMM == null || MM <= minTimestampMM.intValue()) { if (minTimestampMM != null && MM < minTimestampMM.intValue()) { @@ -79,9 +79,9 @@ class SegmentTerminateEarlyTestState { maxTimestampMM = new Integer(MM); maxTimestampDocKeys.add(docKey); } - doc.setField(timestampField, (Integer)MM); - doc.setField(oddField, ""+(numDocs % 2)); - doc.setField(quadField, ""+(numDocs % 4)+1); + doc.setField(TIMESTAMP_FIELD, (Integer)MM); + doc.setField(ODD_FIELD, ""+(numDocs % 2)); + doc.setField(QUAD_FIELD, ""+(numDocs % 4)+1); cloudSolrClient.add(doc); } cloudSolrClient.commit(); @@ -95,9 +95,9 @@ class SegmentTerminateEarlyTestState { TestMiniSolrCloudCluster.assertFalse(maxTimestampDocKeys.isEmpty()); TestMiniSolrCloudCluster.assertTrue("numDocs="+numDocs+" is not even", (numDocs%2)==0); final Long oddFieldValue = new Long(maxTimestampDocKeys.iterator().next().intValue()%2); - final SolrQuery query = new SolrQuery(oddField+":"+oddFieldValue); - query.setSort(timestampField, SolrQuery.ORDER.desc); - query.setFields(keyField, oddField, timestampField); + final SolrQuery query = new SolrQuery(ODD_FIELD +":"+oddFieldValue); + query.setSort(TIMESTAMP_FIELD, SolrQuery.ORDER.desc); + query.setFields(KEY_FIELD, ODD_FIELD, TIMESTAMP_FIELD); query.setRows(1); // CommonParams.SEGMENT_TERMINATE_EARLY parameter intentionally absent final QueryResponse rsp = cloudSolrClient.query(query); @@ -106,9 +106,9 @@ class SegmentTerminateEarlyTestState { // check correctness of the first result if (rsp.getResults().getNumFound() > 0) { final SolrDocument solrDocument0 = rsp.getResults().get(0); - TestMiniSolrCloudCluster.assertTrue(keyField+" of ("+solrDocument0+") is not in maxTimestampDocKeys("+maxTimestampDocKeys+")", - maxTimestampDocKeys.contains(solrDocument0.getFieldValue(keyField))); - TestMiniSolrCloudCluster.assertEquals(oddField, oddFieldValue, solrDocument0.getFieldValue(oddField)); + TestMiniSolrCloudCluster.assertTrue(KEY_FIELD +" of ("+solrDocument0+") is not in maxTimestampDocKeys("+maxTimestampDocKeys+")", + maxTimestampDocKeys.contains(solrDocument0.getFieldValue(KEY_FIELD))); + TestMiniSolrCloudCluster.assertEquals(ODD_FIELD, oddFieldValue, solrDocument0.getFieldValue(ODD_FIELD)); } // check segmentTerminatedEarly flag TestMiniSolrCloudCluster.assertNull("responseHeader.segmentTerminatedEarly present in "+rsp.getResponseHeader(), @@ -119,9 +119,9 @@ class SegmentTerminateEarlyTestState { TestMiniSolrCloudCluster.assertFalse(maxTimestampDocKeys.isEmpty()); TestMiniSolrCloudCluster.assertTrue("numDocs="+numDocs+" is not even", (numDocs%2)==0); final Long oddFieldValue = new Long(maxTimestampDocKeys.iterator().next().intValue()%2); - final SolrQuery query = new SolrQuery(oddField+":"+oddFieldValue); - query.setSort(timestampField, SolrQuery.ORDER.desc); - query.setFields(keyField, oddField, timestampField); + final SolrQuery query = new SolrQuery(ODD_FIELD +":"+oddFieldValue); + query.setSort(TIMESTAMP_FIELD, SolrQuery.ORDER.desc); + query.setFields(KEY_FIELD, ODD_FIELD, TIMESTAMP_FIELD); final int rowsWanted = 1; query.setRows(rowsWanted); final Boolean shardsInfoWanted = (rand.nextBoolean() ? null : new Boolean(rand.nextBoolean())); @@ -136,9 +136,9 @@ class SegmentTerminateEarlyTestState { // check correctness of the first result if (rsp.getResults().getNumFound() > 0) { final SolrDocument solrDocument0 = rsp.getResults().get(0); - TestMiniSolrCloudCluster.assertTrue(keyField+" of ("+solrDocument0+") is not in maxTimestampDocKeys("+maxTimestampDocKeys+")", - maxTimestampDocKeys.contains(solrDocument0.getFieldValue(keyField))); - TestMiniSolrCloudCluster.assertEquals(oddField, oddFieldValue, rsp.getResults().get(0).getFieldValue(oddField)); + TestMiniSolrCloudCluster.assertTrue(KEY_FIELD +" of ("+solrDocument0+") is not in maxTimestampDocKeys("+maxTimestampDocKeys+")", + maxTimestampDocKeys.contains(solrDocument0.getFieldValue(KEY_FIELD))); + TestMiniSolrCloudCluster.assertEquals(ODD_FIELD, oddFieldValue, rsp.getResults().get(0).getFieldValue(ODD_FIELD)); } // check segmentTerminatedEarly flag TestMiniSolrCloudCluster.assertNotNull("responseHeader.segmentTerminatedEarly missing in "+rsp.getResponseHeader(), @@ -167,9 +167,9 @@ class SegmentTerminateEarlyTestState { TestMiniSolrCloudCluster.assertFalse(maxTimestampDocKeys.isEmpty()); TestMiniSolrCloudCluster.assertTrue("numDocs="+numDocs+" is not even", (numDocs%2)==0); final Long oddFieldValue = new Long(maxTimestampDocKeys.iterator().next().intValue()%2); - final SolrQuery query = new SolrQuery(oddField+":"+oddFieldValue); - query.setSort(timestampField, SolrQuery.ORDER.desc); - query.setFields(keyField, oddField, timestampField); + final SolrQuery query = new SolrQuery(ODD_FIELD +":"+oddFieldValue); + query.setSort(TIMESTAMP_FIELD, SolrQuery.ORDER.desc); + query.setFields(KEY_FIELD, ODD_FIELD, TIMESTAMP_FIELD); query.setRows(1); final Boolean shardsInfoWanted = (rand.nextBoolean() ? null : new Boolean(rand.nextBoolean())); if (shardsInfoWanted != null) { @@ -182,9 +182,9 @@ class SegmentTerminateEarlyTestState { // check correctness of the first result if (rsp.getResults().getNumFound() > 0) { final SolrDocument solrDocument0 = rsp.getResults().get(0); - TestMiniSolrCloudCluster.assertTrue(keyField+" of ("+solrDocument0+") is not in maxTimestampDocKeys("+maxTimestampDocKeys+")", - maxTimestampDocKeys.contains(solrDocument0.getFieldValue(keyField))); - TestMiniSolrCloudCluster.assertEquals(oddField, oddFieldValue, rsp.getResults().get(0).getFieldValue(oddField)); + TestMiniSolrCloudCluster.assertTrue(KEY_FIELD +" of ("+solrDocument0+") is not in maxTimestampDocKeys("+maxTimestampDocKeys+")", + maxTimestampDocKeys.contains(solrDocument0.getFieldValue(KEY_FIELD))); + TestMiniSolrCloudCluster.assertEquals(ODD_FIELD, oddFieldValue, rsp.getResults().get(0).getFieldValue(ODD_FIELD)); } // check segmentTerminatedEarly flag TestMiniSolrCloudCluster.assertNull("responseHeader.segmentTerminatedEarly present in "+rsp.getResponseHeader(), @@ -212,13 +212,13 @@ class SegmentTerminateEarlyTestState { TestMiniSolrCloudCluster.assertFalse(maxTimestampDocKeys.isEmpty()); TestMiniSolrCloudCluster.assertTrue("numDocs="+numDocs+" is not even", (numDocs%2)==0); final Long oddFieldValue = new Long(maxTimestampDocKeys.iterator().next().intValue()%2); - final SolrQuery query = new SolrQuery(oddField+":"+oddFieldValue); - query.setSort(timestampField, SolrQuery.ORDER.desc); - query.setFields(keyField, oddField, timestampField); + final SolrQuery query = new SolrQuery(ODD_FIELD +":"+oddFieldValue); + query.setSort(TIMESTAMP_FIELD, SolrQuery.ORDER.desc); + query.setFields(KEY_FIELD, ODD_FIELD, TIMESTAMP_FIELD); query.setRows(1); query.set(CommonParams.SEGMENT_TERMINATE_EARLY, true); TestMiniSolrCloudCluster.assertTrue("numDocs="+numDocs+" is not quad-able", (numDocs%4)==0); - query.add("group.field", quadField); + query.add("group.field", QUAD_FIELD); query.set("group", true); final QueryResponse rsp = cloudSolrClient.query(query); // check correctness of the results count @@ -226,9 +226,9 @@ class SegmentTerminateEarlyTestState { // check correctness of the first result if (rsp.getGroupResponse().getValues().get(0).getMatches() > 0) { final SolrDocument solrDocument = rsp.getGroupResponse().getValues().get(0).getValues().get(0).getResult().get(0); - TestMiniSolrCloudCluster.assertTrue(keyField+" of ("+solrDocument+") is not in maxTimestampDocKeys("+maxTimestampDocKeys+")", - maxTimestampDocKeys.contains(solrDocument.getFieldValue(keyField))); - TestMiniSolrCloudCluster.assertEquals(oddField, oddFieldValue, solrDocument.getFieldValue(oddField)); + TestMiniSolrCloudCluster.assertTrue(KEY_FIELD +" of ("+solrDocument+") is not in maxTimestampDocKeys("+maxTimestampDocKeys+")", + maxTimestampDocKeys.contains(solrDocument.getFieldValue(KEY_FIELD))); + TestMiniSolrCloudCluster.assertEquals(ODD_FIELD, oddFieldValue, solrDocument.getFieldValue(ODD_FIELD)); } // check segmentTerminatedEarly flag // at present segmentTerminateEarly cannot be used with grouped queries @@ -240,9 +240,9 @@ class SegmentTerminateEarlyTestState { TestMiniSolrCloudCluster.assertFalse(minTimestampDocKeys.isEmpty()); TestMiniSolrCloudCluster.assertTrue("numDocs="+numDocs+" is not even", (numDocs%2)==0); final Long oddFieldValue = new Long(minTimestampDocKeys.iterator().next().intValue()%2); - final SolrQuery query = new SolrQuery(oddField+":"+oddFieldValue); - query.setSort(timestampField, SolrQuery.ORDER.asc); // a sort order that is _not_ compatible with the merge sort order - query.setFields(keyField, oddField, timestampField); + final SolrQuery query = new SolrQuery(ODD_FIELD +":"+oddFieldValue); + query.setSort(TIMESTAMP_FIELD, SolrQuery.ORDER.asc); // a sort order that is _not_ compatible with the merge sort order + query.setFields(KEY_FIELD, ODD_FIELD, TIMESTAMP_FIELD); query.setRows(1); query.set(CommonParams.SEGMENT_TERMINATE_EARLY, true); final QueryResponse rsp = cloudSolrClient.query(query); @@ -251,9 +251,9 @@ class SegmentTerminateEarlyTestState { // check correctness of the first result if (rsp.getResults().getNumFound() > 0) { final SolrDocument solrDocument0 = rsp.getResults().get(0); - TestMiniSolrCloudCluster.assertTrue(keyField+" of ("+solrDocument0+") is not in minTimestampDocKeys("+minTimestampDocKeys+")", - minTimestampDocKeys.contains(solrDocument0.getFieldValue(keyField))); - TestMiniSolrCloudCluster.assertEquals(oddField, oddFieldValue, solrDocument0.getFieldValue(oddField)); + TestMiniSolrCloudCluster.assertTrue(KEY_FIELD +" of ("+solrDocument0+") is not in minTimestampDocKeys("+minTimestampDocKeys+")", + minTimestampDocKeys.contains(solrDocument0.getFieldValue(KEY_FIELD))); + TestMiniSolrCloudCluster.assertEquals(ODD_FIELD, oddFieldValue, solrDocument0.getFieldValue(ODD_FIELD)); } // check segmentTerminatedEarly flag TestMiniSolrCloudCluster.assertNotNull("responseHeader.segmentTerminatedEarly missing in "+rsp.getResponseHeader(), diff --git a/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java b/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java index 2642814850b..5e9e1802a22 100644 --- a/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/SolrXmlInZkTest.java @@ -163,7 +163,7 @@ public class SolrXmlInZkTest extends SolrTestCaseJ4 { // Just a random port, I'm not going to use it but just check that the Solr instance constructed from the XML // file in ZK overrides the default port. - private final String XML_FOR_ZK = + private static final String XML_FOR_ZK = "" + " " + " 127.0.0.1" + diff --git a/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java b/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java index 5e6283adce6..fa2449dae8c 100644 --- a/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java +++ b/solr/core/src/test/org/apache/solr/cloud/TestSegmentSorting.java @@ -123,7 +123,7 @@ public class TestSegmentSorting extends SolrCloudTestCase { public void testAtomicUpdateOfSegmentSortField() throws Exception { final CloudSolrClient cloudSolrClient = cluster.getSolrClient(); - final String updateField = SegmentTerminateEarlyTestState.timestampField; + final String updateField = SegmentTerminateEarlyTestState.TIMESTAMP_FIELD; // sanity check that updateField is in fact a DocValues only field, meaning it // would normally be eligable for inplace updates -- if it weren't also used for merge sorting diff --git a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java index 1ebb23239c1..b4ec29fbd3d 100644 --- a/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java +++ b/solr/core/src/test/org/apache/solr/core/OpenCloseCoreStressTest.java @@ -66,7 +66,7 @@ public class OpenCloseCoreStressTest extends SolrTestCaseJ4 { final int indexingThreads = TEST_NIGHTLY ? 9 : 5; final int queryThreads = TEST_NIGHTLY ? 9 : 5; - final int resetInterval = 30 * 60; // minutes to report then delete everything + static final int RESET_INTERVAL = 30 * 60; // minutes to report then delete everything long cumulativeDocs = 0; String url; @@ -165,7 +165,7 @@ public class OpenCloseCoreStressTest extends SolrTestCaseJ4 { int secondsRemaining = secondsToRun; do { - int cycleSeconds = Math.min(resetInterval, secondsRemaining); + int cycleSeconds = Math.min(RESET_INTERVAL, secondsRemaining); log.info(String.format(Locale.ROOT, "\n\n\n\n\nStarting a %,d second cycle, seconds left: %,d. Seconds run so far: %,d.", cycleSeconds, secondsRemaining, secondsRun)); @@ -177,7 +177,7 @@ public class OpenCloseCoreStressTest extends SolrTestCaseJ4 { queries.waitOnThreads(); - secondsRemaining = Math.max(secondsRemaining - resetInterval, 0); + secondsRemaining = Math.max(secondsRemaining - RESET_INTERVAL, 0); checkResults(queryingClients.get(0), queries, idxer); diff --git a/solr/core/src/test/org/apache/solr/schema/SpatialRPTFieldTypeTest.java b/solr/core/src/test/org/apache/solr/schema/SpatialRPTFieldTypeTest.java index a2afa2df371..0ff9e3ab0a8 100644 --- a/solr/core/src/test/org/apache/solr/schema/SpatialRPTFieldTypeTest.java +++ b/solr/core/src/test/org/apache/solr/schema/SpatialRPTFieldTypeTest.java @@ -62,12 +62,12 @@ public class SpatialRPTFieldTypeTest extends AbstractBadConfigTestBase { System.clearProperty("managed.schema.mutable"); System.clearProperty("enable.update.log"); } - - final String INDEXED_COORDINATES = "25,82"; - final String QUERY_COORDINATES = "24,81"; - final String DISTANCE_DEGREES = "1.3520328"; - final String DISTANCE_KILOMETERS = "150.33939"; - final String DISTANCE_MILES = "93.416565"; + + static final String INDEXED_COORDINATES = "25,82"; + static final String QUERY_COORDINATES = "24,81"; + static final String DISTANCE_DEGREES = "1.3520328"; + static final String DISTANCE_KILOMETERS = "150.33939"; + static final String DISTANCE_MILES = "93.416565"; public void testDistanceUnitsDegrees() throws Exception { setupRPTField("degrees", "true"); diff --git a/solr/core/src/test/org/apache/solr/search/TestRTGBase.java b/solr/core/src/test/org/apache/solr/search/TestRTGBase.java index bb1b08a04a5..b2964d8f306 100644 --- a/solr/core/src/test/org/apache/solr/search/TestRTGBase.java +++ b/solr/core/src/test/org/apache/solr/search/TestRTGBase.java @@ -47,7 +47,7 @@ public class TestRTGBase extends SolrTestCaseJ4 { protected long snapshotCount; protected long committedModelClock; protected volatile int lastId; - protected final String field = "val_l"; + protected static final String FIELD = "val_l"; protected Object[] syncArr; protected Object globalLock = this; diff --git a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java index 28ecaa30f6e..4ee4cb70afb 100644 --- a/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java +++ b/solr/core/src/test/org/apache/solr/search/TestRealTimeGet.java @@ -667,7 +667,7 @@ public class TestRealTimeGet extends TestRTGBase { } Long version = null; - SolrInputDocument sd = sdoc("id", Integer.toString(id), field, Long.toString(nextVal)); + SolrInputDocument sd = sdoc("id", Integer.toString(id), FIELD, Long.toString(nextVal)); if (opt) { if (correct) { @@ -762,7 +762,7 @@ public class TestRealTimeGet extends TestRTGBase { // This is also correct when filteredOut==true } else { assertEquals(1, doclist.size()); - long foundVal = (Long)(((Map)doclist.get(0)).get(field)); + long foundVal = (Long)(((Map)doclist.get(0)).get(FIELD)); long foundVer = (Long)(((Map)doclist.get(0)).get("_version_")); if (filteredOut || foundVal < Math.abs(info.val) || (foundVer == info.version && foundVal != info.val) ) { // if the version matches, the val must diff --git a/solr/core/src/test/org/apache/solr/search/TestReloadDeadlock.java b/solr/core/src/test/org/apache/solr/search/TestReloadDeadlock.java index 383070ccdb9..32c91db29bf 100644 --- a/solr/core/src/test/org/apache/solr/search/TestReloadDeadlock.java +++ b/solr/core/src/test/org/apache/solr/search/TestReloadDeadlock.java @@ -197,7 +197,7 @@ public class TestReloadDeadlock extends TestRTGBase { private void addDoc(int id, long nextVal, long version) throws Exception { ifVerbose("adding id", id, "val=", nextVal, "version", version); - Long returnedVersion = addAndGetVersion(sdoc("id", Integer.toString(id), field, Long.toString(nextVal), + Long returnedVersion = addAndGetVersion(sdoc("id", Integer.toString(id), FIELD, Long.toString(nextVal), "_version_", Long.toString(version)), params(DISTRIB_UPDATE_PARAM, FROM_LEADER)); if (returnedVersion != null) { assertEquals(version, returnedVersion.longValue()); diff --git a/solr/core/src/test/org/apache/solr/search/TestStressLucene.java b/solr/core/src/test/org/apache/solr/search/TestStressLucene.java index 779be435ba6..af2ef1db1f1 100644 --- a/solr/core/src/test/org/apache/solr/search/TestStressLucene.java +++ b/solr/core/src/test/org/apache/solr/search/TestStressLucene.java @@ -226,7 +226,7 @@ public class TestStressLucene extends TestRTGBase { if (tombstones) { Document d = new Document(); d.add(new Field("id","-"+Integer.toString(id), idFt)); - d.add(new Field(field, Long.toString(nextVal), ft2)); + d.add(new Field(FIELD, Long.toString(nextVal), ft2)); verbose("adding tombstone for id",id,"val=",nextVal); writer.updateDocument(new Term("id", "-"+Integer.toString(id)), d); } @@ -243,7 +243,7 @@ public class TestStressLucene extends TestRTGBase { if (tombstones) { Document d = new Document(); d.add(new Field("id","-"+Integer.toString(id), idFt)); - d.add(new Field(field, Long.toString(nextVal), ft2)); + d.add(new Field(FIELD, Long.toString(nextVal), ft2)); verbose("adding tombstone for id",id,"val=",nextVal); writer.updateDocument(new Term("id", "-"+Integer.toString(id)), d); } @@ -258,7 +258,7 @@ public class TestStressLucene extends TestRTGBase { // assertU(adoc("id",Integer.toString(id), field, Long.toString(nextVal))); Document d = new Document(); d.add(new Field("id",Integer.toString(id), idFt)); - d.add(new Field(field, Long.toString(nextVal), ft2)); + d.add(new Field(FIELD, Long.toString(nextVal), ft2)); verbose("adding id",id,"val=",nextVal); writer.updateDocument(new Term("id", Integer.toString(id)), d); if (tombstones) { @@ -337,7 +337,7 @@ public class TestStressLucene extends TestRTGBase { } assertTrue(docid >= 0); // we should have found the document, or its tombstone Document doc = r.document(docid); - long foundVal = Long.parseLong(doc.get(field)); + long foundVal = Long.parseLong(doc.get(FIELD)); if (foundVal < Math.abs(val)) { verbose("ERROR: id",id,"model_val=",val," foundVal=",foundVal,"reader=",reader); } diff --git a/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java b/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java index b6ecc2e75e2..933700d91b7 100644 --- a/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java +++ b/solr/core/src/test/org/apache/solr/search/TestStressRecovery.java @@ -228,7 +228,7 @@ public class TestStressRecovery extends TestRTGBase { } else { verbose("adding id", id, "val=", nextVal,"version",version); - Long returnedVersion = addAndGetVersion(sdoc("id", Integer.toString(id), field, Long.toString(nextVal), "_version_",Long.toString(version)), params(DISTRIB_UPDATE_PARAM,FROM_LEADER)); + Long returnedVersion = addAndGetVersion(sdoc("id", Integer.toString(id), FIELD, Long.toString(nextVal), "_version_",Long.toString(version)), params(DISTRIB_UPDATE_PARAM,FROM_LEADER)); if (returnedVersion != null) { assertEquals(version, returnedVersion.longValue()); } @@ -310,7 +310,7 @@ public class TestStressRecovery extends TestRTGBase { // there's no info we can get back with a delete, so not much we can check without further synchronization } else { assertEquals(1, doclist.size()); - long foundVal = (Long)(((Map)doclist.get(0)).get(field)); + long foundVal = (Long)(((Map)doclist.get(0)).get(FIELD)); long foundVer = (Long)(((Map)doclist.get(0)).get("_version_")); if (foundVer < Math.abs(info.version) || (foundVer == info.version && foundVal != info.val) ) { // if the version matches, the val must diff --git a/solr/core/src/test/org/apache/solr/search/TestStressReorder.java b/solr/core/src/test/org/apache/solr/search/TestStressReorder.java index bce1d75af2a..e53fed0b487 100644 --- a/solr/core/src/test/org/apache/solr/search/TestStressReorder.java +++ b/solr/core/src/test/org/apache/solr/search/TestStressReorder.java @@ -223,7 +223,7 @@ public class TestStressReorder extends TestRTGBase { } else { verbose("adding id", id, "val=", nextVal,"version",version); - Long returnedVersion = addAndGetVersion(sdoc("id", Integer.toString(id), field, Long.toString(nextVal), "_version_",Long.toString(version)), params(DISTRIB_UPDATE_PARAM,FROM_LEADER)); + Long returnedVersion = addAndGetVersion(sdoc("id", Integer.toString(id), FIELD, Long.toString(nextVal), "_version_",Long.toString(version)), params(DISTRIB_UPDATE_PARAM,FROM_LEADER)); if (returnedVersion != null) { assertEquals(version, returnedVersion.longValue()); } @@ -301,7 +301,7 @@ public class TestStressReorder extends TestRTGBase { // there's no info we can get back with a delete, so not much we can check without further synchronization } else { assertEquals(1, doclist.size()); - long foundVal = (Long)(((Map)doclist.get(0)).get(field)); + long foundVal = (Long)(((Map)doclist.get(0)).get(FIELD)); long foundVer = (Long)(((Map)doclist.get(0)).get("_version_")); if (foundVer < Math.abs(info.version) || (foundVer == info.version && foundVal != info.val) ) { // if the version matches, the val must diff --git a/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java b/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java index 4eea434b162..5272734a999 100644 --- a/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java +++ b/solr/core/src/test/org/apache/solr/search/TestStressUserVersions.java @@ -203,7 +203,7 @@ public class TestStressUserVersions extends TestRTGBase { } else { verbose("adding id", id, "val=", nextVal,"version",version); - Long returnedVersion = addAndGetVersion(sdoc("id", Integer.toString(id), field, Long.toString(nextVal), vfield, Long.toString(version)), null); + Long returnedVersion = addAndGetVersion(sdoc("id", Integer.toString(id), FIELD, Long.toString(nextVal), vfield, Long.toString(version)), null); // only update model if the version is newer synchronized (model) { @@ -282,7 +282,7 @@ public class TestStressUserVersions extends TestRTGBase { long foundVer = (Long)(((Map)doclist.get(0)).get(vfield)); if (isLive) { - long foundVal = (Long)(((Map)doclist.get(0)).get(field)); + long foundVal = (Long)(((Map)doclist.get(0)).get(FIELD)); if (foundVer < Math.abs(info.version) || (foundVer == info.version && foundVal != info.val) ) { // if the version matches, the val must log.error("ERROR, id=" + id + " found=" + response + " model" + info); @@ -290,7 +290,7 @@ public class TestStressUserVersions extends TestRTGBase { } } else { // if the doc is deleted (via tombstone), it shouldn't have a value on it. - assertNull( ((Map)doclist.get(0)).get(field) ); + assertNull( ((Map)doclist.get(0)).get(FIELD) ); if (foundVer < Math.abs(info.version)) { log.error("ERROR, id=" + id + " found=" + response + " model" + info); diff --git a/solr/core/src/test/org/apache/solr/search/TestStressVersions.java b/solr/core/src/test/org/apache/solr/search/TestStressVersions.java index ed51ae5e074..15b0c68737b 100644 --- a/solr/core/src/test/org/apache/solr/search/TestStressVersions.java +++ b/solr/core/src/test/org/apache/solr/search/TestStressVersions.java @@ -172,7 +172,7 @@ public class TestStressVersions extends TestRTGBase { verbose("adding id", id, "val=", nextVal); // assertU(adoc("id",Integer.toString(id), field, Long.toString(nextVal))); - Long version = addAndGetVersion(sdoc("id", Integer.toString(id), field, Long.toString(nextVal)), null); + Long version = addAndGetVersion(sdoc("id", Integer.toString(id), FIELD, Long.toString(nextVal)), null); assertTrue(version > 0); // only update model if the version is newer @@ -247,7 +247,7 @@ public class TestStressVersions extends TestRTGBase { // there's no info we can get back with a delete, so not much we can check without further synchronization } else { assertEquals(1, doclist.size()); - long foundVal = (Long)(((Map)doclist.get(0)).get(field)); + long foundVal = (Long)(((Map)doclist.get(0)).get(FIELD)); long foundVer = (Long)(((Map)doclist.get(0)).get("_version_")); if (foundVer < Math.abs(info.version) || (foundVer == info.version && foundVal != info.val) ) { // if the version matches, the val must diff --git a/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java b/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java index e2229c59f7f..16ace15cee0 100644 --- a/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java +++ b/solr/solrj/src/test/org/apache/solr/common/util/TestJavaBinCodec.java @@ -47,10 +47,10 @@ import org.noggit.CharArr; public class TestJavaBinCodec extends SolrTestCaseJ4 { private static final String SOLRJ_JAVABIN_BACKCOMPAT_BIN = "/solrj/javabin_backcompat.bin"; - private final String BIN_FILE_LOCATION = "./solr/solrj/src/test-files/solrj/javabin_backcompat.bin"; + private static final String BIN_FILE_LOCATION = "./solr/solrj/src/test-files/solrj/javabin_backcompat.bin"; private static final String SOLRJ_JAVABIN_BACKCOMPAT_BIN_CHILD_DOCS = "/solrj/javabin_backcompat_child_docs.bin"; - private final String BIN_FILE_LOCATION_CHILD_DOCS = "./solr/solrj/src/test-files/solrj/javabin_backcompat_child_docs.bin"; + private static final String BIN_FILE_LOCATION_CHILD_DOCS = "./solr/solrj/src/test-files/solrj/javabin_backcompat_child_docs.bin"; public void testStrings() throws Exception { for (int i = 0; i < 10000 * RANDOM_MULTIPLIER; i++) { From cc623403bd82a176a057e1c9567f37d01f7391c4 Mon Sep 17 00:00:00 2001 From: yonik Date: Thu, 30 Mar 2017 12:55:27 -0400 Subject: [PATCH 095/563] SOLR-7452: refinement of missing buckets and partial facets through missing buckets --- .../solr/search/facet/FacetFieldMerger.java | 25 ++++++++++++++++++ .../search/facet/FacetFieldProcessor.java | 14 ++++++++++ .../facet/FacetRequestSortedMerger.java | 8 ++++++ .../search/facet/TestJsonFacetRefinement.java | 26 ++++++++++++++++++- 4 files changed, 72 insertions(+), 1 deletion(-) diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java index 63e8743c32c..f8f646348ed 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldMerger.java @@ -18,8 +18,11 @@ package org.apache.solr.search.facet; import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import org.apache.solr.common.util.SimpleOrderedMap; @@ -167,7 +170,29 @@ public class FacetFieldMerger extends FacetRequestSortedMerger { // basically , only do at the top-level facet? } + @Override + Map getRefinementSpecial(Context mcontext, Map refinement, Collection tagsWithPartial) { + if (!tagsWithPartial.isEmpty()) { + // Since special buckets missing and allBuckets themselves will always be included, we only need to worry about subfacets being partial. + if (freq.missing) { + refinement = getRefinementSpecial(mcontext, refinement, tagsWithPartial, missingBucket, "missing"); + } + if (freq.allBuckets) { + refinement = getRefinementSpecial(mcontext, refinement, tagsWithPartial, allBuckets, "allBuckets"); + } + } + return refinement; + } + private Map getRefinementSpecial(Context mcontext, Map refinement, Collection tagsWithPartial, FacetBucket bucket, String label) { + // boolean prev = mcontext.setBucketWasMissing(true); // the special buckets should have the same "missing" status as this facet, so no need to set it again + Map bucketRefinement = bucket.getRefinement(mcontext, tagsWithPartial); + if (bucketRefinement != null) { + refinement = refinement == null ? new HashMap<>(2) : refinement; + refinement.put(label, bucketRefinement); + } + return refinement; + } private static class FacetNumBucketsMerger extends FacetMerger { long sumBuckets; diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java index a29e78d9e6c..d4daf08d80a 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java @@ -36,6 +36,8 @@ import org.apache.solr.schema.FieldType; import org.apache.solr.schema.SchemaField; import org.apache.solr.search.DocSet; +import static org.apache.solr.search.facet.FacetContext.SKIP_FACET; + /** * Facet processing based on field values. (not range nor by query) * @see FacetField @@ -528,6 +530,9 @@ abstract class FacetFieldProcessor extends FacetProcessor { } protected SimpleOrderedMap refineFacets() throws IOException { + boolean skipThisFacet = (fcontext.flags & SKIP_FACET) != 0; + + List leaves = asList(fcontext.facetInfo.get("_l")); // We have not seen this bucket: do full faceting for this bucket, including all sub-facets List skip = asList(fcontext.facetInfo.get("_s")); // We have seen this bucket, so skip stats on it, and skip sub-facets except for the specified sub-facets that should calculate specified buckets. List partial = asList(fcontext.facetInfo.get("_p")); // We have not seen this bucket, do full faceting for this bucket, and most sub-facets... but some sub-facets are partial and should only visit specified buckets. @@ -563,6 +568,15 @@ abstract class FacetFieldProcessor extends FacetProcessor { bucketList.add( refineBucket(bucketVal, false, facetInfo ) ); } + if (freq.missing) { + Map bucketFacetInfo = (Map)fcontext.facetInfo.get("missing"); + + if (bucketFacetInfo != null || !skipThisFacet) { + SimpleOrderedMap missingBucket = new SimpleOrderedMap<>(); + fillBucket(missingBucket, getFieldMissingQuery(fcontext.searcher, freq.field), null, skipThisFacet, bucketFacetInfo); + res.add("missing", missingBucket); + } + } // If there are just a couple of leaves, and if the domain is large, then // going by term is likely the most efficient? diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java b/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java index e05064c4148..9ffdea7835a 100644 --- a/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java +++ b/solr/core/src/java/org/apache/solr/search/facet/FacetRequestSortedMerger.java @@ -240,6 +240,14 @@ abstract class FacetRequestSortedMerger getRefinementSpecial(Context mcontext, Map refinement, Collection tagsWithPartial) { return refinement; } diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java index b4b022066ea..52b8be4fae5 100644 --- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java +++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java @@ -219,6 +219,17 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "}" ); + // test partial _p under a missing bucket + doTestRefine("{top:{type:terms, field:Afield, refine:true, limit:1, missing:true, facet:{x : {type:terms, field:X, limit:1, refine:true} } } }", + "{top: {buckets:[], missing:{count:12, x:{buckets:[{val:x2, count:4},{val:x3, count:2}]} } } }", + "{top: {buckets:[], missing:{count:10, x:{buckets:[{val:x1, count:5},{val:x4, count:3}]} } } }", + "=={top: {" + + "missing:{x:{_l:[x1]}}" + + " } " + + "}" + , null + ); + } @@ -266,6 +277,17 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { ); ****/ + // test refining under the special "missing" bucket of a field facet + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "f:{type:terms, field:missing_s, limit:1, overrequest:0, missing:true, refine:true, facet:{ cat:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true } }}" + + "}" + ) + , "facets=={ count:8" + + ", f:{ buckets:[], missing:{count:8, cat:{buckets:[{val:A,count:4}]} } }" + // just like the previous response, just nested under a field facet + "}" + ); + client.testJQ(params(p, "q", "*:*", "json.facet", "{" + @@ -317,7 +339,7 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "}" ); - // test missing buckets (field facet within field facet) + // test partial buckets (field facet within field facet) client.testJQ(params(p, "q", "*:*", "json.facet", "{" + "ab:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true, facet:{ xy:{type:terms, field:${xy_s}, limit:1, overrequest:0, refine:true } }}" + @@ -345,6 +367,8 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "}" ); + + } From bdb0d588ee8129cf42df03f6185fbb3f4d8e0af4 Mon Sep 17 00:00:00 2001 From: yonik Date: Fri, 31 Mar 2017 12:55:15 -0400 Subject: [PATCH 096/563] SOLR-7452: add more tests for refinement of missing buckets --- .../search/facet/TestJsonFacetRefinement.java | 62 ++++++++++++------- 1 file changed, 41 insertions(+), 21 deletions(-) diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java index 52b8be4fae5..bcb5f09c891 100644 --- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java +++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacetRefinement.java @@ -244,21 +244,22 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { client.deleteByQuery("*:*", null); - ModifiableSolrParams p = params("cat_s", "cat_s", "xy_s", "xy_s", "num_d", "num_d"); + ModifiableSolrParams p = params("cat_s", "cat_s", "xy_s", "xy_s", "num_d", "num_d", "qw_s", "qw_s"); String cat_s = p.get("cat_s"); String xy_s = p.get("xy_s"); + String qw_s = p.get("qw_s"); String num_d = p.get("num_d"); - clients.get(0).add( sdoc("id", "01", "all_s","all", cat_s, "A", xy_s, "X" ,num_d, -1) ); // A wins count tie - clients.get(0).add( sdoc("id", "02", "all_s","all", cat_s, "B", xy_s, "Y", num_d, 3) ); + clients.get(0).add( sdoc("id", "01", "all_s","all", cat_s, "A", xy_s, "X" ,num_d, -1, qw_s, "Q") ); // A wins count tie + clients.get(0).add( sdoc("id", "02", "all_s","all", cat_s, "B", xy_s, "Y", num_d, 3 ) ); - clients.get(1).add( sdoc("id", "11", "all_s","all", cat_s, "B", xy_s, "X", num_d, -5) ); // B highest count - clients.get(1).add( sdoc("id", "12", "all_s","all", cat_s, "B", xy_s, "Y", num_d, -11) ); - clients.get(1).add( sdoc("id", "13", "all_s","all", cat_s, "A", xy_s, "X", num_d, 7) ); + clients.get(1).add( sdoc("id", "11", "all_s","all", cat_s, "B", xy_s, "X", num_d, -5 ) ); // B highest count + clients.get(1).add( sdoc("id", "12", "all_s","all", cat_s, "B", xy_s, "Y", num_d, -11, qw_s, "W") ); + clients.get(1).add( sdoc("id", "13", "all_s","all", cat_s, "A", xy_s, "X", num_d, 7 ) ); - clients.get(2).add( sdoc("id", "21", "all_s","all", cat_s, "A", xy_s, "X", num_d, 17) ); // A highest count - clients.get(2).add( sdoc("id", "22", "all_s","all", cat_s, "A", xy_s, "Y", num_d, -19) ); - clients.get(2).add( sdoc("id", "23", "all_s","all", cat_s, "B", xy_s, "X", num_d, 11) ); + clients.get(2).add( sdoc("id", "21", "all_s","all", cat_s, "A", xy_s, "X", num_d, 17, qw_s, "W") ); // A highest count + clients.get(2).add( sdoc("id", "22", "all_s","all", cat_s, "A", xy_s, "Y", num_d, -19 ) ); + clients.get(2).add( sdoc("id", "23", "all_s","all", cat_s, "B", xy_s, "X", num_d, 11 ) ); client.commit(); @@ -277,18 +278,6 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { ); ****/ - // test refining under the special "missing" bucket of a field facet - client.testJQ(params(p, "q", "*:*", - "json.facet", "{" + - "f:{type:terms, field:missing_s, limit:1, overrequest:0, missing:true, refine:true, facet:{ cat:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true } }}" + - "}" - ) - , "facets=={ count:8" + - ", f:{ buckets:[], missing:{count:8, cat:{buckets:[{val:A,count:4}]} } }" + // just like the previous response, just nested under a field facet - "}" - ); - - client.testJQ(params(p, "q", "*:*", "json.facet", "{" + "cat0:{type:terms, field:${cat_s}, sort:'count desc', limit:1, overrequest:0, refine:false}" + @@ -367,6 +356,37 @@ public class TestJsonFacetRefinement extends SolrTestCaseHS { "}" ); + // test refining under the special "missing" bucket of a field facet + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + "f:{type:terms, field:missing_s, limit:1, overrequest:0, missing:true, refine:true, facet:{ cat:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true } }}" + + "}" + ) + , "facets=={ count:8" + + ", f:{ buckets:[], missing:{count:8, cat:{buckets:[{val:A,count:4}]} } }" + // just like the previous response, just nested under a field facet + "}" + ); + + // test filling in "missing" bucket for partially refined facets + client.testJQ(params(p, "q", "*:*", + "json.facet", "{" + + // test all values missing in sub-facet + " ab :{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{ zz:{type:terms, field:missing_s, limit:1, overrequest:0, refine:false, missing:true} }}" + + ",ab2:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , facet:{ zz:{type:terms, field:missing_s, limit:1, overrequest:0, refine:true , missing:true} }}" + + // test some values missing in sub-facet (and test that this works with normal partial bucket refinement) + ", cd :{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:false, facet:{ qw:{type:terms, field:${qw_s}, limit:1, overrequest:0, refine:false, missing:true, facet:{qq:{query:'*:*'}} } }}" + + ", cd2:{type:terms, field:${cat_s}, limit:1, overrequest:0, refine:true , facet:{ qw:{type:terms, field:${qw_s}, limit:1, overrequest:0, refine:true , missing:true, facet:{qq:{query:'*:*'}} } }}" + + + "}" + ) + , "facets=={ count:8" + + ", ab:{ buckets:[ {val:A, count:3, zz:{buckets:[], missing:{count:3}}}] }" + + ",ab2:{ buckets:[ {val:A, count:4, zz:{buckets:[], missing:{count:4}}}] }" + + ", cd:{ buckets:[ {val:A, count:3, qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:1,qq:{count:1}}}}] }" + + ",cd2:{ buckets:[ {val:A, count:4, qw:{buckets:[{val:Q, count:1, qq:{count:1}}], missing:{count:2,qq:{count:2}}}}] }" + + "}" + ); + } From 413555651da61b764c05313b37b44339581be02d Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Fri, 31 Mar 2017 18:10:27 +0100 Subject: [PATCH 097/563] =?UTF-8?q?LUCENE-7763:=20Remove=20outdated=20comm?= =?UTF-8?q?ent=20in=20IndexWriterConfig.setIndexSort=20javadocs.=20(?= =?UTF-8?q?=E9=A9=AC=E5=8F=AF=E9=98=B3=20via=20Christine=20Poerschke)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- lucene/CHANGES.txt | 5 +++++ .../src/java/org/apache/lucene/index/IndexWriterConfig.java | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index e0827e78e90..833fd3c9ef3 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -106,6 +106,11 @@ Bug Fixes * LUCENE-7755: Fixed join queries to not reference IndexReaders, as it could cause leaks if they are cached. (Adrien Grand) +Other + +* LUCENE-7763: Remove outdated comment in IndexWriterConfig.setIndexSort javadocs. + (马可阳 via Christine Poerschke) + ======================= Lucene 6.5.0 ======================= API Changes diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java index 1e1e795d50b..0fdbc3e8652 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java @@ -466,7 +466,7 @@ public final class IndexWriterConfig extends LiveIndexWriterConfig { SortField.Type.FLOAT); /** - * Set the {@link Sort} order to use when merging segments. Note that newly flushed segments will remain unsorted. + * Set the {@link Sort} order to use when merging segments. */ public IndexWriterConfig setIndexSort(Sort sort) { for(SortField sortField : sort.getSort()) { From b60b86ecab797396ab6bd8be82740191922b0aa2 Mon Sep 17 00:00:00 2001 From: Chris Hostetter Date: Fri, 31 Mar 2017 17:01:42 -0700 Subject: [PATCH 098/563] SOLR-10399: Generalize some internal facet logic to simplify points/non-points field handling --- solr/CHANGES.txt | 2 ++ .../java/org/apache/solr/request/SimpleFacets.java | 14 +++----------- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 097ee2b0690..1efefd071ee 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -178,6 +178,8 @@ Other Changes * SOLR-10338: Configure SecureRandom non blocking for tests. (Mihaly Toth, hossman, Ishan Chattopadhyaya, via Mark Miller) +* SOLR-10399: Generalize some internal facet logic to simplify points/non-points field handling (Adrien Grand, hossman) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java index b452802cad1..8972121c5f0 100644 --- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java +++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java @@ -851,17 +851,9 @@ public class SimpleFacets { SchemaField sf = searcher.getSchema().getField(field); FieldType ft = sf.getType(); NamedList res = new NamedList<>(); - if (ft.isPointField()) { - for (String term : terms) { - int count = searcher.numDocs(ft.getFieldQuery(null, sf, term), parsed.docs); - res.add(term, count); - } - } else { - for (String term : terms) { - String internal = ft.toInternal(term); - int count = searcher.numDocs(new TermQuery(new Term(field, internal)), parsed.docs); - res.add(term, count); - } + for (String term : terms) { + int count = searcher.numDocs(ft.getFieldQuery(null, sf, term), parsed.docs); + res.add(term, count); } return res; } From 35aac1d4623a34fe6b03d934eefd9066c61a95d5 Mon Sep 17 00:00:00 2001 From: Chris Hostetter Date: Fri, 31 Mar 2017 18:16:13 -0700 Subject: [PATCH 099/563] SOLR-10399: cleanup unused imports --- solr/core/src/java/org/apache/solr/request/SimpleFacets.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java index 8972121c5f0..0bfef4c7415 100644 --- a/solr/core/src/java/org/apache/solr/request/SimpleFacets.java +++ b/solr/core/src/java/org/apache/solr/request/SimpleFacets.java @@ -42,7 +42,6 @@ import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.MultiPostingsEnum; import org.apache.lucene.index.PostingsEnum; -import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.BooleanClause.Occur; @@ -52,7 +51,6 @@ import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.FilterCollector; import org.apache.lucene.search.LeafCollector; import org.apache.lucene.search.Query; -import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.grouping.AllGroupHeadsCollector; import org.apache.lucene.search.grouping.term.TermAllGroupsCollector; import org.apache.lucene.search.grouping.term.TermGroupFacetCollector; From 580f6e98fb033dbbb8e0921fc3175021714ce956 Mon Sep 17 00:00:00 2001 From: Alexandre Rafalovitch Date: Sat, 1 Apr 2017 13:42:23 -0400 Subject: [PATCH 100/563] SOLR-7383: Replace DIH 'rss' example with 'atom' rss example was broken for multiple reasons. atom example showcases the same - and more - features and uses the smallest config file needed to make it work. --- solr/CHANGES.txt | 2 + solr/README.txt | 2 +- solr/example/README.txt | 2 +- solr/example/example-DIH/README.txt | 11 +- .../solr/atom/conf/atom-data-config.xml | 35 + .../{rss => atom}/conf/lang/stopwords_en.txt | 0 .../example-DIH/solr/atom/conf/managed-schema | 106 + .../solr/{rss => atom}/conf/protwords.txt | 6 +- .../example-DIH/solr/atom/conf/solrconfig.xml | 61 + .../solr/{rss => atom}/conf/synonyms.txt | 0 .../example-DIH/solr/atom/conf/url_types.txt | 1 + .../solr/{rss => atom}/core.properties | 0 .../solr/rss/conf/admin-extra.html | 24 - .../rss/conf/admin-extra.menu-bottom.html | 25 - .../solr/rss/conf/admin-extra.menu-top.html | 25 - .../clustering/carrot2/kmeans-attributes.xml | 19 - .../clustering/carrot2/lingo-attributes.xml | 24 - .../clustering/carrot2/stc-attributes.xml | 19 - .../example-DIH/solr/rss/conf/currency.xml | 67 - .../example-DIH/solr/rss/conf/elevate.xml | 42 - .../solr/rss/conf/lang/contractions_ca.txt | 8 - .../solr/rss/conf/lang/contractions_fr.txt | 15 - .../solr/rss/conf/lang/contractions_ga.txt | 5 - .../solr/rss/conf/lang/contractions_it.txt | 23 - .../solr/rss/conf/lang/hyphenations_ga.txt | 5 - .../solr/rss/conf/lang/stemdict_nl.txt | 6 - .../solr/rss/conf/lang/stoptags_ja.txt | 420 -- .../solr/rss/conf/lang/stopwords_ar.txt | 125 - .../solr/rss/conf/lang/stopwords_bg.txt | 193 - .../solr/rss/conf/lang/stopwords_ca.txt | 220 - .../solr/rss/conf/lang/stopwords_ckb.txt | 136 - .../solr/rss/conf/lang/stopwords_cz.txt | 172 - .../solr/rss/conf/lang/stopwords_da.txt | 110 - .../solr/rss/conf/lang/stopwords_de.txt | 294 -- .../solr/rss/conf/lang/stopwords_el.txt | 78 - .../solr/rss/conf/lang/stopwords_es.txt | 356 -- .../solr/rss/conf/lang/stopwords_eu.txt | 99 - .../solr/rss/conf/lang/stopwords_fa.txt | 313 -- .../solr/rss/conf/lang/stopwords_fi.txt | 97 - .../solr/rss/conf/lang/stopwords_fr.txt | 186 - .../solr/rss/conf/lang/stopwords_ga.txt | 110 - .../solr/rss/conf/lang/stopwords_gl.txt | 161 - .../solr/rss/conf/lang/stopwords_hi.txt | 235 - .../solr/rss/conf/lang/stopwords_hu.txt | 211 - .../solr/rss/conf/lang/stopwords_hy.txt | 46 - .../solr/rss/conf/lang/stopwords_id.txt | 359 -- .../solr/rss/conf/lang/stopwords_it.txt | 303 -- .../solr/rss/conf/lang/stopwords_ja.txt | 127 - .../solr/rss/conf/lang/stopwords_lv.txt | 172 - .../solr/rss/conf/lang/stopwords_nl.txt | 119 - .../solr/rss/conf/lang/stopwords_no.txt | 194 - .../solr/rss/conf/lang/stopwords_pt.txt | 253 -- .../solr/rss/conf/lang/stopwords_ro.txt | 233 - .../solr/rss/conf/lang/stopwords_ru.txt | 243 -- .../solr/rss/conf/lang/stopwords_sv.txt | 133 - .../solr/rss/conf/lang/stopwords_th.txt | 119 - .../solr/rss/conf/lang/stopwords_tr.txt | 212 - .../solr/rss/conf/lang/userdict_ja.txt | 29 - .../example-DIH/solr/rss/conf/managed-schema | 1096 ----- .../solr/rss/conf/mapping-FoldToASCII.txt | 3813 ----------------- .../solr/rss/conf/mapping-ISOLatin1Accent.txt | 246 -- .../solr/rss/conf/rss-data-config.xml | 26 - .../example-DIH/solr/rss/conf/solrconfig.xml | 1396 ------ .../example-DIH/solr/rss/conf/spellings.txt | 2 - .../example-DIH/solr/rss/conf/stopwords.txt | 14 - .../solr/rss/conf/update-script.js | 53 - .../solr/rss/conf/xslt/example.xsl | 132 - .../solr/rss/conf/xslt/example_atom.xsl | 67 - .../solr/rss/conf/xslt/example_rss.xsl | 66 - .../example-DIH/solr/rss/conf/xslt/luke.xsl | 337 -- .../solr/rss/conf/xslt/updateXml.xsl | 70 - 71 files changed, 213 insertions(+), 13696 deletions(-) create mode 100644 solr/example/example-DIH/solr/atom/conf/atom-data-config.xml rename solr/example/example-DIH/solr/{rss => atom}/conf/lang/stopwords_en.txt (100%) create mode 100644 solr/example/example-DIH/solr/atom/conf/managed-schema rename solr/example/example-DIH/solr/{rss => atom}/conf/protwords.txt (86%) create mode 100644 solr/example/example-DIH/solr/atom/conf/solrconfig.xml rename solr/example/example-DIH/solr/{rss => atom}/conf/synonyms.txt (100%) create mode 100644 solr/example/example-DIH/solr/atom/conf/url_types.txt rename solr/example/example-DIH/solr/{rss => atom}/core.properties (100%) delete mode 100644 solr/example/example-DIH/solr/rss/conf/admin-extra.html delete mode 100644 solr/example/example-DIH/solr/rss/conf/admin-extra.menu-bottom.html delete mode 100644 solr/example/example-DIH/solr/rss/conf/admin-extra.menu-top.html delete mode 100644 solr/example/example-DIH/solr/rss/conf/clustering/carrot2/kmeans-attributes.xml delete mode 100644 solr/example/example-DIH/solr/rss/conf/clustering/carrot2/lingo-attributes.xml delete mode 100644 solr/example/example-DIH/solr/rss/conf/clustering/carrot2/stc-attributes.xml delete mode 100644 solr/example/example-DIH/solr/rss/conf/currency.xml delete mode 100644 solr/example/example-DIH/solr/rss/conf/elevate.xml delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/contractions_ca.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/contractions_fr.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/contractions_ga.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/contractions_it.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/hyphenations_ga.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stemdict_nl.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stoptags_ja.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_ar.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_bg.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_ca.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_ckb.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_cz.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_da.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_de.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_el.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_es.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_eu.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_fa.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_fi.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_fr.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_ga.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_gl.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_hi.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_hu.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_hy.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_id.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_it.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_ja.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_lv.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_nl.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_no.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_pt.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_ro.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_ru.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_sv.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_th.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/stopwords_tr.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/lang/userdict_ja.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/managed-schema delete mode 100644 solr/example/example-DIH/solr/rss/conf/mapping-FoldToASCII.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/mapping-ISOLatin1Accent.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/rss-data-config.xml delete mode 100644 solr/example/example-DIH/solr/rss/conf/solrconfig.xml delete mode 100644 solr/example/example-DIH/solr/rss/conf/spellings.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/stopwords.txt delete mode 100644 solr/example/example-DIH/solr/rss/conf/update-script.js delete mode 100644 solr/example/example-DIH/solr/rss/conf/xslt/example.xsl delete mode 100644 solr/example/example-DIH/solr/rss/conf/xslt/example_atom.xsl delete mode 100644 solr/example/example-DIH/solr/rss/conf/xslt/example_rss.xsl delete mode 100644 solr/example/example-DIH/solr/rss/conf/xslt/luke.xsl delete mode 100644 solr/example/example-DIH/solr/rss/conf/xslt/updateXml.xsl diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 1efefd071ee..3187dc3c7bc 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -180,6 +180,8 @@ Other Changes * SOLR-10399: Generalize some internal facet logic to simplify points/non-points field handling (Adrien Grand, hossman) +* SOLR-7383: New DataImportHandler 'atom' example, replacing broken 'rss' example (Alexandre Rafalovitch) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/README.txt b/solr/README.txt index 4ef5eac9fd6..6af0cc65e5e 100644 --- a/solr/README.txt +++ b/solr/README.txt @@ -67,7 +67,7 @@ Solr includes a few examples to help you get started. To run a specific example, bin/solr -e where is one of: cloud : SolrCloud example - dih : Data Import Handler (rdbms, mail, rss, tika) + dih : Data Import Handler (rdbms, mail, atom, tika) schemaless : Schema-less example (schema is inferred from data during indexing) techproducts : Kitchen sink example providing comprehensive examples of Solr features diff --git a/solr/example/README.txt b/solr/example/README.txt index d8402eb0974..4c8cca1fd88 100644 --- a/solr/example/README.txt +++ b/solr/example/README.txt @@ -22,7 +22,7 @@ separate directory. To run a specific example, do: bin/solr -e where is one of: cloud : SolrCloud example - dih : Data Import Handler (rdbms, mail, rss, tika) + dih : Data Import Handler (rdbms, mail, atom, tika) schemaless : Schema-less example (schema is inferred from data during indexing) techproducts : Kitchen sink example providing comprehensive examples of Solr features diff --git a/solr/example/example-DIH/README.txt b/solr/example/example-DIH/README.txt index 0926bb657bc..ea8d28fb335 100644 --- a/solr/example/example-DIH/README.txt +++ b/solr/example/example-DIH/README.txt @@ -16,7 +16,7 @@ Solr DataImportHandler example configuration -------------------------------------------- -To run this example, use the "-e" option of the bin/solr script: +To run this multi-core example, use the "-e" option of the bin/solr script: > bin/solr -e dih @@ -28,9 +28,9 @@ When Solr is started connect to: http://localhost:8983/solr/db/dataimport?command=full-import -* To import data from an RSS feed, connect to: +* To import data from an ATOM feed, connect to: - http://localhost:8983/solr/rss/dataimport?command=full-import + http://localhost:8983/solr/atom/dataimport?command=full-import * To import data from your IMAP server: @@ -45,6 +45,5 @@ When Solr is started connect to: http://localhost:8983/solr/tika/dataimport?command=full-import -See also README.txt in the solr subdirectory, and check -http://wiki.apache.org/solr/DataImportHandler for detailed -usage guide and tutorial. +Check also the Solr Reference Guide for detailed usage guide: +https://cwiki.apache.org/confluence/display/solr/Uploading+Structured+Data+Store+Data+with+the+Data+Import+Handler diff --git a/solr/example/example-DIH/solr/atom/conf/atom-data-config.xml b/solr/example/example-DIH/solr/atom/conf/atom-data-config.xml new file mode 100644 index 00000000000..53b50607f37 --- /dev/null +++ b/solr/example/example-DIH/solr/atom/conf/atom-data-config.xml @@ -0,0 +1,35 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_en.txt b/solr/example/example-DIH/solr/atom/conf/lang/stopwords_en.txt similarity index 100% rename from solr/example/example-DIH/solr/rss/conf/lang/stopwords_en.txt rename to solr/example/example-DIH/solr/atom/conf/lang/stopwords_en.txt diff --git a/solr/example/example-DIH/solr/atom/conf/managed-schema b/solr/example/example-DIH/solr/atom/conf/managed-schema new file mode 100644 index 00000000000..58751520d5c --- /dev/null +++ b/solr/example/example-DIH/solr/atom/conf/managed-schema @@ -0,0 +1,106 @@ + + + + + id + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/solr/example/example-DIH/solr/rss/conf/protwords.txt b/solr/example/example-DIH/solr/atom/conf/protwords.txt similarity index 86% rename from solr/example/example-DIH/solr/rss/conf/protwords.txt rename to solr/example/example-DIH/solr/atom/conf/protwords.txt index 1dfc0abecbf..1303e42a061 100644 --- a/solr/example/example-DIH/solr/rss/conf/protwords.txt +++ b/solr/example/example-DIH/solr/atom/conf/protwords.txt @@ -14,8 +14,4 @@ # Use a protected word file to protect against the stemmer reducing two # unrelated words to the same base word. -# Some non-words that normally won't be encountered, -# just to test that they won't be stemmed. -dontstems -zwhacky - +lucene diff --git a/solr/example/example-DIH/solr/atom/conf/solrconfig.xml b/solr/example/example-DIH/solr/atom/conf/solrconfig.xml new file mode 100644 index 00000000000..22005ddcaba --- /dev/null +++ b/solr/example/example-DIH/solr/atom/conf/solrconfig.xml @@ -0,0 +1,61 @@ + + + + + + + + 7.0.0 + + + + + + explicit + text + + + + + + atom-data-config.xml + trim_text + + + + + text_en_splitting + + + diff --git a/solr/example/example-DIH/solr/rss/conf/synonyms.txt b/solr/example/example-DIH/solr/atom/conf/synonyms.txt similarity index 100% rename from solr/example/example-DIH/solr/rss/conf/synonyms.txt rename to solr/example/example-DIH/solr/atom/conf/synonyms.txt diff --git a/solr/example/example-DIH/solr/atom/conf/url_types.txt b/solr/example/example-DIH/solr/atom/conf/url_types.txt new file mode 100644 index 00000000000..808f3138466 --- /dev/null +++ b/solr/example/example-DIH/solr/atom/conf/url_types.txt @@ -0,0 +1 @@ + diff --git a/solr/example/example-DIH/solr/rss/core.properties b/solr/example/example-DIH/solr/atom/core.properties similarity index 100% rename from solr/example/example-DIH/solr/rss/core.properties rename to solr/example/example-DIH/solr/atom/core.properties diff --git a/solr/example/example-DIH/solr/rss/conf/admin-extra.html b/solr/example/example-DIH/solr/rss/conf/admin-extra.html deleted file mode 100644 index fecab20513d..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/admin-extra.html +++ /dev/null @@ -1,24 +0,0 @@ - - - diff --git a/solr/example/example-DIH/solr/rss/conf/admin-extra.menu-bottom.html b/solr/example/example-DIH/solr/rss/conf/admin-extra.menu-bottom.html deleted file mode 100644 index 3359a460a48..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/admin-extra.menu-bottom.html +++ /dev/null @@ -1,25 +0,0 @@ - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/admin-extra.menu-top.html b/solr/example/example-DIH/solr/rss/conf/admin-extra.menu-top.html deleted file mode 100644 index 0886cee37a4..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/admin-extra.menu-top.html +++ /dev/null @@ -1,25 +0,0 @@ - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/clustering/carrot2/kmeans-attributes.xml b/solr/example/example-DIH/solr/rss/conf/clustering/carrot2/kmeans-attributes.xml deleted file mode 100644 index d802465f669..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/clustering/carrot2/kmeans-attributes.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - - - - - - - - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/clustering/carrot2/lingo-attributes.xml b/solr/example/example-DIH/solr/rss/conf/clustering/carrot2/lingo-attributes.xml deleted file mode 100644 index 4bf13608b36..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/clustering/carrot2/lingo-attributes.xml +++ /dev/null @@ -1,24 +0,0 @@ - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/solr/example/example-DIH/solr/rss/conf/clustering/carrot2/stc-attributes.xml b/solr/example/example-DIH/solr/rss/conf/clustering/carrot2/stc-attributes.xml deleted file mode 100644 index c1bf110c8fd..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/clustering/carrot2/stc-attributes.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - - - - - - - - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/currency.xml b/solr/example/example-DIH/solr/rss/conf/currency.xml deleted file mode 100644 index 3a9c58afee8..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/currency.xml +++ /dev/null @@ -1,67 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/elevate.xml b/solr/example/example-DIH/solr/rss/conf/elevate.xml deleted file mode 100644 index 2c09ebed669..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/elevate.xml +++ /dev/null @@ -1,42 +0,0 @@ - - - - - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/lang/contractions_ca.txt b/solr/example/example-DIH/solr/rss/conf/lang/contractions_ca.txt deleted file mode 100644 index 307a85f913d..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/contractions_ca.txt +++ /dev/null @@ -1,8 +0,0 @@ -# Set of Catalan contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -l -m -n -s -t diff --git a/solr/example/example-DIH/solr/rss/conf/lang/contractions_fr.txt b/solr/example/example-DIH/solr/rss/conf/lang/contractions_fr.txt deleted file mode 100644 index f1bba51b23e..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/contractions_fr.txt +++ /dev/null @@ -1,15 +0,0 @@ -# Set of French contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -l -m -t -qu -n -s -j -d -c -jusqu -quoiqu -lorsqu -puisqu diff --git a/solr/example/example-DIH/solr/rss/conf/lang/contractions_ga.txt b/solr/example/example-DIH/solr/rss/conf/lang/contractions_ga.txt deleted file mode 100644 index 9ebe7fa349a..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/contractions_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -d -m -b diff --git a/solr/example/example-DIH/solr/rss/conf/lang/contractions_it.txt b/solr/example/example-DIH/solr/rss/conf/lang/contractions_it.txt deleted file mode 100644 index cac04095372..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/contractions_it.txt +++ /dev/null @@ -1,23 +0,0 @@ -# Set of Italian contractions for ElisionFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -c -l -all -dall -dell -nell -sull -coll -pell -gl -agl -dagl -degl -negl -sugl -un -m -t -s -v -d diff --git a/solr/example/example-DIH/solr/rss/conf/lang/hyphenations_ga.txt b/solr/example/example-DIH/solr/rss/conf/lang/hyphenations_ga.txt deleted file mode 100644 index 4d2642cc5a3..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/hyphenations_ga.txt +++ /dev/null @@ -1,5 +0,0 @@ -# Set of Irish hyphenations for StopFilter -# TODO: load this as a resource from the analyzer and sync it in build.xml -h -n -t diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stemdict_nl.txt b/solr/example/example-DIH/solr/rss/conf/lang/stemdict_nl.txt deleted file mode 100644 index 441072971d3..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stemdict_nl.txt +++ /dev/null @@ -1,6 +0,0 @@ -# Set of overrides for the dutch stemmer -# TODO: load this as a resource from the analyzer and sync it in build.xml -fiets fiets -bromfiets bromfiets -ei eier -kind kinder diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stoptags_ja.txt b/solr/example/example-DIH/solr/rss/conf/lang/stoptags_ja.txt deleted file mode 100644 index 71b750845e3..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stoptags_ja.txt +++ /dev/null @@ -1,420 +0,0 @@ -# -# This file defines a Japanese stoptag set for JapanesePartOfSpeechStopFilter. -# -# Any token with a part-of-speech tag that exactly matches those defined in this -# file are removed from the token stream. -# -# Set your own stoptags by uncommenting the lines below. Note that comments are -# not allowed on the same line as a stoptag. See LUCENE-3745 for frequency lists, -# etc. that can be useful for building you own stoptag set. -# -# The entire possible tagset is provided below for convenience. -# -##### -# noun: unclassified nouns -#名詞 -# -# noun-common: Common nouns or nouns where the sub-classification is undefined -#名詞-一般 -# -# noun-proper: Proper nouns where the sub-classification is undefined -#名詞-固有名詞 -# -# noun-proper-misc: miscellaneous proper nouns -#名詞-固有名詞-一般 -# -# noun-proper-person: Personal names where the sub-classification is undefined -#名詞-固有名詞-人名 -# -# noun-proper-person-misc: names that cannot be divided into surname and -# given name; foreign names; names where the surname or given name is unknown. -# e.g. お市の方 -#名詞-固有名詞-人名-一般 -# -# noun-proper-person-surname: Mainly Japanese surnames. -# e.g. 山田 -#名詞-固有名詞-人名-姓 -# -# noun-proper-person-given_name: Mainly Japanese given names. -# e.g. 太郎 -#名詞-固有名詞-人名-名 -# -# noun-proper-organization: Names representing organizations. -# e.g. 通産省, NHK -#名詞-固有名詞-組織 -# -# noun-proper-place: Place names where the sub-classification is undefined -#名詞-固有名詞-地域 -# -# noun-proper-place-misc: Place names excluding countries. -# e.g. アジア, バルセロナ, 京都 -#名詞-固有名詞-地域-一般 -# -# noun-proper-place-country: Country names. -# e.g. 日本, オーストラリア -#名詞-固有名詞-地域-国 -# -# noun-pronoun: Pronouns where the sub-classification is undefined -#名詞-代名詞 -# -# noun-pronoun-misc: miscellaneous pronouns: -# e.g. それ, ここ, あいつ, あなた, あちこち, いくつ, どこか, なに, みなさん, みんな, わたくし, われわれ -#名詞-代名詞-一般 -# -# noun-pronoun-contraction: Spoken language contraction made by combining a -# pronoun and the particle 'wa'. -# e.g. ありゃ, こりゃ, こりゃあ, そりゃ, そりゃあ -#名詞-代名詞-縮約 -# -# noun-adverbial: Temporal nouns such as names of days or months that behave -# like adverbs. Nouns that represent amount or ratios and can be used adverbially, -# e.g. 金曜, 一月, 午後, 少量 -#名詞-副詞可能 -# -# noun-verbal: Nouns that take arguments with case and can appear followed by -# 'suru' and related verbs (する, できる, なさる, くださる) -# e.g. インプット, 愛着, 悪化, 悪戦苦闘, 一安心, 下取り -#名詞-サ変接続 -# -# noun-adjective-base: The base form of adjectives, words that appear before な ("na") -# e.g. 健康, 安易, 駄目, だめ -#名詞-形容動詞語幹 -# -# noun-numeric: Arabic numbers, Chinese numerals, and counters like 何 (回), 数. -# e.g. 0, 1, 2, 何, 数, 幾 -#名詞-数 -# -# noun-affix: noun affixes where the sub-classification is undefined -#名詞-非自立 -# -# noun-affix-misc: Of adnominalizers, the case-marker の ("no"), and words that -# attach to the base form of inflectional words, words that cannot be classified -# into any of the other categories below. This category includes indefinite nouns. -# e.g. あかつき, 暁, かい, 甲斐, 気, きらい, 嫌い, くせ, 癖, こと, 事, ごと, 毎, しだい, 次第, -# 順, せい, 所為, ついで, 序で, つもり, 積もり, 点, どころ, の, はず, 筈, はずみ, 弾み, -# 拍子, ふう, ふり, 振り, ほう, 方, 旨, もの, 物, 者, ゆえ, 故, ゆえん, 所以, わけ, 訳, -# わり, 割り, 割, ん-口語/, もん-口語/ -#名詞-非自立-一般 -# -# noun-affix-adverbial: noun affixes that that can behave as adverbs. -# e.g. あいだ, 間, あげく, 挙げ句, あと, 後, 余り, 以外, 以降, 以後, 以上, 以前, 一方, うえ, -# 上, うち, 内, おり, 折り, かぎり, 限り, きり, っきり, 結果, ころ, 頃, さい, 際, 最中, さなか, -# 最中, じたい, 自体, たび, 度, ため, 為, つど, 都度, とおり, 通り, とき, 時, ところ, 所, -# とたん, 途端, なか, 中, のち, 後, ばあい, 場合, 日, ぶん, 分, ほか, 他, まえ, 前, まま, -# 儘, 侭, みぎり, 矢先 -#名詞-非自立-副詞可能 -# -# noun-affix-aux: noun affixes treated as 助動詞 ("auxiliary verb") in school grammars -# with the stem よう(だ) ("you(da)"). -# e.g. よう, やう, 様 (よう) -#名詞-非自立-助動詞語幹 -# -# noun-affix-adjective-base: noun affixes that can connect to the indeclinable -# connection form な (aux "da"). -# e.g. みたい, ふう -#名詞-非自立-形容動詞語幹 -# -# noun-special: special nouns where the sub-classification is undefined. -#名詞-特殊 -# -# noun-special-aux: The そうだ ("souda") stem form that is used for reporting news, is -# treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the base -# form of inflectional words. -# e.g. そう -#名詞-特殊-助動詞語幹 -# -# noun-suffix: noun suffixes where the sub-classification is undefined. -#名詞-接尾 -# -# noun-suffix-misc: Of the nouns or stem forms of other parts of speech that connect -# to ガル or タイ and can combine into compound nouns, words that cannot be classified into -# any of the other categories below. In general, this category is more inclusive than -# 接尾語 ("suffix") and is usually the last element in a compound noun. -# e.g. おき, かた, 方, 甲斐 (がい), がかり, ぎみ, 気味, ぐるみ, (~した) さ, 次第, 済 (ず) み, -# よう, (でき)っこ, 感, 観, 性, 学, 類, 面, 用 -#名詞-接尾-一般 -# -# noun-suffix-person: Suffixes that form nouns and attach to person names more often -# than other nouns. -# e.g. 君, 様, 著 -#名詞-接尾-人名 -# -# noun-suffix-place: Suffixes that form nouns and attach to place names more often -# than other nouns. -# e.g. 町, 市, 県 -#名詞-接尾-地域 -# -# noun-suffix-verbal: Of the suffixes that attach to nouns and form nouns, those that -# can appear before スル ("suru"). -# e.g. 化, 視, 分け, 入り, 落ち, 買い -#名詞-接尾-サ変接続 -# -# noun-suffix-aux: The stem form of そうだ (様態) that is used to indicate conditions, -# is treated as 助動詞 ("auxiliary verb") in school grammars, and attach to the -# conjunctive form of inflectional words. -# e.g. そう -#名詞-接尾-助動詞語幹 -# -# noun-suffix-adjective-base: Suffixes that attach to other nouns or the conjunctive -# form of inflectional words and appear before the copula だ ("da"). -# e.g. 的, げ, がち -#名詞-接尾-形容動詞語幹 -# -# noun-suffix-adverbial: Suffixes that attach to other nouns and can behave as adverbs. -# e.g. 後 (ご), 以後, 以降, 以前, 前後, 中, 末, 上, 時 (じ) -#名詞-接尾-副詞可能 -# -# noun-suffix-classifier: Suffixes that attach to numbers and form nouns. This category -# is more inclusive than 助数詞 ("classifier") and includes common nouns that attach -# to numbers. -# e.g. 個, つ, 本, 冊, パーセント, cm, kg, カ月, か国, 区画, 時間, 時半 -#名詞-接尾-助数詞 -# -# noun-suffix-special: Special suffixes that mainly attach to inflecting words. -# e.g. (楽し) さ, (考え) 方 -#名詞-接尾-特殊 -# -# noun-suffix-conjunctive: Nouns that behave like conjunctions and join two words -# together. -# e.g. (日本) 対 (アメリカ), 対 (アメリカ), (3) 対 (5), (女優) 兼 (主婦) -#名詞-接続詞的 -# -# noun-verbal_aux: Nouns that attach to the conjunctive particle て ("te") and are -# semantically verb-like. -# e.g. ごらん, ご覧, 御覧, 頂戴 -#名詞-動詞非自立的 -# -# noun-quotation: text that cannot be segmented into words, proverbs, Chinese poetry, -# dialects, English, etc. Currently, the only entry for 名詞 引用文字列 ("noun quotation") -# is いわく ("iwaku"). -#名詞-引用文字列 -# -# noun-nai_adjective: Words that appear before the auxiliary verb ない ("nai") and -# behave like an adjective. -# e.g. 申し訳, 仕方, とんでも, 違い -#名詞-ナイ形容詞語幹 -# -##### -# prefix: unclassified prefixes -#接頭詞 -# -# prefix-nominal: Prefixes that attach to nouns (including adjective stem forms) -# excluding numerical expressions. -# e.g. お (水), 某 (氏), 同 (社), 故 (~氏), 高 (品質), お (見事), ご (立派) -#接頭詞-名詞接続 -# -# prefix-verbal: Prefixes that attach to the imperative form of a verb or a verb -# in conjunctive form followed by なる/なさる/くださる. -# e.g. お (読みなさい), お (座り) -#接頭詞-動詞接続 -# -# prefix-adjectival: Prefixes that attach to adjectives. -# e.g. お (寒いですねえ), バカ (でかい) -#接頭詞-形容詞接続 -# -# prefix-numerical: Prefixes that attach to numerical expressions. -# e.g. 約, およそ, 毎時 -#接頭詞-数接続 -# -##### -# verb: unclassified verbs -#動詞 -# -# verb-main: -#動詞-自立 -# -# verb-auxiliary: -#動詞-非自立 -# -# verb-suffix: -#動詞-接尾 -# -##### -# adjective: unclassified adjectives -#形容詞 -# -# adjective-main: -#形容詞-自立 -# -# adjective-auxiliary: -#形容詞-非自立 -# -# adjective-suffix: -#形容詞-接尾 -# -##### -# adverb: unclassified adverbs -#副詞 -# -# adverb-misc: Words that can be segmented into one unit and where adnominal -# modification is not possible. -# e.g. あいかわらず, 多分 -#副詞-一般 -# -# adverb-particle_conjunction: Adverbs that can be followed by の, は, に, -# な, する, だ, etc. -# e.g. こんなに, そんなに, あんなに, なにか, なんでも -#副詞-助詞類接続 -# -##### -# adnominal: Words that only have noun-modifying forms. -# e.g. この, その, あの, どの, いわゆる, なんらかの, 何らかの, いろんな, こういう, そういう, ああいう, -# どういう, こんな, そんな, あんな, どんな, 大きな, 小さな, おかしな, ほんの, たいした, -# 「(, も) さる (ことながら)」, 微々たる, 堂々たる, 単なる, いかなる, 我が」「同じ, 亡き -#連体詞 -# -##### -# conjunction: Conjunctions that can occur independently. -# e.g. が, けれども, そして, じゃあ, それどころか -接続詞 -# -##### -# particle: unclassified particles. -助詞 -# -# particle-case: case particles where the subclassification is undefined. -助詞-格助詞 -# -# particle-case-misc: Case particles. -# e.g. から, が, で, と, に, へ, より, を, の, にて -助詞-格助詞-一般 -# -# particle-case-quote: the "to" that appears after nouns, a person’s speech, -# quotation marks, expressions of decisions from a meeting, reasons, judgements, -# conjectures, etc. -# e.g. ( だ) と (述べた.), ( である) と (して執行猶予...) -助詞-格助詞-引用 -# -# particle-case-compound: Compounds of particles and verbs that mainly behave -# like case particles. -# e.g. という, といった, とかいう, として, とともに, と共に, でもって, にあたって, に当たって, に当って, -# にあたり, に当たり, に当り, に当たる, にあたる, において, に於いて,に於て, における, に於ける, -# にかけ, にかけて, にかんし, に関し, にかんして, に関して, にかんする, に関する, に際し, -# に際して, にしたがい, に従い, に従う, にしたがって, に従って, にたいし, に対し, にたいして, -# に対して, にたいする, に対する, について, につき, につけ, につけて, につれ, につれて, にとって, -# にとり, にまつわる, によって, に依って, に因って, により, に依り, に因り, による, に依る, に因る, -# にわたって, にわたる, をもって, を以って, を通じ, を通じて, を通して, をめぐって, をめぐり, をめぐる, -# って-口語/, ちゅう-関西弁「という」/, (何) ていう (人)-口語/, っていう-口語/, といふ, とかいふ -助詞-格助詞-連語 -# -# particle-conjunctive: -# e.g. から, からには, が, けれど, けれども, けど, し, つつ, て, で, と, ところが, どころか, とも, ども, -# ながら, なり, ので, のに, ば, ものの, や ( した), やいなや, (ころん) じゃ(いけない)-口語/, -# (行っ) ちゃ(いけない)-口語/, (言っ) たって (しかたがない)-口語/, (それがなく)ったって (平気)-口語/ -助詞-接続助詞 -# -# particle-dependency: -# e.g. こそ, さえ, しか, すら, は, も, ぞ -助詞-係助詞 -# -# particle-adverbial: -# e.g. がてら, かも, くらい, 位, ぐらい, しも, (学校) じゃ(これが流行っている)-口語/, -# (それ)じゃあ (よくない)-口語/, ずつ, (私) なぞ, など, (私) なり (に), (先生) なんか (大嫌い)-口語/, -# (私) なんぞ, (先生) なんて (大嫌い)-口語/, のみ, だけ, (私) だって-口語/, だに, -# (彼)ったら-口語/, (お茶) でも (いかが), 等 (とう), (今後) とも, ばかり, ばっか-口語/, ばっかり-口語/, -# ほど, 程, まで, 迄, (誰) も (が)([助詞-格助詞] および [助詞-係助詞] の前に位置する「も」) -助詞-副助詞 -# -# particle-interjective: particles with interjective grammatical roles. -# e.g. (松島) や -助詞-間投助詞 -# -# particle-coordinate: -# e.g. と, たり, だの, だり, とか, なり, や, やら -助詞-並立助詞 -# -# particle-final: -# e.g. かい, かしら, さ, ぜ, (だ)っけ-口語/, (とまってる) で-方言/, な, ナ, なあ-口語/, ぞ, ね, ネ, -# ねぇ-口語/, ねえ-口語/, ねん-方言/, の, のう-口語/, や, よ, ヨ, よぉ-口語/, わ, わい-口語/ -助詞-終助詞 -# -# particle-adverbial/conjunctive/final: The particle "ka" when unknown whether it is -# adverbial, conjunctive, or sentence final. For example: -# (a) 「A か B か」. Ex:「(国内で運用する) か,(海外で運用する) か (.)」 -# (b) Inside an adverb phrase. Ex:「(幸いという) か (, 死者はいなかった.)」 -# 「(祈りが届いたせい) か (, 試験に合格した.)」 -# (c) 「かのように」. Ex:「(何もなかった) か (のように振る舞った.)」 -# e.g. か -助詞-副助詞/並立助詞/終助詞 -# -# particle-adnominalizer: The "no" that attaches to nouns and modifies -# non-inflectional words. -助詞-連体化 -# -# particle-adnominalizer: The "ni" and "to" that appear following nouns and adverbs -# that are giongo, giseigo, or gitaigo. -# e.g. に, と -助詞-副詞化 -# -# particle-special: A particle that does not fit into one of the above classifications. -# This includes particles that are used in Tanka, Haiku, and other poetry. -# e.g. かな, けむ, ( しただろう) に, (あんた) にゃ(わからん), (俺) ん (家) -助詞-特殊 -# -##### -# auxiliary-verb: -助動詞 -# -##### -# interjection: Greetings and other exclamations. -# e.g. おはよう, おはようございます, こんにちは, こんばんは, ありがとう, どうもありがとう, ありがとうございます, -# いただきます, ごちそうさま, さよなら, さようなら, はい, いいえ, ごめん, ごめんなさい -#感動詞 -# -##### -# symbol: unclassified Symbols. -記号 -# -# symbol-misc: A general symbol not in one of the categories below. -# e.g. [○◎@$〒→+] -記号-一般 -# -# symbol-comma: Commas -# e.g. [,、] -記号-読点 -# -# symbol-period: Periods and full stops. -# e.g. [..。] -記号-句点 -# -# symbol-space: Full-width whitespace. -記号-空白 -# -# symbol-open_bracket: -# e.g. [({‘“『【] -記号-括弧開 -# -# symbol-close_bracket: -# e.g. [)}’”』」】] -記号-括弧閉 -# -# symbol-alphabetic: -#記号-アルファベット -# -##### -# other: unclassified other -#その他 -# -# other-interjection: Words that are hard to classify as noun-suffixes or -# sentence-final particles. -# e.g. (だ)ァ -その他-間投 -# -##### -# filler: Aizuchi that occurs during a conversation or sounds inserted as filler. -# e.g. あの, うんと, えと -フィラー -# -##### -# non-verbal: non-verbal sound. -非言語音 -# -##### -# fragment: -#語断片 -# -##### -# unknown: unknown part of speech. -#未知語 -# -##### End of file diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ar.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ar.txt deleted file mode 100644 index 046829db6a2..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ar.txt +++ /dev/null @@ -1,125 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Cleaned on October 11, 2009 (not normalized, so use before normalization) -# This means that when modifying this list, you might need to add some -# redundant entries, for example containing forms with both أ and ا -من -ومن -منها -منه -في -وفي -فيها -فيه -و -ف -ثم -او -أو -ب -بها -به -ا -أ -اى -اي -أي -أى -لا -ولا -الا -ألا -إلا -لكن -ما -وما -كما -فما -عن -مع -اذا -إذا -ان -أن -إن -انها -أنها -إنها -انه -أنه -إنه -بان -بأن -فان -فأن -وان -وأن -وإن -التى -التي -الذى -الذي -الذين -الى -الي -إلى -إلي -على -عليها -عليه -اما -أما -إما -ايضا -أيضا -كل -وكل -لم -ولم -لن -ولن -هى -هي -هو -وهى -وهي -وهو -فهى -فهي -فهو -انت -أنت -لك -لها -له -هذه -هذا -تلك -ذلك -هناك -كانت -كان -يكون -تكون -وكانت -وكان -غير -بعض -قد -نحو -بين -بينما -منذ -ضمن -حيث -الان -الآن -خلال -بعد -قبل -حتى -عند -عندما -لدى -جميع diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_bg.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_bg.txt deleted file mode 100644 index 1ae4ba2ae38..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_bg.txt +++ /dev/null @@ -1,193 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -а -аз -ако -ала -бе -без -беше -би -бил -била -били -било -близо -бъдат -бъде -бяха -в -вас -ваш -ваша -вероятно -вече -взема -ви -вие -винаги -все -всеки -всички -всичко -всяка -във -въпреки -върху -г -ги -главно -го -д -да -дали -до -докато -докога -дори -досега -доста -е -едва -един -ето -за -зад -заедно -заради -засега -затова -защо -защото -и -из -или -им -има -имат -иска -й -каза -как -каква -какво -както -какъв -като -кога -когато -което -които -кой -който -колко -която -къде -където -към -ли -м -ме -между -мен -ми -мнозина -мога -могат -може -моля -момента -му -н -на -над -назад -най -направи -напред -например -нас -не -него -нея -ни -ние -никой -нито -но -някои -някой -няма -обаче -около -освен -особено -от -отгоре -отново -още -пак -по -повече -повечето -под -поне -поради -после -почти -прави -пред -преди -през -при -пък -първо -с -са -само -се -сега -си -скоро -след -сме -според -сред -срещу -сте -съм -със -също -т -тази -така -такива -такъв -там -твой -те -тези -ти -тн -то -това -тогава -този -той -толкова -точно -трябва -тук -тъй -тя -тях -у -харесва -ч -че -често -чрез -ще -щом -я diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ca.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ca.txt deleted file mode 100644 index 3da65deafe1..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ca.txt +++ /dev/null @@ -1,220 +0,0 @@ -# Catalan stopwords from http://github.com/vcl/cue.language (Apache 2 Licensed) -a -abans -ací -ah -així -això -al -als -aleshores -algun -alguna -algunes -alguns -alhora -allà -allí -allò -altra -altre -altres -amb -ambdós -ambdues -apa -aquell -aquella -aquelles -aquells -aquest -aquesta -aquestes -aquests -aquí -baix -cada -cadascú -cadascuna -cadascunes -cadascuns -com -contra -d'un -d'una -d'unes -d'uns -dalt -de -del -dels -des -després -dins -dintre -donat -doncs -durant -e -eh -el -els -em -en -encara -ens -entre -érem -eren -éreu -es -és -esta -està -estàvem -estaven -estàveu -esteu -et -etc -ets -fins -fora -gairebé -ha -han -has -havia -he -hem -heu -hi -ho -i -igual -iguals -ja -l'hi -la -les -li -li'n -llavors -m'he -ma -mal -malgrat -mateix -mateixa -mateixes -mateixos -me -mentre -més -meu -meus -meva -meves -molt -molta -moltes -molts -mon -mons -n'he -n'hi -ne -ni -no -nogensmenys -només -nosaltres -nostra -nostre -nostres -o -oh -oi -on -pas -pel -pels -per -però -perquè -poc -poca -pocs -poques -potser -propi -qual -quals -quan -quant -que -què -quelcom -qui -quin -quina -quines -quins -s'ha -s'han -sa -semblant -semblants -ses -seu -seus -seva -seva -seves -si -sobre -sobretot -sóc -solament -sols -son -són -sons -sota -sou -t'ha -t'han -t'he -ta -tal -també -tampoc -tan -tant -tanta -tantes -teu -teus -teva -teves -ton -tons -tot -tota -totes -tots -un -una -unes -uns -us -va -vaig -vam -van -vas -veu -vosaltres -vostra -vostre -vostres diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ckb.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ckb.txt deleted file mode 100644 index 87abf118fec..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ckb.txt +++ /dev/null @@ -1,136 +0,0 @@ -# set of kurdish stopwords -# note these have been normalized with our scheme (e represented with U+06D5, etc) -# constructed from: -# * Fig 5 of "Building A Test Collection For Sorani Kurdish" (Esmaili et al) -# * "Sorani Kurdish: A Reference Grammar with selected readings" (Thackston) -# * Corpus-based analysis of 77M word Sorani collection: wikipedia, news, blogs, etc - -# and -و -# which -کە -# of -ی -# made/did -کرد -# that/which -ئەوەی -# on/head -سەر -# two -دوو -# also -هەروەها -# from/that -لەو -# makes/does -دەکات -# some -چەند -# every -هەر - -# demonstratives -# that -ئەو -# this -ئەم - -# personal pronouns -# I -من -# we -ئێمە -# you -تۆ -# you -ئێوە -# he/she/it -ئەو -# they -ئەوان - -# prepositions -# to/with/by -بە -پێ -# without -بەبێ -# along with/while/during -بەدەم -# in the opinion of -بەلای -# according to -بەپێی -# before -بەرلە -# in the direction of -بەرەوی -# in front of/toward -بەرەوە -# before/in the face of -بەردەم -# without -بێ -# except for -بێجگە -# for -بۆ -# on/in -دە -تێ -# with -دەگەڵ -# after -دوای -# except for/aside from -جگە -# in/from -لە -لێ -# in front of/before/because of -لەبەر -# between/among -لەبەینی -# concerning/about -لەبابەت -# concerning -لەبارەی -# instead of -لەباتی -# beside -لەبن -# instead of -لەبرێتی -# behind -لەدەم -# with/together with -لەگەڵ -# by -لەلایەن -# within -لەناو -# between/among -لەنێو -# for the sake of -لەپێناوی -# with respect to -لەرەوی -# by means of/for -لەرێ -# for the sake of -لەرێگا -# on/on top of/according to -لەسەر -# under -لەژێر -# between/among -ناو -# between/among -نێوان -# after -پاش -# before -پێش -# like -وەک diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_cz.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_cz.txt deleted file mode 100644 index 53c6097dac7..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_cz.txt +++ /dev/null @@ -1,172 +0,0 @@ -a -s -k -o -i -u -v -z -dnes -cz -tímto -budeš -budem -byli -jseš -můj -svým -ta -tomto -tohle -tuto -tyto -jej -zda -proč -máte -tato -kam -tohoto -kdo -kteří -mi -nám -tom -tomuto -mít -nic -proto -kterou -byla -toho -protože -asi -ho -naši -napište -re -což -tím -takže -svých -její -svými -jste -aj -tu -tedy -teto -bylo -kde -ke -pravé -ji -nad -nejsou -či -pod -téma -mezi -přes -ty -pak -vám -ani -když -však -neg -jsem -tento -článku -články -aby -jsme -před -pta -jejich -byl -ještě -až -bez -také -pouze -první -vaše -která -nás -nový -tipy -pokud -může -strana -jeho -své -jiné -zprávy -nové -není -vás -jen -podle -zde -už -být -více -bude -již -než -který -by -které -co -nebo -ten -tak -má -při -od -po -jsou -jak -další -ale -si -se -ve -to -jako -za -zpět -ze -do -pro -je -na -atd -atp -jakmile -přičemž -já -on -ona -ono -oni -ony -my -vy -jí -ji -mě -mne -jemu -tomu -těm -těmu -němu -němuž -jehož -jíž -jelikož -jež -jakož -načež diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_da.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_da.txt deleted file mode 100644 index 42e6145b98e..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_da.txt +++ /dev/null @@ -1,110 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/danish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | A Danish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - -og | and -i | in -jeg | I -det | that (dem. pronoun)/it (pers. pronoun) -at | that (in front of a sentence)/to (with infinitive) -en | a/an -den | it (pers. pronoun)/that (dem. pronoun) -til | to/at/for/until/against/by/of/into, more -er | present tense of "to be" -som | who, as -på | on/upon/in/on/at/to/after/of/with/for, on -de | they -med | with/by/in, along -han | he -af | of/by/from/off/for/in/with/on, off -for | at/for/to/from/by/of/ago, in front/before, because -ikke | not -der | who/which, there/those -var | past tense of "to be" -mig | me/myself -sig | oneself/himself/herself/itself/themselves -men | but -et | a/an/one, one (number), someone/somebody/one -har | present tense of "to have" -om | round/about/for/in/a, about/around/down, if -vi | we -min | my -havde | past tense of "to have" -ham | him -hun | she -nu | now -over | over/above/across/by/beyond/past/on/about, over/past -da | then, when/as/since -fra | from/off/since, off, since -du | you -ud | out -sin | his/her/its/one's -dem | them -os | us/ourselves -op | up -man | you/one -hans | his -hvor | where -eller | or -hvad | what -skal | must/shall etc. -selv | myself/youself/herself/ourselves etc., even -her | here -alle | all/everyone/everybody etc. -vil | will (verb) -blev | past tense of "to stay/to remain/to get/to become" -kunne | could -ind | in -når | when -være | present tense of "to be" -dog | however/yet/after all -noget | something -ville | would -jo | you know/you see (adv), yes -deres | their/theirs -efter | after/behind/according to/for/by/from, later/afterwards -ned | down -skulle | should -denne | this -end | than -dette | this -mit | my/mine -også | also -under | under/beneath/below/during, below/underneath -have | have -dig | you -anden | other -hende | her -mine | my -alt | everything -meget | much/very, plenty of -sit | his, her, its, one's -sine | his, her, its, one's -vor | our -mod | against -disse | these -hvis | if -din | your/yours -nogle | some -hos | by/at -blive | be/become -mange | many -ad | by/through -bliver | present tense of "to be/to become" -hendes | her/hers -været | be -thi | for (conj) -jer | you -sådan | such, like this/like that diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_de.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_de.txt deleted file mode 100644 index 86525e7ae08..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_de.txt +++ /dev/null @@ -1,294 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/german/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | A German stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | The number of forms in this list is reduced significantly by passing it - | through the German stemmer. - - -aber | but - -alle | all -allem -allen -aller -alles - -als | than, as -also | so -am | an + dem -an | at - -ander | other -andere -anderem -anderen -anderer -anderes -anderm -andern -anderr -anders - -auch | also -auf | on -aus | out of -bei | by -bin | am -bis | until -bist | art -da | there -damit | with it -dann | then - -der | the -den -des -dem -die -das - -daß | that - -derselbe | the same -derselben -denselben -desselben -demselben -dieselbe -dieselben -dasselbe - -dazu | to that - -dein | thy -deine -deinem -deinen -deiner -deines - -denn | because - -derer | of those -dessen | of him - -dich | thee -dir | to thee -du | thou - -dies | this -diese -diesem -diesen -dieser -dieses - - -doch | (several meanings) -dort | (over) there - - -durch | through - -ein | a -eine -einem -einen -einer -eines - -einig | some -einige -einigem -einigen -einiger -einiges - -einmal | once - -er | he -ihn | him -ihm | to him - -es | it -etwas | something - -euer | your -eure -eurem -euren -eurer -eures - -für | for -gegen | towards -gewesen | p.p. of sein -hab | have -habe | have -haben | have -hat | has -hatte | had -hatten | had -hier | here -hin | there -hinter | behind - -ich | I -mich | me -mir | to me - - -ihr | you, to her -ihre -ihrem -ihren -ihrer -ihres -euch | to you - -im | in + dem -in | in -indem | while -ins | in + das -ist | is - -jede | each, every -jedem -jeden -jeder -jedes - -jene | that -jenem -jenen -jener -jenes - -jetzt | now -kann | can - -kein | no -keine -keinem -keinen -keiner -keines - -können | can -könnte | could -machen | do -man | one - -manche | some, many a -manchem -manchen -mancher -manches - -mein | my -meine -meinem -meinen -meiner -meines - -mit | with -muss | must -musste | had to -nach | to(wards) -nicht | not -nichts | nothing -noch | still, yet -nun | now -nur | only -ob | whether -oder | or -ohne | without -sehr | very - -sein | his -seine -seinem -seinen -seiner -seines - -selbst | self -sich | herself - -sie | they, she -ihnen | to them - -sind | are -so | so - -solche | such -solchem -solchen -solcher -solches - -soll | shall -sollte | should -sondern | but -sonst | else -über | over -um | about, around -und | and - -uns | us -unse -unsem -unsen -unser -unses - -unter | under -viel | much -vom | von + dem -von | from -vor | before -während | while -war | was -waren | were -warst | wast -was | what -weg | away, off -weil | because -weiter | further - -welche | which -welchem -welchen -welcher -welches - -wenn | when -werde | will -werden | will -wie | how -wieder | again -will | want -wir | we -wird | will -wirst | willst -wo | where -wollen | want -wollte | wanted -würde | would -würden | would -zu | to -zum | zu + dem -zur | zu + der -zwar | indeed -zwischen | between - diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_el.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_el.txt deleted file mode 100644 index 232681f5bd6..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_el.txt +++ /dev/null @@ -1,78 +0,0 @@ -# Lucene Greek Stopwords list -# Note: by default this file is used after GreekLowerCaseFilter, -# so when modifying this file use 'σ' instead of 'ς' -ο -η -το -οι -τα -του -τησ -των -τον -την -και -κι -κ -ειμαι -εισαι -ειναι -ειμαστε -ειστε -στο -στον -στη -στην -μα -αλλα -απο -για -προσ -με -σε -ωσ -παρα -αντι -κατα -μετα -θα -να -δε -δεν -μη -μην -επι -ενω -εαν -αν -τοτε -που -πωσ -ποιοσ -ποια -ποιο -ποιοι -ποιεσ -ποιων -ποιουσ -αυτοσ -αυτη -αυτο -αυτοι -αυτων -αυτουσ -αυτεσ -αυτα -εκεινοσ -εκεινη -εκεινο -εκεινοι -εκεινεσ -εκεινα -εκεινων -εκεινουσ -οπωσ -ομωσ -ισωσ -οσο -οτι diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_es.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_es.txt deleted file mode 100644 index 487d78c8d56..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_es.txt +++ /dev/null @@ -1,356 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/spanish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | A Spanish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | from, of -la | the, her -que | who, that -el | the -en | in -y | and -a | to -los | the, them -del | de + el -se | himself, from him etc -las | the, them -por | for, by, etc -un | a -para | for -con | with -no | no -una | a -su | his, her -al | a + el - | es from SER -lo | him -como | how -más | more -pero | pero -sus | su plural -le | to him, her -ya | already -o | or - | fue from SER -este | this - | ha from HABER -sí | himself etc -porque | because -esta | this - | son from SER -entre | between - | está from ESTAR -cuando | when -muy | very -sin | without -sobre | on - | ser from SER - | tiene from TENER -también | also -me | me -hasta | until -hay | there is/are -donde | where - | han from HABER -quien | whom, that - | están from ESTAR - | estado from ESTAR -desde | from -todo | all -nos | us -durante | during - | estados from ESTAR -todos | all -uno | a -les | to them -ni | nor -contra | against -otros | other - | fueron from SER -ese | that -eso | that - | había from HABER -ante | before -ellos | they -e | and (variant of y) -esto | this -mí | me -antes | before -algunos | some -qué | what? -unos | a -yo | I -otro | other -otras | other -otra | other -él | he -tanto | so much, many -esa | that -estos | these -mucho | much, many -quienes | who -nada | nothing -muchos | many -cual | who - | sea from SER -poco | few -ella | she -estar | to be - | haber from HABER -estas | these - | estaba from ESTAR - | estamos from ESTAR -algunas | some -algo | something -nosotros | we - - | other forms - -mi | me -mis | mi plural -tú | thou -te | thee -ti | thee -tu | thy -tus | tu plural -ellas | they -nosotras | we -vosotros | you -vosotras | you -os | you -mío | mine -mía | -míos | -mías | -tuyo | thine -tuya | -tuyos | -tuyas | -suyo | his, hers, theirs -suya | -suyos | -suyas | -nuestro | ours -nuestra | -nuestros | -nuestras | -vuestro | yours -vuestra | -vuestros | -vuestras | -esos | those -esas | those - - | forms of estar, to be (not including the infinitive): -estoy -estás -está -estamos -estáis -están -esté -estés -estemos -estéis -estén -estaré -estarás -estará -estaremos -estaréis -estarán -estaría -estarías -estaríamos -estaríais -estarían -estaba -estabas -estábamos -estabais -estaban -estuve -estuviste -estuvo -estuvimos -estuvisteis -estuvieron -estuviera -estuvieras -estuviéramos -estuvierais -estuvieran -estuviese -estuvieses -estuviésemos -estuvieseis -estuviesen -estando -estado -estada -estados -estadas -estad - - | forms of haber, to have (not including the infinitive): -he -has -ha -hemos -habéis -han -haya -hayas -hayamos -hayáis -hayan -habré -habrás -habrá -habremos -habréis -habrán -habría -habrías -habríamos -habríais -habrían -había -habías -habíamos -habíais -habían -hube -hubiste -hubo -hubimos -hubisteis -hubieron -hubiera -hubieras -hubiéramos -hubierais -hubieran -hubiese -hubieses -hubiésemos -hubieseis -hubiesen -habiendo -habido -habida -habidos -habidas - - | forms of ser, to be (not including the infinitive): -soy -eres -es -somos -sois -son -sea -seas -seamos -seáis -sean -seré -serás -será -seremos -seréis -serán -sería -serías -seríamos -seríais -serían -era -eras -éramos -erais -eran -fui -fuiste -fue -fuimos -fuisteis -fueron -fuera -fueras -fuéramos -fuerais -fueran -fuese -fueses -fuésemos -fueseis -fuesen -siendo -sido - | sed also means 'thirst' - - | forms of tener, to have (not including the infinitive): -tengo -tienes -tiene -tenemos -tenéis -tienen -tenga -tengas -tengamos -tengáis -tengan -tendré -tendrás -tendrá -tendremos -tendréis -tendrán -tendría -tendrías -tendríamos -tendríais -tendrían -tenía -tenías -teníamos -teníais -tenían -tuve -tuviste -tuvo -tuvimos -tuvisteis -tuvieron -tuviera -tuvieras -tuviéramos -tuvierais -tuvieran -tuviese -tuvieses -tuviésemos -tuvieseis -tuviesen -teniendo -tenido -tenida -tenidos -tenidas -tened - diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_eu.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_eu.txt deleted file mode 100644 index 25f1db93460..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_eu.txt +++ /dev/null @@ -1,99 +0,0 @@ -# example set of basque stopwords -al -anitz -arabera -asko -baina -bat -batean -batek -bati -batzuei -batzuek -batzuetan -batzuk -bera -beraiek -berau -berauek -bere -berori -beroriek -beste -bezala -da -dago -dira -ditu -du -dute -edo -egin -ere -eta -eurak -ez -gainera -gu -gutxi -guzti -haiei -haiek -haietan -hainbeste -hala -han -handik -hango -hara -hari -hark -hartan -hau -hauei -hauek -hauetan -hemen -hemendik -hemengo -hi -hona -honek -honela -honetan -honi -hor -hori -horiei -horiek -horietan -horko -horra -horrek -horrela -horretan -horri -hortik -hura -izan -ni -noiz -nola -non -nondik -nongo -nor -nora -ze -zein -zen -zenbait -zenbat -zer -zergatik -ziren -zituen -zu -zuek -zuen -zuten diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_fa.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_fa.txt deleted file mode 100644 index 723641c6da7..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_fa.txt +++ /dev/null @@ -1,313 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -# Note: by default this file is used after normalization, so when adding entries -# to this file, use the arabic 'ي' instead of 'ی' -انان -نداشته -سراسر -خياه -ايشان -وي -تاكنون -بيشتري -دوم -پس -ناشي -وگو -يا -داشتند -سپس -هنگام -هرگز -پنج -نشان -امسال -ديگر -گروهي -شدند -چطور -ده -و -دو -نخستين -ولي -چرا -چه -وسط -ه -كدام -قابل -يك -رفت -هفت -همچنين -در -هزار -بله -بلي -شايد -اما -شناسي -گرفته -دهد -داشته -دانست -داشتن -خواهيم -ميليارد -وقتيكه -امد -خواهد -جز -اورده -شده -بلكه -خدمات -شدن -برخي -نبود -بسياري -جلوگيري -حق -كردند -نوعي -بعري -نكرده -نظير -نبايد -بوده -بودن -داد -اورد -هست -جايي -شود -دنبال -داده -بايد -سابق -هيچ -همان -انجا -كمتر -كجاست -گردد -كسي -تر -مردم -تان -دادن -بودند -سري -جدا -ندارند -مگر -يكديگر -دارد -دهند -بنابراين -هنگامي -سمت -جا -انچه -خود -دادند -زياد -دارند -اثر -بدون -بهترين -بيشتر -البته -به -براساس -بيرون -كرد -بعضي -گرفت -توي -اي -ميليون -او -جريان -تول -بر -مانند -برابر -باشيم -مدتي -گويند -اكنون -تا -تنها -جديد -چند -بي -نشده -كردن -كردم -گويد -كرده -كنيم -نمي -نزد -روي -قصد -فقط -بالاي -ديگران -اين -ديروز -توسط -سوم -ايم -دانند -سوي -استفاده -شما -كنار -داريم -ساخته -طور -امده -رفته -نخست -بيست -نزديك -طي -كنيد -از -انها -تمامي -داشت -يكي -طريق -اش -چيست -روب -نمايد -گفت -چندين -چيزي -تواند -ام -ايا -با -ان -ايد -ترين -اينكه -ديگري -راه -هايي -بروز -همچنان -پاعين -كس -حدود -مختلف -مقابل -چيز -گيرد -ندارد -ضد -همچون -سازي -شان -مورد -باره -مرسي -خويش -برخوردار -چون -خارج -شش -هنوز -تحت -ضمن -هستيم -گفته -فكر -بسيار -پيش -براي -روزهاي -انكه -نخواهد -بالا -كل -وقتي -كي -چنين -كه -گيري -نيست -است -كجا -كند -نيز -يابد -بندي -حتي -توانند -عقب -خواست -كنند -بين -تمام -همه -ما -باشند -مثل -شد -اري -باشد -اره -طبق -بعد -اگر -صورت -غير -جاي -بيش -ريزي -اند -زيرا -چگونه -بار -لطفا -مي -درباره -من -ديده -همين -گذاري -برداري -علت -گذاشته -هم -فوق -نه -ها -شوند -اباد -همواره -هر -اول -خواهند -چهار -نام -امروز -مان -هاي -قبل -كنم -سعي -تازه -را -هستند -زير -جلوي -عنوان -بود diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_fi.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_fi.txt deleted file mode 100644 index 4372c9a055b..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_fi.txt +++ /dev/null @@ -1,97 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/finnish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - -| forms of BE - -olla -olen -olet -on -olemme -olette -ovat -ole | negative form - -oli -olisi -olisit -olisin -olisimme -olisitte -olisivat -olit -olin -olimme -olitte -olivat -ollut -olleet - -en | negation -et -ei -emme -ette -eivät - -|Nom Gen Acc Part Iness Elat Illat Adess Ablat Allat Ess Trans -minä minun minut minua minussa minusta minuun minulla minulta minulle | I -sinä sinun sinut sinua sinussa sinusta sinuun sinulla sinulta sinulle | you -hän hänen hänet häntä hänessä hänestä häneen hänellä häneltä hänelle | he she -me meidän meidät meitä meissä meistä meihin meillä meiltä meille | we -te teidän teidät teitä teissä teistä teihin teillä teiltä teille | you -he heidän heidät heitä heissä heistä heihin heillä heiltä heille | they - -tämä tämän tätä tässä tästä tähän tallä tältä tälle tänä täksi | this -tuo tuon tuotä tuossa tuosta tuohon tuolla tuolta tuolle tuona tuoksi | that -se sen sitä siinä siitä siihen sillä siltä sille sinä siksi | it -nämä näiden näitä näissä näistä näihin näillä näiltä näille näinä näiksi | these -nuo noiden noita noissa noista noihin noilla noilta noille noina noiksi | those -ne niiden niitä niissä niistä niihin niillä niiltä niille niinä niiksi | they - -kuka kenen kenet ketä kenessä kenestä keneen kenellä keneltä kenelle kenenä keneksi| who -ketkä keiden ketkä keitä keissä keistä keihin keillä keiltä keille keinä keiksi | (pl) -mikä minkä minkä mitä missä mistä mihin millä miltä mille minä miksi | which what -mitkä | (pl) - -joka jonka jota jossa josta johon jolla jolta jolle jona joksi | who which -jotka joiden joita joissa joista joihin joilla joilta joille joina joiksi | (pl) - -| conjunctions - -että | that -ja | and -jos | if -koska | because -kuin | than -mutta | but -niin | so -sekä | and -sillä | for -tai | or -vaan | but -vai | or -vaikka | although - - -| prepositions - -kanssa | with -mukaan | according to -noin | about -poikki | across -yli | over, across - -| other - -kun | when -niin | so -nyt | now -itse | self - diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_fr.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_fr.txt deleted file mode 100644 index 749abae6846..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_fr.txt +++ /dev/null @@ -1,186 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/french/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | A French stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -au | a + le -aux | a + les -avec | with -ce | this -ces | these -dans | with -de | of -des | de + les -du | de + le -elle | she -en | `of them' etc -et | and -eux | them -il | he -je | I -la | the -le | the -leur | their -lui | him -ma | my (fem) -mais | but -me | me -même | same; as in moi-même (myself) etc -mes | me (pl) -moi | me -mon | my (masc) -ne | not -nos | our (pl) -notre | our -nous | we -on | one -ou | where -par | by -pas | not -pour | for -qu | que before vowel -que | that -qui | who -sa | his, her (fem) -se | oneself -ses | his (pl) -son | his, her (masc) -sur | on -ta | thy (fem) -te | thee -tes | thy (pl) -toi | thee -ton | thy (masc) -tu | thou -un | a -une | a -vos | your (pl) -votre | your -vous | you - - | single letter forms - -c | c' -d | d' -j | j' -l | l' -à | to, at -m | m' -n | n' -s | s' -t | t' -y | there - - | forms of être (not including the infinitive): -été -étée -étées -étés -étant -suis -es -est -sommes -êtes -sont -serai -seras -sera -serons -serez -seront -serais -serait -serions -seriez -seraient -étais -était -étions -étiez -étaient -fus -fut -fûmes -fûtes -furent -sois -soit -soyons -soyez -soient -fusse -fusses -fût -fussions -fussiez -fussent - - | forms of avoir (not including the infinitive): -ayant -eu -eue -eues -eus -ai -as -avons -avez -ont -aurai -auras -aura -aurons -aurez -auront -aurais -aurait -aurions -auriez -auraient -avais -avait -avions -aviez -avaient -eut -eûmes -eûtes -eurent -aie -aies -ait -ayons -ayez -aient -eusse -eusses -eût -eussions -eussiez -eussent - - | Later additions (from Jean-Christophe Deschamps) -ceci | this -cela | that -celà | that -cet | this -cette | this -ici | here -ils | they -les | the (pl) -leurs | their (pl) -quel | which -quels | which -quelle | which -quelles | which -sans | without -soi | oneself - diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ga.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ga.txt deleted file mode 100644 index 9ff88d747e5..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ga.txt +++ /dev/null @@ -1,110 +0,0 @@ - -a -ach -ag -agus -an -aon -ar -arna -as -b' -ba -beirt -bhúr -caoga -ceathair -ceathrar -chomh -chtó -chuig -chun -cois -céad -cúig -cúigear -d' -daichead -dar -de -deich -deichniúr -den -dhá -do -don -dtí -dá -dár -dó -faoi -faoin -faoina -faoinár -fara -fiche -gach -gan -go -gur -haon -hocht -i -iad -idir -in -ina -ins -inár -is -le -leis -lena -lenár -m' -mar -mo -mé -na -nach -naoi -naonúr -ná -ní -níor -nó -nócha -ocht -ochtar -os -roimh -sa -seacht -seachtar -seachtó -seasca -seisear -siad -sibh -sinn -sna -sé -sí -tar -thar -thú -triúr -trí -trína -trínár -tríocha -tú -um -ár -é -éis -í -ó -ón -óna -ónár diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_gl.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_gl.txt deleted file mode 100644 index d8760b12c14..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_gl.txt +++ /dev/null @@ -1,161 +0,0 @@ -# galican stopwords -a -aínda -alí -aquel -aquela -aquelas -aqueles -aquilo -aquí -ao -aos -as -así -á -ben -cando -che -co -coa -comigo -con -connosco -contigo -convosco -coas -cos -cun -cuns -cunha -cunhas -da -dalgunha -dalgunhas -dalgún -dalgúns -das -de -del -dela -delas -deles -desde -deste -do -dos -dun -duns -dunha -dunhas -e -el -ela -elas -eles -en -era -eran -esa -esas -ese -eses -esta -estar -estaba -está -están -este -estes -estiven -estou -eu -é -facer -foi -foron -fun -había -hai -iso -isto -la -las -lle -lles -lo -los -mais -me -meu -meus -min -miña -miñas -moi -na -nas -neste -nin -no -non -nos -nosa -nosas -noso -nosos -nós -nun -nunha -nuns -nunhas -o -os -ou -ó -ós -para -pero -pode -pois -pola -polas -polo -polos -por -que -se -senón -ser -seu -seus -sexa -sido -sobre -súa -súas -tamén -tan -te -ten -teñen -teño -ter -teu -teus -ti -tido -tiña -tiven -túa -túas -un -unha -unhas -uns -vos -vosa -vosas -voso -vosos -vós diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_hi.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_hi.txt deleted file mode 100644 index 86286bb083b..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_hi.txt +++ /dev/null @@ -1,235 +0,0 @@ -# Also see http://www.opensource.org/licenses/bsd-license.html -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# This file was created by Jacques Savoy and is distributed under the BSD license. -# Note: by default this file also contains forms normalized by HindiNormalizer -# for spelling variation (see section below), such that it can be used whether or -# not you enable that feature. When adding additional entries to this list, -# please add the normalized form as well. -अंदर -अत -अपना -अपनी -अपने -अभी -आदि -आप -इत्यादि -इन -इनका -इन्हीं -इन्हें -इन्हों -इस -इसका -इसकी -इसके -इसमें -इसी -इसे -उन -उनका -उनकी -उनके -उनको -उन्हीं -उन्हें -उन्हों -उस -उसके -उसी -उसे -एक -एवं -एस -ऐसे -और -कई -कर -करता -करते -करना -करने -करें -कहते -कहा -का -काफ़ी -कि -कितना -किन्हें -किन्हों -किया -किर -किस -किसी -किसे -की -कुछ -कुल -के -को -कोई -कौन -कौनसा -गया -घर -जब -जहाँ -जा -जितना -जिन -जिन्हें -जिन्हों -जिस -जिसे -जीधर -जैसा -जैसे -जो -तक -तब -तरह -तिन -तिन्हें -तिन्हों -तिस -तिसे -तो -था -थी -थे -दबारा -दिया -दुसरा -दूसरे -दो -द्वारा -न -नहीं -ना -निहायत -नीचे -ने -पर -पर -पहले -पूरा -पे -फिर -बनी -बही -बहुत -बाद -बाला -बिलकुल -भी -भीतर -मगर -मानो -मे -में -यदि -यह -यहाँ -यही -या -यिह -ये -रखें -रहा -रहे -ऱ्वासा -लिए -लिये -लेकिन -व -वर्ग -वह -वह -वहाँ -वहीं -वाले -वुह -वे -वग़ैरह -संग -सकता -सकते -सबसे -सभी -साथ -साबुत -साभ -सारा -से -सो -ही -हुआ -हुई -हुए -है -हैं -हो -होता -होती -होते -होना -होने -# additional normalized forms of the above -अपनि -जेसे -होति -सभि -तिंहों -इंहों -दवारा -इसि -किंहें -थि -उंहों -ओर -जिंहें -वहिं -अभि -बनि -हि -उंहिं -उंहें -हें -वगेरह -एसे -रवासा -कोन -निचे -काफि -उसि -पुरा -भितर -हे -बहि -वहां -कोइ -यहां -जिंहों -तिंहें -किसि -कइ -यहि -इंहिं -जिधर -इंहें -अदि -इतयादि -हुइ -कोनसा -इसकि -दुसरे -जहां -अप -किंहों -उनकि -भि -वरग -हुअ -जेसा -नहिं diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_hu.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_hu.txt deleted file mode 100644 index 37526da8aa9..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_hu.txt +++ /dev/null @@ -1,211 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/hungarian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - -| Hungarian stop word list -| prepared by Anna Tordai - -a -ahogy -ahol -aki -akik -akkor -alatt -által -általában -amely -amelyek -amelyekben -amelyeket -amelyet -amelynek -ami -amit -amolyan -amíg -amikor -át -abban -ahhoz -annak -arra -arról -az -azok -azon -azt -azzal -azért -aztán -azután -azonban -bár -be -belül -benne -cikk -cikkek -cikkeket -csak -de -e -eddig -egész -egy -egyes -egyetlen -egyéb -egyik -egyre -ekkor -el -elég -ellen -elő -először -előtt -első -én -éppen -ebben -ehhez -emilyen -ennek -erre -ez -ezt -ezek -ezen -ezzel -ezért -és -fel -felé -hanem -hiszen -hogy -hogyan -igen -így -illetve -ill. -ill -ilyen -ilyenkor -ison -ismét -itt -jó -jól -jobban -kell -kellett -keresztül -keressünk -ki -kívül -között -közül -legalább -lehet -lehetett -legyen -lenne -lenni -lesz -lett -maga -magát -majd -majd -már -más -másik -meg -még -mellett -mert -mely -melyek -mi -mit -míg -miért -milyen -mikor -minden -mindent -mindenki -mindig -mint -mintha -mivel -most -nagy -nagyobb -nagyon -ne -néha -nekem -neki -nem -néhány -nélkül -nincs -olyan -ott -össze -ő -ők -őket -pedig -persze -rá -s -saját -sem -semmi -sok -sokat -sokkal -számára -szemben -szerint -szinte -talán -tehát -teljes -tovább -továbbá -több -úgy -ugyanis -új -újabb -újra -után -utána -utolsó -vagy -vagyis -valaki -valami -valamint -való -vagyok -van -vannak -volt -voltam -voltak -voltunk -vissza -vele -viszont -volna diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_hy.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_hy.txt deleted file mode 100644 index 60c1c50fbc8..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_hy.txt +++ /dev/null @@ -1,46 +0,0 @@ -# example set of Armenian stopwords. -այդ -այլ -այն -այս -դու -դուք -եմ -են -ենք -ես -եք -է -էի -էին -էինք -էիր -էիք -էր -ըստ -թ -ի -ին -իսկ -իր -կամ -համար -հետ -հետո -մենք -մեջ -մի -ն -նա -նաև -նրա -նրանք -որ -որը -որոնք -որպես -ու -ում -պիտի -վրա -և diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_id.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_id.txt deleted file mode 100644 index 4617f83a5c5..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_id.txt +++ /dev/null @@ -1,359 +0,0 @@ -# from appendix D of: A Study of Stemming Effects on Information -# Retrieval in Bahasa Indonesia -ada -adanya -adalah -adapun -agak -agaknya -agar -akan -akankah -akhirnya -aku -akulah -amat -amatlah -anda -andalah -antar -diantaranya -antara -antaranya -diantara -apa -apaan -mengapa -apabila -apakah -apalagi -apatah -atau -ataukah -ataupun -bagai -bagaikan -sebagai -sebagainya -bagaimana -bagaimanapun -sebagaimana -bagaimanakah -bagi -bahkan -bahwa -bahwasanya -sebaliknya -banyak -sebanyak -beberapa -seberapa -begini -beginian -beginikah -beginilah -sebegini -begitu -begitukah -begitulah -begitupun -sebegitu -belum -belumlah -sebelum -sebelumnya -sebenarnya -berapa -berapakah -berapalah -berapapun -betulkah -sebetulnya -biasa -biasanya -bila -bilakah -bisa -bisakah -sebisanya -boleh -bolehkah -bolehlah -buat -bukan -bukankah -bukanlah -bukannya -cuma -percuma -dahulu -dalam -dan -dapat -dari -daripada -dekat -demi -demikian -demikianlah -sedemikian -dengan -depan -di -dia -dialah -dini -diri -dirinya -terdiri -dong -dulu -enggak -enggaknya -entah -entahlah -terhadap -terhadapnya -hal -hampir -hanya -hanyalah -harus -haruslah -harusnya -seharusnya -hendak -hendaklah -hendaknya -hingga -sehingga -ia -ialah -ibarat -ingin -inginkah -inginkan -ini -inikah -inilah -itu -itukah -itulah -jangan -jangankan -janganlah -jika -jikalau -juga -justru -kala -kalau -kalaulah -kalaupun -kalian -kami -kamilah -kamu -kamulah -kan -kapan -kapankah -kapanpun -dikarenakan -karena -karenanya -ke -kecil -kemudian -kenapa -kepada -kepadanya -ketika -seketika -khususnya -kini -kinilah -kiranya -sekiranya -kita -kitalah -kok -lagi -lagian -selagi -lah -lain -lainnya -melainkan -selaku -lalu -melalui -terlalu -lama -lamanya -selama -selama -selamanya -lebih -terlebih -bermacam -macam -semacam -maka -makanya -makin -malah -malahan -mampu -mampukah -mana -manakala -manalagi -masih -masihkah -semasih -masing -mau -maupun -semaunya -memang -mereka -merekalah -meski -meskipun -semula -mungkin -mungkinkah -nah -namun -nanti -nantinya -nyaris -oleh -olehnya -seorang -seseorang -pada -padanya -padahal -paling -sepanjang -pantas -sepantasnya -sepantasnyalah -para -pasti -pastilah -per -pernah -pula -pun -merupakan -rupanya -serupa -saat -saatnya -sesaat -saja -sajalah -saling -bersama -sama -sesama -sambil -sampai -sana -sangat -sangatlah -saya -sayalah -se -sebab -sebabnya -sebuah -tersebut -tersebutlah -sedang -sedangkan -sedikit -sedikitnya -segala -segalanya -segera -sesegera -sejak -sejenak -sekali -sekalian -sekalipun -sesekali -sekaligus -sekarang -sekarang -sekitar -sekitarnya -sela -selain -selalu -seluruh -seluruhnya -semakin -sementara -sempat -semua -semuanya -sendiri -sendirinya -seolah -seperti -sepertinya -sering -seringnya -serta -siapa -siapakah -siapapun -disini -disinilah -sini -sinilah -sesuatu -sesuatunya -suatu -sesudah -sesudahnya -sudah -sudahkah -sudahlah -supaya -tadi -tadinya -tak -tanpa -setelah -telah -tentang -tentu -tentulah -tentunya -tertentu -seterusnya -tapi -tetapi -setiap -tiap -setidaknya -tidak -tidakkah -tidaklah -toh -waduh -wah -wahai -sewaktu -walau -walaupun -wong -yaitu -yakni -yang diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_it.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_it.txt deleted file mode 100644 index 1219cc773ab..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_it.txt +++ /dev/null @@ -1,303 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/italian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | An Italian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - -ad | a (to) before vowel -al | a + il -allo | a + lo -ai | a + i -agli | a + gli -all | a + l' -agl | a + gl' -alla | a + la -alle | a + le -con | with -col | con + il -coi | con + i (forms collo, cogli etc are now very rare) -da | from -dal | da + il -dallo | da + lo -dai | da + i -dagli | da + gli -dall | da + l' -dagl | da + gll' -dalla | da + la -dalle | da + le -di | of -del | di + il -dello | di + lo -dei | di + i -degli | di + gli -dell | di + l' -degl | di + gl' -della | di + la -delle | di + le -in | in -nel | in + el -nello | in + lo -nei | in + i -negli | in + gli -nell | in + l' -negl | in + gl' -nella | in + la -nelle | in + le -su | on -sul | su + il -sullo | su + lo -sui | su + i -sugli | su + gli -sull | su + l' -sugl | su + gl' -sulla | su + la -sulle | su + le -per | through, by -tra | among -contro | against -io | I -tu | thou -lui | he -lei | she -noi | we -voi | you -loro | they -mio | my -mia | -miei | -mie | -tuo | -tua | -tuoi | thy -tue | -suo | -sua | -suoi | his, her -sue | -nostro | our -nostra | -nostri | -nostre | -vostro | your -vostra | -vostri | -vostre | -mi | me -ti | thee -ci | us, there -vi | you, there -lo | him, the -la | her, the -li | them -le | them, the -gli | to him, the -ne | from there etc -il | the -un | a -uno | a -una | a -ma | but -ed | and -se | if -perché | why, because -anche | also -come | how -dov | where (as dov') -dove | where -che | who, that -chi | who -cui | whom -non | not -più | more -quale | who, that -quanto | how much -quanti | -quanta | -quante | -quello | that -quelli | -quella | -quelle | -questo | this -questi | -questa | -queste | -si | yes -tutto | all -tutti | all - - | single letter forms: - -a | at -c | as c' for ce or ci -e | and -i | the -l | as l' -o | or - - | forms of avere, to have (not including the infinitive): - -ho -hai -ha -abbiamo -avete -hanno -abbia -abbiate -abbiano -avrò -avrai -avrà -avremo -avrete -avranno -avrei -avresti -avrebbe -avremmo -avreste -avrebbero -avevo -avevi -aveva -avevamo -avevate -avevano -ebbi -avesti -ebbe -avemmo -aveste -ebbero -avessi -avesse -avessimo -avessero -avendo -avuto -avuta -avuti -avute - - | forms of essere, to be (not including the infinitive): -sono -sei -è -siamo -siete -sia -siate -siano -sarò -sarai -sarà -saremo -sarete -saranno -sarei -saresti -sarebbe -saremmo -sareste -sarebbero -ero -eri -era -eravamo -eravate -erano -fui -fosti -fu -fummo -foste -furono -fossi -fosse -fossimo -fossero -essendo - - | forms of fare, to do (not including the infinitive, fa, fat-): -faccio -fai -facciamo -fanno -faccia -facciate -facciano -farò -farai -farà -faremo -farete -faranno -farei -faresti -farebbe -faremmo -fareste -farebbero -facevo -facevi -faceva -facevamo -facevate -facevano -feci -facesti -fece -facemmo -faceste -fecero -facessi -facesse -facessimo -facessero -facendo - - | forms of stare, to be (not including the infinitive): -sto -stai -sta -stiamo -stanno -stia -stiate -stiano -starò -starai -starà -staremo -starete -staranno -starei -staresti -starebbe -staremmo -stareste -starebbero -stavo -stavi -stava -stavamo -stavate -stavano -stetti -stesti -stette -stemmo -steste -stettero -stessi -stesse -stessimo -stessero -stando diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ja.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ja.txt deleted file mode 100644 index d4321be6b16..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ja.txt +++ /dev/null @@ -1,127 +0,0 @@ -# -# This file defines a stopword set for Japanese. -# -# This set is made up of hand-picked frequent terms from segmented Japanese Wikipedia. -# Punctuation characters and frequent kanji have mostly been left out. See LUCENE-3745 -# for frequency lists, etc. that can be useful for making your own set (if desired) -# -# Note that there is an overlap between these stopwords and the terms stopped when used -# in combination with the JapanesePartOfSpeechStopFilter. When editing this file, note -# that comments are not allowed on the same line as stopwords. -# -# Also note that stopping is done in a case-insensitive manner. Change your StopFilter -# configuration if you need case-sensitive stopping. Lastly, note that stopping is done -# using the same character width as the entries in this file. Since this StopFilter is -# normally done after a CJKWidthFilter in your chain, you would usually want your romaji -# entries to be in half-width and your kana entries to be in full-width. -# -の -に -は -を -た -が -で -て -と -し -れ -さ -ある -いる -も -する -から -な -こと -として -い -や -れる -など -なっ -ない -この -ため -その -あっ -よう -また -もの -という -あり -まで -られ -なる -へ -か -だ -これ -によって -により -おり -より -による -ず -なり -られる -において -ば -なかっ -なく -しかし -について -せ -だっ -その後 -できる -それ -う -ので -なお -のみ -でき -き -つ -における -および -いう -さらに -でも -ら -たり -その他 -に関する -たち -ます -ん -なら -に対して -特に -せる -及び -これら -とき -では -にて -ほか -ながら -うち -そして -とともに -ただし -かつて -それぞれ -または -お -ほど -ものの -に対する -ほとんど -と共に -といった -です -とも -ところ -ここ -##### End of file diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_lv.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_lv.txt deleted file mode 100644 index e21a23c06c3..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_lv.txt +++ /dev/null @@ -1,172 +0,0 @@ -# Set of Latvian stopwords from A Stemming Algorithm for Latvian, Karlis Kreslins -# the original list of over 800 forms was refined: -# pronouns, adverbs, interjections were removed -# -# prepositions -aiz -ap -ar -apakš -ārpus -augšpus -bez -caur -dēļ -gar -iekš -iz -kopš -labad -lejpus -līdz -no -otrpus -pa -par -pār -pēc -pie -pirms -pret -priekš -starp -šaipus -uz -viņpus -virs -virspus -zem -apakšpus -# Conjunctions -un -bet -jo -ja -ka -lai -tomēr -tikko -turpretī -arī -kaut -gan -tādēļ -tā -ne -tikvien -vien -kā -ir -te -vai -kamēr -# Particles -ar -diezin -droši -diemžēl -nebūt -ik -it -taču -nu -pat -tiklab -iekšpus -nedz -tik -nevis -turpretim -jeb -iekam -iekām -iekāms -kolīdz -līdzko -tiklīdz -jebšu -tālab -tāpēc -nekā -itin -jā -jau -jel -nē -nezin -tad -tikai -vis -tak -iekams -vien -# modal verbs -būt -biju -biji -bija -bijām -bijāt -esmu -esi -esam -esat -būšu -būsi -būs -būsim -būsiet -tikt -tiku -tiki -tika -tikām -tikāt -tieku -tiec -tiek -tiekam -tiekat -tikšu -tiks -tiksim -tiksiet -tapt -tapi -tapāt -topat -tapšu -tapsi -taps -tapsim -tapsiet -kļūt -kļuvu -kļuvi -kļuva -kļuvām -kļuvāt -kļūstu -kļūsti -kļūst -kļūstam -kļūstat -kļūšu -kļūsi -kļūs -kļūsim -kļūsiet -# verbs -varēt -varēju -varējām -varēšu -varēsim -var -varēji -varējāt -varēsi -varēsiet -varat -varēja -varēs diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_nl.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_nl.txt deleted file mode 100644 index 47a2aeacf6f..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_nl.txt +++ /dev/null @@ -1,119 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/dutch/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | A Dutch stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large sample of Dutch text. - - | Dutch stop words frequently exhibit homonym clashes. These are indicated - | clearly below. - -de | the -en | and -van | of, from -ik | I, the ego -te | (1) chez, at etc, (2) to, (3) too -dat | that, which -die | that, those, who, which -in | in, inside -een | a, an, one -hij | he -het | the, it -niet | not, nothing, naught -zijn | (1) to be, being, (2) his, one's, its -is | is -was | (1) was, past tense of all persons sing. of 'zijn' (to be) (2) wax, (3) the washing, (4) rise of river -op | on, upon, at, in, up, used up -aan | on, upon, to (as dative) -met | with, by -als | like, such as, when -voor | (1) before, in front of, (2) furrow -had | had, past tense all persons sing. of 'hebben' (have) -er | there -maar | but, only -om | round, about, for etc -hem | him -dan | then -zou | should/would, past tense all persons sing. of 'zullen' -of | or, whether, if -wat | what, something, anything -mijn | possessive and noun 'mine' -men | people, 'one' -dit | this -zo | so, thus, in this way -door | through by -over | over, across -ze | she, her, they, them -zich | oneself -bij | (1) a bee, (2) by, near, at -ook | also, too -tot | till, until -je | you -mij | me -uit | out of, from -der | Old Dutch form of 'van der' still found in surnames -daar | (1) there, (2) because -haar | (1) her, their, them, (2) hair -naar | (1) unpleasant, unwell etc, (2) towards, (3) as -heb | present first person sing. of 'to have' -hoe | how, why -heeft | present third person sing. of 'to have' -hebben | 'to have' and various parts thereof -deze | this -u | you -want | (1) for, (2) mitten, (3) rigging -nog | yet, still -zal | 'shall', first and third person sing. of verb 'zullen' (will) -me | me -zij | she, they -nu | now -ge | 'thou', still used in Belgium and south Netherlands -geen | none -omdat | because -iets | something, somewhat -worden | to become, grow, get -toch | yet, still -al | all, every, each -waren | (1) 'were' (2) to wander, (3) wares, (3) -veel | much, many -meer | (1) more, (2) lake -doen | to do, to make -toen | then, when -moet | noun 'spot/mote' and present form of 'to must' -ben | (1) am, (2) 'are' in interrogative second person singular of 'to be' -zonder | without -kan | noun 'can' and present form of 'to be able' -hun | their, them -dus | so, consequently -alles | all, everything, anything -onder | under, beneath -ja | yes, of course -eens | once, one day -hier | here -wie | who -werd | imperfect third person sing. of 'become' -altijd | always -doch | yet, but etc -wordt | present third person sing. of 'become' -wezen | (1) to be, (2) 'been' as in 'been fishing', (3) orphans -kunnen | to be able -ons | us/our -zelf | self -tegen | against, towards, at -na | after, near -reeds | already -wil | (1) present tense of 'want', (2) 'will', noun, (3) fender -kon | could; past tense of 'to be able' -niets | nothing -uw | your -iemand | somebody -geweest | been; past participle of 'be' -andere | other diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_no.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_no.txt deleted file mode 100644 index a7a2c28ba54..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_no.txt +++ /dev/null @@ -1,194 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/norwegian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | A Norwegian stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This stop word list is for the dominant bokmål dialect. Words unique - | to nynorsk are marked *. - - | Revised by Jan Bruusgaard , Jan 2005 - -og | and -i | in -jeg | I -det | it/this/that -at | to (w. inf.) -en | a/an -et | a/an -den | it/this/that -til | to -er | is/am/are -som | who/that -på | on -de | they / you(formal) -med | with -han | he -av | of -ikke | not -ikkje | not * -der | there -så | so -var | was/were -meg | me -seg | you -men | but -ett | one -har | have -om | about -vi | we -min | my -mitt | my -ha | have -hadde | had -hun | she -nå | now -over | over -da | when/as -ved | by/know -fra | from -du | you -ut | out -sin | your -dem | them -oss | us -opp | up -man | you/one -kan | can -hans | his -hvor | where -eller | or -hva | what -skal | shall/must -selv | self (reflective) -sjøl | self (reflective) -her | here -alle | all -vil | will -bli | become -ble | became -blei | became * -blitt | have become -kunne | could -inn | in -når | when -være | be -kom | come -noen | some -noe | some -ville | would -dere | you -som | who/which/that -deres | their/theirs -kun | only/just -ja | yes -etter | after -ned | down -skulle | should -denne | this -for | for/because -deg | you -si | hers/his -sine | hers/his -sitt | hers/his -mot | against -å | to -meget | much -hvorfor | why -dette | this -disse | these/those -uten | without -hvordan | how -ingen | none -din | your -ditt | your -blir | become -samme | same -hvilken | which -hvilke | which (plural) -sånn | such a -inni | inside/within -mellom | between -vår | our -hver | each -hvem | who -vors | us/ours -hvis | whose -både | both -bare | only/just -enn | than -fordi | as/because -før | before -mange | many -også | also -slik | just -vært | been -være | to be -båe | both * -begge | both -siden | since -dykk | your * -dykkar | yours * -dei | they * -deira | them * -deires | theirs * -deim | them * -di | your (fem.) * -då | as/when * -eg | I * -ein | a/an * -eit | a/an * -eitt | a/an * -elles | or * -honom | he * -hjå | at * -ho | she * -hoe | she * -henne | her -hennar | her/hers -hennes | hers -hoss | how * -hossen | how * -ikkje | not * -ingi | noone * -inkje | noone * -korleis | how * -korso | how * -kva | what/which * -kvar | where * -kvarhelst | where * -kven | who/whom * -kvi | why * -kvifor | why * -me | we * -medan | while * -mi | my * -mine | my * -mykje | much * -no | now * -nokon | some (masc./neut.) * -noka | some (fem.) * -nokor | some * -noko | some * -nokre | some * -si | his/hers * -sia | since * -sidan | since * -so | so * -somt | some * -somme | some * -um | about* -upp | up * -vere | be * -vore | was * -verte | become * -vort | become * -varte | became * -vart | became * - diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_pt.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_pt.txt deleted file mode 100644 index acfeb01af6b..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_pt.txt +++ /dev/null @@ -1,253 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/portuguese/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | A Portuguese stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - - | The following is a ranked list (commonest to rarest) of stopwords - | deriving from a large sample of text. - - | Extra words have been added at the end. - -de | of, from -a | the; to, at; her -o | the; him -que | who, that -e | and -do | de + o -da | de + a -em | in -um | a -para | for - | é from SER -com | with -não | not, no -uma | a -os | the; them -no | em + o -se | himself etc -na | em + a -por | for -mais | more -as | the; them -dos | de + os -como | as, like -mas | but - | foi from SER -ao | a + o -ele | he -das | de + as - | tem from TER -à | a + a -seu | his -sua | her -ou | or - | ser from SER -quando | when -muito | much - | há from HAV -nos | em + os; us -já | already, now - | está from EST -eu | I -também | also -só | only, just -pelo | per + o -pela | per + a -até | up to -isso | that -ela | he -entre | between - | era from SER -depois | after -sem | without -mesmo | same -aos | a + os - | ter from TER -seus | his -quem | whom -nas | em + as -me | me -esse | that -eles | they - | estão from EST -você | you - | tinha from TER - | foram from SER -essa | that -num | em + um -nem | nor -suas | her -meu | my -às | a + as -minha | my - | têm from TER -numa | em + uma -pelos | per + os -elas | they - | havia from HAV - | seja from SER -qual | which - | será from SER -nós | we - | tenho from TER -lhe | to him, her -deles | of them -essas | those -esses | those -pelas | per + as -este | this - | fosse from SER -dele | of him - - | other words. There are many contractions such as naquele = em+aquele, - | mo = me+o, but they are rare. - | Indefinite article plural forms are also rare. - -tu | thou -te | thee -vocês | you (plural) -vos | you -lhes | to them -meus | my -minhas -teu | thy -tua -teus -tuas -nosso | our -nossa -nossos -nossas - -dela | of her -delas | of them - -esta | this -estes | these -estas | these -aquele | that -aquela | that -aqueles | those -aquelas | those -isto | this -aquilo | that - - | forms of estar, to be (not including the infinitive): -estou -está -estamos -estão -estive -esteve -estivemos -estiveram -estava -estávamos -estavam -estivera -estivéramos -esteja -estejamos -estejam -estivesse -estivéssemos -estivessem -estiver -estivermos -estiverem - - | forms of haver, to have (not including the infinitive): -hei -há -havemos -hão -houve -houvemos -houveram -houvera -houvéramos -haja -hajamos -hajam -houvesse -houvéssemos -houvessem -houver -houvermos -houverem -houverei -houverá -houveremos -houverão -houveria -houveríamos -houveriam - - | forms of ser, to be (not including the infinitive): -sou -somos -são -era -éramos -eram -fui -foi -fomos -foram -fora -fôramos -seja -sejamos -sejam -fosse -fôssemos -fossem -for -formos -forem -serei -será -seremos -serão -seria -seríamos -seriam - - | forms of ter, to have (not including the infinitive): -tenho -tem -temos -tém -tinha -tínhamos -tinham -tive -teve -tivemos -tiveram -tivera -tivéramos -tenha -tenhamos -tenham -tivesse -tivéssemos -tivessem -tiver -tivermos -tiverem -terei -terá -teremos -terão -teria -teríamos -teriam diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ro.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ro.txt deleted file mode 100644 index 4fdee90a5ba..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ro.txt +++ /dev/null @@ -1,233 +0,0 @@ -# This file was created by Jacques Savoy and is distributed under the BSD license. -# See http://members.unine.ch/jacques.savoy/clef/index.html. -# Also see http://www.opensource.org/licenses/bsd-license.html -acea -aceasta -această -aceea -acei -aceia -acel -acela -acele -acelea -acest -acesta -aceste -acestea -aceşti -aceştia -acolo -acum -ai -aia -aibă -aici -al -ăla -ale -alea -ălea -altceva -altcineva -am -ar -are -aş -aşadar -asemenea -asta -ăsta -astăzi -astea -ăstea -ăştia -asupra -aţi -au -avea -avem -aveţi -azi -bine -bucur -bună -ca -că -căci -când -care -cărei -căror -cărui -cât -câte -câţi -către -câtva -ce -cel -ceva -chiar -cînd -cine -cineva -cît -cîte -cîţi -cîtva -contra -cu -cum -cumva -curând -curînd -da -dă -dacă -dar -datorită -de -deci -deja -deoarece -departe -deşi -din -dinaintea -dintr -dintre -drept -după -ea -ei -el -ele -eram -este -eşti -eu -face -fără -fi -fie -fiecare -fii -fim -fiţi -iar -ieri -îi -îl -îmi -împotriva -în -înainte -înaintea -încât -încît -încotro -între -întrucât -întrucît -îţi -la -lângă -le -li -lîngă -lor -lui -mă -mâine -mea -mei -mele -mereu -meu -mi -mine -mult -multă -mulţi -ne -nicăieri -nici -nimeni -nişte -noastră -noastre -noi -noştri -nostru -nu -ori -oricând -oricare -oricât -orice -oricînd -oricine -oricît -oricum -oriunde -până -pe -pentru -peste -pînă -poate -pot -prea -prima -primul -prin -printr -sa -să -săi -sale -sau -său -se -şi -sînt -sîntem -sînteţi -spre -sub -sunt -suntem -sunteţi -ta -tăi -tale -tău -te -ţi -ţie -tine -toată -toate -tot -toţi -totuşi -tu -un -una -unde -undeva -unei -unele -uneori -unor -vă -vi -voastră -voastre -voi -voştri -vostru -vouă -vreo -vreun diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ru.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ru.txt deleted file mode 100644 index 55271400c64..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_ru.txt +++ /dev/null @@ -1,243 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/russian/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | a russian stop word list. comments begin with vertical bar. each stop - | word is at the start of a line. - - | this is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | letter `ё' is translated to `е'. - -и | and -в | in/into -во | alternative form -не | not -что | what/that -он | he -на | on/onto -я | i -с | from -со | alternative form -как | how -а | milder form of `no' (but) -то | conjunction and form of `that' -все | all -она | she -так | so, thus -его | him -но | but -да | yes/and -ты | thou -к | towards, by -у | around, chez -же | intensifier particle -вы | you -за | beyond, behind -бы | conditional/subj. particle -по | up to, along -только | only -ее | her -мне | to me -было | it was -вот | here is/are, particle -от | away from -меня | me -еще | still, yet, more -нет | no, there isnt/arent -о | about -из | out of -ему | to him -теперь | now -когда | when -даже | even -ну | so, well -вдруг | suddenly -ли | interrogative particle -если | if -уже | already, but homonym of `narrower' -или | or -ни | neither -быть | to be -был | he was -него | prepositional form of его -до | up to -вас | you accusative -нибудь | indef. suffix preceded by hyphen -опять | again -уж | already, but homonym of `adder' -вам | to you -сказал | he said -ведь | particle `after all' -там | there -потом | then -себя | oneself -ничего | nothing -ей | to her -может | usually with `быть' as `maybe' -они | they -тут | here -где | where -есть | there is/are -надо | got to, must -ней | prepositional form of ей -для | for -мы | we -тебя | thee -их | them, their -чем | than -была | she was -сам | self -чтоб | in order to -без | without -будто | as if -человек | man, person, one -чего | genitive form of `what' -раз | once -тоже | also -себе | to oneself -под | beneath -жизнь | life -будет | will be -ж | short form of intensifer particle `же' -тогда | then -кто | who -этот | this -говорил | was saying -того | genitive form of `that' -потому | for that reason -этого | genitive form of `this' -какой | which -совсем | altogether -ним | prepositional form of `его', `они' -здесь | here -этом | prepositional form of `этот' -один | one -почти | almost -мой | my -тем | instrumental/dative plural of `тот', `то' -чтобы | full form of `in order that' -нее | her (acc.) -кажется | it seems -сейчас | now -были | they were -куда | where to -зачем | why -сказать | to say -всех | all (acc., gen. preposn. plural) -никогда | never -сегодня | today -можно | possible, one can -при | by -наконец | finally -два | two -об | alternative form of `о', about -другой | another -хоть | even -после | after -над | above -больше | more -тот | that one (masc.) -через | across, in -эти | these -нас | us -про | about -всего | in all, only, of all -них | prepositional form of `они' (they) -какая | which, feminine -много | lots -разве | interrogative particle -сказала | she said -три | three -эту | this, acc. fem. sing. -моя | my, feminine -впрочем | moreover, besides -хорошо | good -свою | ones own, acc. fem. sing. -этой | oblique form of `эта', fem. `this' -перед | in front of -иногда | sometimes -лучше | better -чуть | a little -том | preposn. form of `that one' -нельзя | one must not -такой | such a one -им | to them -более | more -всегда | always -конечно | of course -всю | acc. fem. sing of `all' -между | between - - - | b: some paradigms - | - | personal pronouns - | - | я меня мне мной [мною] - | ты тебя тебе тобой [тобою] - | он его ему им [него, нему, ним] - | она ее эи ею [нее, нэи, нею] - | оно его ему им [него, нему, ним] - | - | мы нас нам нами - | вы вас вам вами - | они их им ими [них, ним, ними] - | - | себя себе собой [собою] - | - | demonstrative pronouns: этот (this), тот (that) - | - | этот эта это эти - | этого эты это эти - | этого этой этого этих - | этому этой этому этим - | этим этой этим [этою] этими - | этом этой этом этих - | - | тот та то те - | того ту то те - | того той того тех - | тому той тому тем - | тем той тем [тою] теми - | том той том тех - | - | determinative pronouns - | - | (a) весь (all) - | - | весь вся все все - | всего всю все все - | всего всей всего всех - | всему всей всему всем - | всем всей всем [всею] всеми - | всем всей всем всех - | - | (b) сам (himself etc) - | - | сам сама само сами - | самого саму само самих - | самого самой самого самих - | самому самой самому самим - | самим самой самим [самою] самими - | самом самой самом самих - | - | stems of verbs `to be', `to have', `to do' and modal - | - | быть бы буд быв есть суть - | име - | дел - | мог мож мочь - | уме - | хоч хот - | долж - | можн - | нужн - | нельзя - diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_sv.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_sv.txt deleted file mode 100644 index 096f87f6766..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_sv.txt +++ /dev/null @@ -1,133 +0,0 @@ - | From svn.tartarus.org/snowball/trunk/website/algorithms/swedish/stop.txt - | This file is distributed under the BSD License. - | See http://snowball.tartarus.org/license.php - | Also see http://www.opensource.org/licenses/bsd-license.html - | - Encoding was converted to UTF-8. - | - This notice was added. - | - | NOTE: To use this file with StopFilterFactory, you must specify format="snowball" - - | A Swedish stop word list. Comments begin with vertical bar. Each stop - | word is at the start of a line. - - | This is a ranked list (commonest to rarest) of stopwords derived from - | a large text sample. - - | Swedish stop words occasionally exhibit homonym clashes. For example - | så = so, but also seed. These are indicated clearly below. - -och | and -det | it, this/that -att | to (with infinitive) -i | in, at -en | a -jag | I -hon | she -som | who, that -han | he -på | on -den | it, this/that -med | with -var | where, each -sig | him(self) etc -för | for -så | so (also: seed) -till | to -är | is -men | but -ett | a -om | if; around, about -hade | had -de | they, these/those -av | of -icke | not, no -mig | me -du | you -henne | her -då | then, when -sin | his -nu | now -har | have -inte | inte någon = no one -hans | his -honom | him -skulle | 'sake' -hennes | her -där | there -min | my -man | one (pronoun) -ej | nor -vid | at, by, on (also: vast) -kunde | could -något | some etc -från | from, off -ut | out -när | when -efter | after, behind -upp | up -vi | we -dem | them -vara | be -vad | what -över | over -än | than -dig | you -kan | can -sina | his -här | here -ha | have -mot | towards -alla | all -under | under (also: wonder) -någon | some etc -eller | or (else) -allt | all -mycket | much -sedan | since -ju | why -denna | this/that -själv | myself, yourself etc -detta | this/that -åt | to -utan | without -varit | was -hur | how -ingen | no -mitt | my -ni | you -bli | to be, become -blev | from bli -oss | us -din | thy -dessa | these/those -några | some etc -deras | their -blir | from bli -mina | my -samma | (the) same -vilken | who, that -er | you, your -sådan | such a -vår | our -blivit | from bli -dess | its -inom | within -mellan | between -sådant | such a -varför | why -varje | each -vilka | who, that -ditt | thy -vem | who -vilket | who, that -sitta | his -sådana | such a -vart | each -dina | thy -vars | whose -vårt | our -våra | our -ert | your -era | your -vilkas | whose - diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_th.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_th.txt deleted file mode 100644 index 07f0fabe692..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_th.txt +++ /dev/null @@ -1,119 +0,0 @@ -# Thai stopwords from: -# "Opinion Detection in Thai Political News Columns -# Based on Subjectivity Analysis" -# Khampol Sukhum, Supot Nitsuwat, and Choochart Haruechaiyasak -ไว้ -ไม่ -ไป -ได้ -ให้ -ใน -โดย -แห่ง -แล้ว -และ -แรก -แบบ -แต่ -เอง -เห็น -เลย -เริ่ม -เรา -เมื่อ -เพื่อ -เพราะ -เป็นการ -เป็น -เปิดเผย -เปิด -เนื่องจาก -เดียวกัน -เดียว -เช่น -เฉพาะ -เคย -เข้า -เขา -อีก -อาจ -อะไร -ออก -อย่าง -อยู่ -อยาก -หาก -หลาย -หลังจาก -หลัง -หรือ -หนึ่ง -ส่วน -ส่ง -สุด -สําหรับ -ว่า -วัน -ลง -ร่วม -ราย -รับ -ระหว่าง -รวม -ยัง -มี -มาก -มา -พร้อม -พบ -ผ่าน -ผล -บาง -น่า -นี้ -นํา -นั้น -นัก -นอกจาก -ทุก -ที่สุด -ที่ -ทําให้ -ทํา -ทาง -ทั้งนี้ -ทั้ง -ถ้า -ถูก -ถึง -ต้อง -ต่างๆ -ต่าง -ต่อ -ตาม -ตั้งแต่ -ตั้ง -ด้าน -ด้วย -ดัง -ซึ่ง -ช่วง -จึง -จาก -จัด -จะ -คือ -ความ -ครั้ง -คง -ขึ้น -ของ -ขอ -ขณะ -ก่อน -ก็ -การ -กับ -กัน -กว่า -กล่าว diff --git a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_tr.txt b/solr/example/example-DIH/solr/rss/conf/lang/stopwords_tr.txt deleted file mode 100644 index 84d9408d4ea..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/stopwords_tr.txt +++ /dev/null @@ -1,212 +0,0 @@ -# Turkish stopwords from LUCENE-559 -# merged with the list from "Information Retrieval on Turkish Texts" -# (http://www.users.muohio.edu/canf/papers/JASIST2008offPrint.pdf) -acaba -altmış -altı -ama -ancak -arada -aslında -ayrıca -bana -bazı -belki -ben -benden -beni -benim -beri -beş -bile -bin -bir -birçok -biri -birkaç -birkez -birşey -birşeyi -biz -bize -bizden -bizi -bizim -böyle -böylece -bu -buna -bunda -bundan -bunlar -bunları -bunların -bunu -bunun -burada -çok -çünkü -da -daha -dahi -de -defa -değil -diğer -diye -doksan -dokuz -dolayı -dolayısıyla -dört -edecek -eden -ederek -edilecek -ediliyor -edilmesi -ediyor -eğer -elli -en -etmesi -etti -ettiği -ettiğini -gibi -göre -halen -hangi -hatta -hem -henüz -hep -hepsi -her -herhangi -herkesin -hiç -hiçbir -için -iki -ile -ilgili -ise -işte -itibaren -itibariyle -kadar -karşın -katrilyon -kendi -kendilerine -kendini -kendisi -kendisine -kendisini -kez -ki -kim -kimden -kime -kimi -kimse -kırk -milyar -milyon -mu -mü -mı -nasıl -ne -neden -nedenle -nerde -nerede -nereye -niye -niçin -o -olan -olarak -oldu -olduğu -olduğunu -olduklarını -olmadı -olmadığı -olmak -olması -olmayan -olmaz -olsa -olsun -olup -olur -olursa -oluyor -on -ona -ondan -onlar -onlardan -onları -onların -onu -onun -otuz -oysa -öyle -pek -rağmen -sadece -sanki -sekiz -seksen -sen -senden -seni -senin -siz -sizden -sizi -sizin -şey -şeyden -şeyi -şeyler -şöyle -şu -şuna -şunda -şundan -şunları -şunu -tarafından -trilyon -tüm -üç -üzere -var -vardı -ve -veya -ya -yani -yapacak -yapılan -yapılması -yapıyor -yapmak -yaptı -yaptığı -yaptığını -yaptıkları -yedi -yerine -yetmiş -yine -yirmi -yoksa -yüz -zaten diff --git a/solr/example/example-DIH/solr/rss/conf/lang/userdict_ja.txt b/solr/example/example-DIH/solr/rss/conf/lang/userdict_ja.txt deleted file mode 100644 index 6f0368e4d81..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/lang/userdict_ja.txt +++ /dev/null @@ -1,29 +0,0 @@ -# -# This is a sample user dictionary for Kuromoji (JapaneseTokenizer) -# -# Add entries to this file in order to override the statistical model in terms -# of segmentation, readings and part-of-speech tags. Notice that entries do -# not have weights since they are always used when found. This is by-design -# in order to maximize ease-of-use. -# -# Entries are defined using the following CSV format: -# , ... , ... , -# -# Notice that a single half-width space separates tokens and readings, and -# that the number tokens and readings must match exactly. -# -# Also notice that multiple entries with the same is undefined. -# -# Whitespace only lines are ignored. Comments are not allowed on entry lines. -# - -# Custom segmentation for kanji compounds -日本経済新聞,日本 経済 新聞,ニホン ケイザイ シンブン,カスタム名詞 -関西国際空港,関西 国際 空港,カンサイ コクサイ クウコウ,カスタム名詞 - -# Custom segmentation for compound katakana -トートバッグ,トート バッグ,トート バッグ,かずカナ名詞 -ショルダーバッグ,ショルダー バッグ,ショルダー バッグ,かずカナ名詞 - -# Custom reading for former sumo wrestler -朝青龍,朝青龍,アサショウリュウ,カスタム人名 diff --git a/solr/example/example-DIH/solr/rss/conf/managed-schema b/solr/example/example-DIH/solr/rss/conf/managed-schema deleted file mode 100644 index 9417902c92e..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/managed-schema +++ /dev/null @@ -1,1096 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - link - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/mapping-FoldToASCII.txt b/solr/example/example-DIH/solr/rss/conf/mapping-FoldToASCII.txt deleted file mode 100644 index 9a84b6eac34..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/mapping-FoldToASCII.txt +++ /dev/null @@ -1,3813 +0,0 @@ -# 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. - - -# This map converts alphabetic, numeric, and symbolic Unicode characters -# which are not in the first 127 ASCII characters (the "Basic Latin" Unicode -# block) into their ASCII equivalents, if one exists. -# -# Characters from the following Unicode blocks are converted; however, only -# those characters with reasonable ASCII alternatives are converted: -# -# - C1 Controls and Latin-1 Supplement: http://www.unicode.org/charts/PDF/U0080.pdf -# - Latin Extended-A: http://www.unicode.org/charts/PDF/U0100.pdf -# - Latin Extended-B: http://www.unicode.org/charts/PDF/U0180.pdf -# - Latin Extended Additional: http://www.unicode.org/charts/PDF/U1E00.pdf -# - Latin Extended-C: http://www.unicode.org/charts/PDF/U2C60.pdf -# - Latin Extended-D: http://www.unicode.org/charts/PDF/UA720.pdf -# - IPA Extensions: http://www.unicode.org/charts/PDF/U0250.pdf -# - Phonetic Extensions: http://www.unicode.org/charts/PDF/U1D00.pdf -# - Phonetic Extensions Supplement: http://www.unicode.org/charts/PDF/U1D80.pdf -# - General Punctuation: http://www.unicode.org/charts/PDF/U2000.pdf -# - Superscripts and Subscripts: http://www.unicode.org/charts/PDF/U2070.pdf -# - Enclosed Alphanumerics: http://www.unicode.org/charts/PDF/U2460.pdf -# - Dingbats: http://www.unicode.org/charts/PDF/U2700.pdf -# - Supplemental Punctuation: http://www.unicode.org/charts/PDF/U2E00.pdf -# - Alphabetic Presentation Forms: http://www.unicode.org/charts/PDF/UFB00.pdf -# - Halfwidth and Fullwidth Forms: http://www.unicode.org/charts/PDF/UFF00.pdf -# -# See: http://en.wikipedia.org/wiki/Latin_characters_in_Unicode -# -# The set of character conversions supported by this map is a superset of -# those supported by the map represented by mapping-ISOLatin1Accent.txt. -# -# See the bottom of this file for the Perl script used to generate the contents -# of this file (without this header) from ASCIIFoldingFilter.java. - - -# Syntax: -# "source" => "target" -# "source".length() > 0 (source cannot be empty.) -# "target".length() >= 0 (target can be empty.) - - -# À [LATIN CAPITAL LETTER A WITH GRAVE] -"\u00C0" => "A" - -# Á [LATIN CAPITAL LETTER A WITH ACUTE] -"\u00C1" => "A" - -#  [LATIN CAPITAL LETTER A WITH CIRCUMFLEX] -"\u00C2" => "A" - -# à [LATIN CAPITAL LETTER A WITH TILDE] -"\u00C3" => "A" - -# Ä [LATIN CAPITAL LETTER A WITH DIAERESIS] -"\u00C4" => "A" - -# Å [LATIN CAPITAL LETTER A WITH RING ABOVE] -"\u00C5" => "A" - -# Ā [LATIN CAPITAL LETTER A WITH MACRON] -"\u0100" => "A" - -# Ă [LATIN CAPITAL LETTER A WITH BREVE] -"\u0102" => "A" - -# Ą [LATIN CAPITAL LETTER A WITH OGONEK] -"\u0104" => "A" - -# Ə http://en.wikipedia.org/wiki/Schwa [LATIN CAPITAL LETTER SCHWA] -"\u018F" => "A" - -# Ǎ [LATIN CAPITAL LETTER A WITH CARON] -"\u01CD" => "A" - -# Ǟ [LATIN CAPITAL LETTER A WITH DIAERESIS AND MACRON] -"\u01DE" => "A" - -# Ǡ [LATIN CAPITAL LETTER A WITH DOT ABOVE AND MACRON] -"\u01E0" => "A" - -# Ǻ [LATIN CAPITAL LETTER A WITH RING ABOVE AND ACUTE] -"\u01FA" => "A" - -# Ȁ [LATIN CAPITAL LETTER A WITH DOUBLE GRAVE] -"\u0200" => "A" - -# Ȃ [LATIN CAPITAL LETTER A WITH INVERTED BREVE] -"\u0202" => "A" - -# Ȧ [LATIN CAPITAL LETTER A WITH DOT ABOVE] -"\u0226" => "A" - -# Ⱥ [LATIN CAPITAL LETTER A WITH STROKE] -"\u023A" => "A" - -# ᴀ [LATIN LETTER SMALL CAPITAL A] -"\u1D00" => "A" - -# Ḁ [LATIN CAPITAL LETTER A WITH RING BELOW] -"\u1E00" => "A" - -# Ạ [LATIN CAPITAL LETTER A WITH DOT BELOW] -"\u1EA0" => "A" - -# Ả [LATIN CAPITAL LETTER A WITH HOOK ABOVE] -"\u1EA2" => "A" - -# Ấ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND ACUTE] -"\u1EA4" => "A" - -# Ầ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND GRAVE] -"\u1EA6" => "A" - -# Ẩ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND HOOK ABOVE] -"\u1EA8" => "A" - -# Ẫ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND TILDE] -"\u1EAA" => "A" - -# Ậ [LATIN CAPITAL LETTER A WITH CIRCUMFLEX AND DOT BELOW] -"\u1EAC" => "A" - -# Ắ [LATIN CAPITAL LETTER A WITH BREVE AND ACUTE] -"\u1EAE" => "A" - -# Ằ [LATIN CAPITAL LETTER A WITH BREVE AND GRAVE] -"\u1EB0" => "A" - -# Ẳ [LATIN CAPITAL LETTER A WITH BREVE AND HOOK ABOVE] -"\u1EB2" => "A" - -# Ẵ [LATIN CAPITAL LETTER A WITH BREVE AND TILDE] -"\u1EB4" => "A" - -# Ặ [LATIN CAPITAL LETTER A WITH BREVE AND DOT BELOW] -"\u1EB6" => "A" - -# Ⓐ [CIRCLED LATIN CAPITAL LETTER A] -"\u24B6" => "A" - -# A [FULLWIDTH LATIN CAPITAL LETTER A] -"\uFF21" => "A" - -# à [LATIN SMALL LETTER A WITH GRAVE] -"\u00E0" => "a" - -# á [LATIN SMALL LETTER A WITH ACUTE] -"\u00E1" => "a" - -# â [LATIN SMALL LETTER A WITH CIRCUMFLEX] -"\u00E2" => "a" - -# ã [LATIN SMALL LETTER A WITH TILDE] -"\u00E3" => "a" - -# ä [LATIN SMALL LETTER A WITH DIAERESIS] -"\u00E4" => "a" - -# å [LATIN SMALL LETTER A WITH RING ABOVE] -"\u00E5" => "a" - -# ā [LATIN SMALL LETTER A WITH MACRON] -"\u0101" => "a" - -# ă [LATIN SMALL LETTER A WITH BREVE] -"\u0103" => "a" - -# ą [LATIN SMALL LETTER A WITH OGONEK] -"\u0105" => "a" - -# ǎ [LATIN SMALL LETTER A WITH CARON] -"\u01CE" => "a" - -# ǟ [LATIN SMALL LETTER A WITH DIAERESIS AND MACRON] -"\u01DF" => "a" - -# ǡ [LATIN SMALL LETTER A WITH DOT ABOVE AND MACRON] -"\u01E1" => "a" - -# ǻ [LATIN SMALL LETTER A WITH RING ABOVE AND ACUTE] -"\u01FB" => "a" - -# ȁ [LATIN SMALL LETTER A WITH DOUBLE GRAVE] -"\u0201" => "a" - -# ȃ [LATIN SMALL LETTER A WITH INVERTED BREVE] -"\u0203" => "a" - -# ȧ [LATIN SMALL LETTER A WITH DOT ABOVE] -"\u0227" => "a" - -# ɐ [LATIN SMALL LETTER TURNED A] -"\u0250" => "a" - -# ə [LATIN SMALL LETTER SCHWA] -"\u0259" => "a" - -# ɚ [LATIN SMALL LETTER SCHWA WITH HOOK] -"\u025A" => "a" - -# ᶏ [LATIN SMALL LETTER A WITH RETROFLEX HOOK] -"\u1D8F" => "a" - -# ᶕ [LATIN SMALL LETTER SCHWA WITH RETROFLEX HOOK] -"\u1D95" => "a" - -# ạ [LATIN SMALL LETTER A WITH RING BELOW] -"\u1E01" => "a" - -# ả [LATIN SMALL LETTER A WITH RIGHT HALF RING] -"\u1E9A" => "a" - -# ạ [LATIN SMALL LETTER A WITH DOT BELOW] -"\u1EA1" => "a" - -# ả [LATIN SMALL LETTER A WITH HOOK ABOVE] -"\u1EA3" => "a" - -# ấ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND ACUTE] -"\u1EA5" => "a" - -# ầ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND GRAVE] -"\u1EA7" => "a" - -# ẩ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND HOOK ABOVE] -"\u1EA9" => "a" - -# ẫ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND TILDE] -"\u1EAB" => "a" - -# ậ [LATIN SMALL LETTER A WITH CIRCUMFLEX AND DOT BELOW] -"\u1EAD" => "a" - -# ắ [LATIN SMALL LETTER A WITH BREVE AND ACUTE] -"\u1EAF" => "a" - -# ằ [LATIN SMALL LETTER A WITH BREVE AND GRAVE] -"\u1EB1" => "a" - -# ẳ [LATIN SMALL LETTER A WITH BREVE AND HOOK ABOVE] -"\u1EB3" => "a" - -# ẵ [LATIN SMALL LETTER A WITH BREVE AND TILDE] -"\u1EB5" => "a" - -# ặ [LATIN SMALL LETTER A WITH BREVE AND DOT BELOW] -"\u1EB7" => "a" - -# ₐ [LATIN SUBSCRIPT SMALL LETTER A] -"\u2090" => "a" - -# ₔ [LATIN SUBSCRIPT SMALL LETTER SCHWA] -"\u2094" => "a" - -# ⓐ [CIRCLED LATIN SMALL LETTER A] -"\u24D0" => "a" - -# ⱥ [LATIN SMALL LETTER A WITH STROKE] -"\u2C65" => "a" - -# Ɐ [LATIN CAPITAL LETTER TURNED A] -"\u2C6F" => "a" - -# a [FULLWIDTH LATIN SMALL LETTER A] -"\uFF41" => "a" - -# Ꜳ [LATIN CAPITAL LETTER AA] -"\uA732" => "AA" - -# Æ [LATIN CAPITAL LETTER AE] -"\u00C6" => "AE" - -# Ǣ [LATIN CAPITAL LETTER AE WITH MACRON] -"\u01E2" => "AE" - -# Ǽ [LATIN CAPITAL LETTER AE WITH ACUTE] -"\u01FC" => "AE" - -# ᴁ [LATIN LETTER SMALL CAPITAL AE] -"\u1D01" => "AE" - -# Ꜵ [LATIN CAPITAL LETTER AO] -"\uA734" => "AO" - -# Ꜷ [LATIN CAPITAL LETTER AU] -"\uA736" => "AU" - -# Ꜹ [LATIN CAPITAL LETTER AV] -"\uA738" => "AV" - -# Ꜻ [LATIN CAPITAL LETTER AV WITH HORIZONTAL BAR] -"\uA73A" => "AV" - -# Ꜽ [LATIN CAPITAL LETTER AY] -"\uA73C" => "AY" - -# ⒜ [PARENTHESIZED LATIN SMALL LETTER A] -"\u249C" => "(a)" - -# ꜳ [LATIN SMALL LETTER AA] -"\uA733" => "aa" - -# æ [LATIN SMALL LETTER AE] -"\u00E6" => "ae" - -# ǣ [LATIN SMALL LETTER AE WITH MACRON] -"\u01E3" => "ae" - -# ǽ [LATIN SMALL LETTER AE WITH ACUTE] -"\u01FD" => "ae" - -# ᴂ [LATIN SMALL LETTER TURNED AE] -"\u1D02" => "ae" - -# ꜵ [LATIN SMALL LETTER AO] -"\uA735" => "ao" - -# ꜷ [LATIN SMALL LETTER AU] -"\uA737" => "au" - -# ꜹ [LATIN SMALL LETTER AV] -"\uA739" => "av" - -# ꜻ [LATIN SMALL LETTER AV WITH HORIZONTAL BAR] -"\uA73B" => "av" - -# ꜽ [LATIN SMALL LETTER AY] -"\uA73D" => "ay" - -# Ɓ [LATIN CAPITAL LETTER B WITH HOOK] -"\u0181" => "B" - -# Ƃ [LATIN CAPITAL LETTER B WITH TOPBAR] -"\u0182" => "B" - -# Ƀ [LATIN CAPITAL LETTER B WITH STROKE] -"\u0243" => "B" - -# ʙ [LATIN LETTER SMALL CAPITAL B] -"\u0299" => "B" - -# ᴃ [LATIN LETTER SMALL CAPITAL BARRED B] -"\u1D03" => "B" - -# Ḃ [LATIN CAPITAL LETTER B WITH DOT ABOVE] -"\u1E02" => "B" - -# Ḅ [LATIN CAPITAL LETTER B WITH DOT BELOW] -"\u1E04" => "B" - -# Ḇ [LATIN CAPITAL LETTER B WITH LINE BELOW] -"\u1E06" => "B" - -# Ⓑ [CIRCLED LATIN CAPITAL LETTER B] -"\u24B7" => "B" - -# B [FULLWIDTH LATIN CAPITAL LETTER B] -"\uFF22" => "B" - -# ƀ [LATIN SMALL LETTER B WITH STROKE] -"\u0180" => "b" - -# ƃ [LATIN SMALL LETTER B WITH TOPBAR] -"\u0183" => "b" - -# ɓ [LATIN SMALL LETTER B WITH HOOK] -"\u0253" => "b" - -# ᵬ [LATIN SMALL LETTER B WITH MIDDLE TILDE] -"\u1D6C" => "b" - -# ᶀ [LATIN SMALL LETTER B WITH PALATAL HOOK] -"\u1D80" => "b" - -# ḃ [LATIN SMALL LETTER B WITH DOT ABOVE] -"\u1E03" => "b" - -# ḅ [LATIN SMALL LETTER B WITH DOT BELOW] -"\u1E05" => "b" - -# ḇ [LATIN SMALL LETTER B WITH LINE BELOW] -"\u1E07" => "b" - -# ⓑ [CIRCLED LATIN SMALL LETTER B] -"\u24D1" => "b" - -# b [FULLWIDTH LATIN SMALL LETTER B] -"\uFF42" => "b" - -# ⒝ [PARENTHESIZED LATIN SMALL LETTER B] -"\u249D" => "(b)" - -# Ç [LATIN CAPITAL LETTER C WITH CEDILLA] -"\u00C7" => "C" - -# Ć [LATIN CAPITAL LETTER C WITH ACUTE] -"\u0106" => "C" - -# Ĉ [LATIN CAPITAL LETTER C WITH CIRCUMFLEX] -"\u0108" => "C" - -# Ċ [LATIN CAPITAL LETTER C WITH DOT ABOVE] -"\u010A" => "C" - -# Č [LATIN CAPITAL LETTER C WITH CARON] -"\u010C" => "C" - -# Ƈ [LATIN CAPITAL LETTER C WITH HOOK] -"\u0187" => "C" - -# Ȼ [LATIN CAPITAL LETTER C WITH STROKE] -"\u023B" => "C" - -# ʗ [LATIN LETTER STRETCHED C] -"\u0297" => "C" - -# ᴄ [LATIN LETTER SMALL CAPITAL C] -"\u1D04" => "C" - -# Ḉ [LATIN CAPITAL LETTER C WITH CEDILLA AND ACUTE] -"\u1E08" => "C" - -# Ⓒ [CIRCLED LATIN CAPITAL LETTER C] -"\u24B8" => "C" - -# C [FULLWIDTH LATIN CAPITAL LETTER C] -"\uFF23" => "C" - -# ç [LATIN SMALL LETTER C WITH CEDILLA] -"\u00E7" => "c" - -# ć [LATIN SMALL LETTER C WITH ACUTE] -"\u0107" => "c" - -# ĉ [LATIN SMALL LETTER C WITH CIRCUMFLEX] -"\u0109" => "c" - -# ċ [LATIN SMALL LETTER C WITH DOT ABOVE] -"\u010B" => "c" - -# č [LATIN SMALL LETTER C WITH CARON] -"\u010D" => "c" - -# ƈ [LATIN SMALL LETTER C WITH HOOK] -"\u0188" => "c" - -# ȼ [LATIN SMALL LETTER C WITH STROKE] -"\u023C" => "c" - -# ɕ [LATIN SMALL LETTER C WITH CURL] -"\u0255" => "c" - -# ḉ [LATIN SMALL LETTER C WITH CEDILLA AND ACUTE] -"\u1E09" => "c" - -# ↄ [LATIN SMALL LETTER REVERSED C] -"\u2184" => "c" - -# ⓒ [CIRCLED LATIN SMALL LETTER C] -"\u24D2" => "c" - -# Ꜿ [LATIN CAPITAL LETTER REVERSED C WITH DOT] -"\uA73E" => "c" - -# ꜿ [LATIN SMALL LETTER REVERSED C WITH DOT] -"\uA73F" => "c" - -# c [FULLWIDTH LATIN SMALL LETTER C] -"\uFF43" => "c" - -# ⒞ [PARENTHESIZED LATIN SMALL LETTER C] -"\u249E" => "(c)" - -# Ð [LATIN CAPITAL LETTER ETH] -"\u00D0" => "D" - -# Ď [LATIN CAPITAL LETTER D WITH CARON] -"\u010E" => "D" - -# Đ [LATIN CAPITAL LETTER D WITH STROKE] -"\u0110" => "D" - -# Ɖ [LATIN CAPITAL LETTER AFRICAN D] -"\u0189" => "D" - -# Ɗ [LATIN CAPITAL LETTER D WITH HOOK] -"\u018A" => "D" - -# Ƌ [LATIN CAPITAL LETTER D WITH TOPBAR] -"\u018B" => "D" - -# ᴅ [LATIN LETTER SMALL CAPITAL D] -"\u1D05" => "D" - -# ᴆ [LATIN LETTER SMALL CAPITAL ETH] -"\u1D06" => "D" - -# Ḋ [LATIN CAPITAL LETTER D WITH DOT ABOVE] -"\u1E0A" => "D" - -# Ḍ [LATIN CAPITAL LETTER D WITH DOT BELOW] -"\u1E0C" => "D" - -# Ḏ [LATIN CAPITAL LETTER D WITH LINE BELOW] -"\u1E0E" => "D" - -# Ḑ [LATIN CAPITAL LETTER D WITH CEDILLA] -"\u1E10" => "D" - -# Ḓ [LATIN CAPITAL LETTER D WITH CIRCUMFLEX BELOW] -"\u1E12" => "D" - -# Ⓓ [CIRCLED LATIN CAPITAL LETTER D] -"\u24B9" => "D" - -# Ꝺ [LATIN CAPITAL LETTER INSULAR D] -"\uA779" => "D" - -# D [FULLWIDTH LATIN CAPITAL LETTER D] -"\uFF24" => "D" - -# ð [LATIN SMALL LETTER ETH] -"\u00F0" => "d" - -# ď [LATIN SMALL LETTER D WITH CARON] -"\u010F" => "d" - -# đ [LATIN SMALL LETTER D WITH STROKE] -"\u0111" => "d" - -# ƌ [LATIN SMALL LETTER D WITH TOPBAR] -"\u018C" => "d" - -# ȡ [LATIN SMALL LETTER D WITH CURL] -"\u0221" => "d" - -# ɖ [LATIN SMALL LETTER D WITH TAIL] -"\u0256" => "d" - -# ɗ [LATIN SMALL LETTER D WITH HOOK] -"\u0257" => "d" - -# ᵭ [LATIN SMALL LETTER D WITH MIDDLE TILDE] -"\u1D6D" => "d" - -# ᶁ [LATIN SMALL LETTER D WITH PALATAL HOOK] -"\u1D81" => "d" - -# ᶑ [LATIN SMALL LETTER D WITH HOOK AND TAIL] -"\u1D91" => "d" - -# ḋ [LATIN SMALL LETTER D WITH DOT ABOVE] -"\u1E0B" => "d" - -# ḍ [LATIN SMALL LETTER D WITH DOT BELOW] -"\u1E0D" => "d" - -# ḏ [LATIN SMALL LETTER D WITH LINE BELOW] -"\u1E0F" => "d" - -# ḑ [LATIN SMALL LETTER D WITH CEDILLA] -"\u1E11" => "d" - -# ḓ [LATIN SMALL LETTER D WITH CIRCUMFLEX BELOW] -"\u1E13" => "d" - -# ⓓ [CIRCLED LATIN SMALL LETTER D] -"\u24D3" => "d" - -# ꝺ [LATIN SMALL LETTER INSULAR D] -"\uA77A" => "d" - -# d [FULLWIDTH LATIN SMALL LETTER D] -"\uFF44" => "d" - -# DŽ [LATIN CAPITAL LETTER DZ WITH CARON] -"\u01C4" => "DZ" - -# DZ [LATIN CAPITAL LETTER DZ] -"\u01F1" => "DZ" - -# Dž [LATIN CAPITAL LETTER D WITH SMALL LETTER Z WITH CARON] -"\u01C5" => "Dz" - -# Dz [LATIN CAPITAL LETTER D WITH SMALL LETTER Z] -"\u01F2" => "Dz" - -# ⒟ [PARENTHESIZED LATIN SMALL LETTER D] -"\u249F" => "(d)" - -# ȸ [LATIN SMALL LETTER DB DIGRAPH] -"\u0238" => "db" - -# dž [LATIN SMALL LETTER DZ WITH CARON] -"\u01C6" => "dz" - -# dz [LATIN SMALL LETTER DZ] -"\u01F3" => "dz" - -# ʣ [LATIN SMALL LETTER DZ DIGRAPH] -"\u02A3" => "dz" - -# ʥ [LATIN SMALL LETTER DZ DIGRAPH WITH CURL] -"\u02A5" => "dz" - -# È [LATIN CAPITAL LETTER E WITH GRAVE] -"\u00C8" => "E" - -# É [LATIN CAPITAL LETTER E WITH ACUTE] -"\u00C9" => "E" - -# Ê [LATIN CAPITAL LETTER E WITH CIRCUMFLEX] -"\u00CA" => "E" - -# Ë [LATIN CAPITAL LETTER E WITH DIAERESIS] -"\u00CB" => "E" - -# Ē [LATIN CAPITAL LETTER E WITH MACRON] -"\u0112" => "E" - -# Ĕ [LATIN CAPITAL LETTER E WITH BREVE] -"\u0114" => "E" - -# Ė [LATIN CAPITAL LETTER E WITH DOT ABOVE] -"\u0116" => "E" - -# Ę [LATIN CAPITAL LETTER E WITH OGONEK] -"\u0118" => "E" - -# Ě [LATIN CAPITAL LETTER E WITH CARON] -"\u011A" => "E" - -# Ǝ [LATIN CAPITAL LETTER REVERSED E] -"\u018E" => "E" - -# Ɛ [LATIN CAPITAL LETTER OPEN E] -"\u0190" => "E" - -# Ȅ [LATIN CAPITAL LETTER E WITH DOUBLE GRAVE] -"\u0204" => "E" - -# Ȇ [LATIN CAPITAL LETTER E WITH INVERTED BREVE] -"\u0206" => "E" - -# Ȩ [LATIN CAPITAL LETTER E WITH CEDILLA] -"\u0228" => "E" - -# Ɇ [LATIN CAPITAL LETTER E WITH STROKE] -"\u0246" => "E" - -# ᴇ [LATIN LETTER SMALL CAPITAL E] -"\u1D07" => "E" - -# Ḕ [LATIN CAPITAL LETTER E WITH MACRON AND GRAVE] -"\u1E14" => "E" - -# Ḗ [LATIN CAPITAL LETTER E WITH MACRON AND ACUTE] -"\u1E16" => "E" - -# Ḙ [LATIN CAPITAL LETTER E WITH CIRCUMFLEX BELOW] -"\u1E18" => "E" - -# Ḛ [LATIN CAPITAL LETTER E WITH TILDE BELOW] -"\u1E1A" => "E" - -# Ḝ [LATIN CAPITAL LETTER E WITH CEDILLA AND BREVE] -"\u1E1C" => "E" - -# Ẹ [LATIN CAPITAL LETTER E WITH DOT BELOW] -"\u1EB8" => "E" - -# Ẻ [LATIN CAPITAL LETTER E WITH HOOK ABOVE] -"\u1EBA" => "E" - -# Ẽ [LATIN CAPITAL LETTER E WITH TILDE] -"\u1EBC" => "E" - -# Ế [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND ACUTE] -"\u1EBE" => "E" - -# Ề [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND GRAVE] -"\u1EC0" => "E" - -# Ể [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND HOOK ABOVE] -"\u1EC2" => "E" - -# Ễ [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND TILDE] -"\u1EC4" => "E" - -# Ệ [LATIN CAPITAL LETTER E WITH CIRCUMFLEX AND DOT BELOW] -"\u1EC6" => "E" - -# Ⓔ [CIRCLED LATIN CAPITAL LETTER E] -"\u24BA" => "E" - -# ⱻ [LATIN LETTER SMALL CAPITAL TURNED E] -"\u2C7B" => "E" - -# E [FULLWIDTH LATIN CAPITAL LETTER E] -"\uFF25" => "E" - -# è [LATIN SMALL LETTER E WITH GRAVE] -"\u00E8" => "e" - -# é [LATIN SMALL LETTER E WITH ACUTE] -"\u00E9" => "e" - -# ê [LATIN SMALL LETTER E WITH CIRCUMFLEX] -"\u00EA" => "e" - -# ë [LATIN SMALL LETTER E WITH DIAERESIS] -"\u00EB" => "e" - -# ē [LATIN SMALL LETTER E WITH MACRON] -"\u0113" => "e" - -# ĕ [LATIN SMALL LETTER E WITH BREVE] -"\u0115" => "e" - -# ė [LATIN SMALL LETTER E WITH DOT ABOVE] -"\u0117" => "e" - -# ę [LATIN SMALL LETTER E WITH OGONEK] -"\u0119" => "e" - -# ě [LATIN SMALL LETTER E WITH CARON] -"\u011B" => "e" - -# ǝ [LATIN SMALL LETTER TURNED E] -"\u01DD" => "e" - -# ȅ [LATIN SMALL LETTER E WITH DOUBLE GRAVE] -"\u0205" => "e" - -# ȇ [LATIN SMALL LETTER E WITH INVERTED BREVE] -"\u0207" => "e" - -# ȩ [LATIN SMALL LETTER E WITH CEDILLA] -"\u0229" => "e" - -# ɇ [LATIN SMALL LETTER E WITH STROKE] -"\u0247" => "e" - -# ɘ [LATIN SMALL LETTER REVERSED E] -"\u0258" => "e" - -# ɛ [LATIN SMALL LETTER OPEN E] -"\u025B" => "e" - -# ɜ [LATIN SMALL LETTER REVERSED OPEN E] -"\u025C" => "e" - -# ɝ [LATIN SMALL LETTER REVERSED OPEN E WITH HOOK] -"\u025D" => "e" - -# ɞ [LATIN SMALL LETTER CLOSED REVERSED OPEN E] -"\u025E" => "e" - -# ʚ [LATIN SMALL LETTER CLOSED OPEN E] -"\u029A" => "e" - -# ᴈ [LATIN SMALL LETTER TURNED OPEN E] -"\u1D08" => "e" - -# ᶒ [LATIN SMALL LETTER E WITH RETROFLEX HOOK] -"\u1D92" => "e" - -# ᶓ [LATIN SMALL LETTER OPEN E WITH RETROFLEX HOOK] -"\u1D93" => "e" - -# ᶔ [LATIN SMALL LETTER REVERSED OPEN E WITH RETROFLEX HOOK] -"\u1D94" => "e" - -# ḕ [LATIN SMALL LETTER E WITH MACRON AND GRAVE] -"\u1E15" => "e" - -# ḗ [LATIN SMALL LETTER E WITH MACRON AND ACUTE] -"\u1E17" => "e" - -# ḙ [LATIN SMALL LETTER E WITH CIRCUMFLEX BELOW] -"\u1E19" => "e" - -# ḛ [LATIN SMALL LETTER E WITH TILDE BELOW] -"\u1E1B" => "e" - -# ḝ [LATIN SMALL LETTER E WITH CEDILLA AND BREVE] -"\u1E1D" => "e" - -# ẹ [LATIN SMALL LETTER E WITH DOT BELOW] -"\u1EB9" => "e" - -# ẻ [LATIN SMALL LETTER E WITH HOOK ABOVE] -"\u1EBB" => "e" - -# ẽ [LATIN SMALL LETTER E WITH TILDE] -"\u1EBD" => "e" - -# ế [LATIN SMALL LETTER E WITH CIRCUMFLEX AND ACUTE] -"\u1EBF" => "e" - -# ề [LATIN SMALL LETTER E WITH CIRCUMFLEX AND GRAVE] -"\u1EC1" => "e" - -# ể [LATIN SMALL LETTER E WITH CIRCUMFLEX AND HOOK ABOVE] -"\u1EC3" => "e" - -# ễ [LATIN SMALL LETTER E WITH CIRCUMFLEX AND TILDE] -"\u1EC5" => "e" - -# ệ [LATIN SMALL LETTER E WITH CIRCUMFLEX AND DOT BELOW] -"\u1EC7" => "e" - -# ₑ [LATIN SUBSCRIPT SMALL LETTER E] -"\u2091" => "e" - -# ⓔ [CIRCLED LATIN SMALL LETTER E] -"\u24D4" => "e" - -# ⱸ [LATIN SMALL LETTER E WITH NOTCH] -"\u2C78" => "e" - -# e [FULLWIDTH LATIN SMALL LETTER E] -"\uFF45" => "e" - -# ⒠ [PARENTHESIZED LATIN SMALL LETTER E] -"\u24A0" => "(e)" - -# Ƒ [LATIN CAPITAL LETTER F WITH HOOK] -"\u0191" => "F" - -# Ḟ [LATIN CAPITAL LETTER F WITH DOT ABOVE] -"\u1E1E" => "F" - -# Ⓕ [CIRCLED LATIN CAPITAL LETTER F] -"\u24BB" => "F" - -# ꜰ [LATIN LETTER SMALL CAPITAL F] -"\uA730" => "F" - -# Ꝼ [LATIN CAPITAL LETTER INSULAR F] -"\uA77B" => "F" - -# ꟻ [LATIN EPIGRAPHIC LETTER REVERSED F] -"\uA7FB" => "F" - -# F [FULLWIDTH LATIN CAPITAL LETTER F] -"\uFF26" => "F" - -# ƒ [LATIN SMALL LETTER F WITH HOOK] -"\u0192" => "f" - -# ᵮ [LATIN SMALL LETTER F WITH MIDDLE TILDE] -"\u1D6E" => "f" - -# ᶂ [LATIN SMALL LETTER F WITH PALATAL HOOK] -"\u1D82" => "f" - -# ḟ [LATIN SMALL LETTER F WITH DOT ABOVE] -"\u1E1F" => "f" - -# ẛ [LATIN SMALL LETTER LONG S WITH DOT ABOVE] -"\u1E9B" => "f" - -# ⓕ [CIRCLED LATIN SMALL LETTER F] -"\u24D5" => "f" - -# ꝼ [LATIN SMALL LETTER INSULAR F] -"\uA77C" => "f" - -# f [FULLWIDTH LATIN SMALL LETTER F] -"\uFF46" => "f" - -# ⒡ [PARENTHESIZED LATIN SMALL LETTER F] -"\u24A1" => "(f)" - -# ff [LATIN SMALL LIGATURE FF] -"\uFB00" => "ff" - -# ffi [LATIN SMALL LIGATURE FFI] -"\uFB03" => "ffi" - -# ffl [LATIN SMALL LIGATURE FFL] -"\uFB04" => "ffl" - -# fi [LATIN SMALL LIGATURE FI] -"\uFB01" => "fi" - -# fl [LATIN SMALL LIGATURE FL] -"\uFB02" => "fl" - -# Ĝ [LATIN CAPITAL LETTER G WITH CIRCUMFLEX] -"\u011C" => "G" - -# Ğ [LATIN CAPITAL LETTER G WITH BREVE] -"\u011E" => "G" - -# Ġ [LATIN CAPITAL LETTER G WITH DOT ABOVE] -"\u0120" => "G" - -# Ģ [LATIN CAPITAL LETTER G WITH CEDILLA] -"\u0122" => "G" - -# Ɠ [LATIN CAPITAL LETTER G WITH HOOK] -"\u0193" => "G" - -# Ǥ [LATIN CAPITAL LETTER G WITH STROKE] -"\u01E4" => "G" - -# ǥ [LATIN SMALL LETTER G WITH STROKE] -"\u01E5" => "G" - -# Ǧ [LATIN CAPITAL LETTER G WITH CARON] -"\u01E6" => "G" - -# ǧ [LATIN SMALL LETTER G WITH CARON] -"\u01E7" => "G" - -# Ǵ [LATIN CAPITAL LETTER G WITH ACUTE] -"\u01F4" => "G" - -# ɢ [LATIN LETTER SMALL CAPITAL G] -"\u0262" => "G" - -# ʛ [LATIN LETTER SMALL CAPITAL G WITH HOOK] -"\u029B" => "G" - -# Ḡ [LATIN CAPITAL LETTER G WITH MACRON] -"\u1E20" => "G" - -# Ⓖ [CIRCLED LATIN CAPITAL LETTER G] -"\u24BC" => "G" - -# Ᵹ [LATIN CAPITAL LETTER INSULAR G] -"\uA77D" => "G" - -# Ꝿ [LATIN CAPITAL LETTER TURNED INSULAR G] -"\uA77E" => "G" - -# G [FULLWIDTH LATIN CAPITAL LETTER G] -"\uFF27" => "G" - -# ĝ [LATIN SMALL LETTER G WITH CIRCUMFLEX] -"\u011D" => "g" - -# ğ [LATIN SMALL LETTER G WITH BREVE] -"\u011F" => "g" - -# ġ [LATIN SMALL LETTER G WITH DOT ABOVE] -"\u0121" => "g" - -# ģ [LATIN SMALL LETTER G WITH CEDILLA] -"\u0123" => "g" - -# ǵ [LATIN SMALL LETTER G WITH ACUTE] -"\u01F5" => "g" - -# ɠ [LATIN SMALL LETTER G WITH HOOK] -"\u0260" => "g" - -# ɡ [LATIN SMALL LETTER SCRIPT G] -"\u0261" => "g" - -# ᵷ [LATIN SMALL LETTER TURNED G] -"\u1D77" => "g" - -# ᵹ [LATIN SMALL LETTER INSULAR G] -"\u1D79" => "g" - -# ᶃ [LATIN SMALL LETTER G WITH PALATAL HOOK] -"\u1D83" => "g" - -# ḡ [LATIN SMALL LETTER G WITH MACRON] -"\u1E21" => "g" - -# ⓖ [CIRCLED LATIN SMALL LETTER G] -"\u24D6" => "g" - -# ꝿ [LATIN SMALL LETTER TURNED INSULAR G] -"\uA77F" => "g" - -# g [FULLWIDTH LATIN SMALL LETTER G] -"\uFF47" => "g" - -# ⒢ [PARENTHESIZED LATIN SMALL LETTER G] -"\u24A2" => "(g)" - -# Ĥ [LATIN CAPITAL LETTER H WITH CIRCUMFLEX] -"\u0124" => "H" - -# Ħ [LATIN CAPITAL LETTER H WITH STROKE] -"\u0126" => "H" - -# Ȟ [LATIN CAPITAL LETTER H WITH CARON] -"\u021E" => "H" - -# ʜ [LATIN LETTER SMALL CAPITAL H] -"\u029C" => "H" - -# Ḣ [LATIN CAPITAL LETTER H WITH DOT ABOVE] -"\u1E22" => "H" - -# Ḥ [LATIN CAPITAL LETTER H WITH DOT BELOW] -"\u1E24" => "H" - -# Ḧ [LATIN CAPITAL LETTER H WITH DIAERESIS] -"\u1E26" => "H" - -# Ḩ [LATIN CAPITAL LETTER H WITH CEDILLA] -"\u1E28" => "H" - -# Ḫ [LATIN CAPITAL LETTER H WITH BREVE BELOW] -"\u1E2A" => "H" - -# Ⓗ [CIRCLED LATIN CAPITAL LETTER H] -"\u24BD" => "H" - -# Ⱨ [LATIN CAPITAL LETTER H WITH DESCENDER] -"\u2C67" => "H" - -# Ⱶ [LATIN CAPITAL LETTER HALF H] -"\u2C75" => "H" - -# H [FULLWIDTH LATIN CAPITAL LETTER H] -"\uFF28" => "H" - -# ĥ [LATIN SMALL LETTER H WITH CIRCUMFLEX] -"\u0125" => "h" - -# ħ [LATIN SMALL LETTER H WITH STROKE] -"\u0127" => "h" - -# ȟ [LATIN SMALL LETTER H WITH CARON] -"\u021F" => "h" - -# ɥ [LATIN SMALL LETTER TURNED H] -"\u0265" => "h" - -# ɦ [LATIN SMALL LETTER H WITH HOOK] -"\u0266" => "h" - -# ʮ [LATIN SMALL LETTER TURNED H WITH FISHHOOK] -"\u02AE" => "h" - -# ʯ [LATIN SMALL LETTER TURNED H WITH FISHHOOK AND TAIL] -"\u02AF" => "h" - -# ḣ [LATIN SMALL LETTER H WITH DOT ABOVE] -"\u1E23" => "h" - -# ḥ [LATIN SMALL LETTER H WITH DOT BELOW] -"\u1E25" => "h" - -# ḧ [LATIN SMALL LETTER H WITH DIAERESIS] -"\u1E27" => "h" - -# ḩ [LATIN SMALL LETTER H WITH CEDILLA] -"\u1E29" => "h" - -# ḫ [LATIN SMALL LETTER H WITH BREVE BELOW] -"\u1E2B" => "h" - -# ẖ [LATIN SMALL LETTER H WITH LINE BELOW] -"\u1E96" => "h" - -# ⓗ [CIRCLED LATIN SMALL LETTER H] -"\u24D7" => "h" - -# ⱨ [LATIN SMALL LETTER H WITH DESCENDER] -"\u2C68" => "h" - -# ⱶ [LATIN SMALL LETTER HALF H] -"\u2C76" => "h" - -# h [FULLWIDTH LATIN SMALL LETTER H] -"\uFF48" => "h" - -# Ƕ http://en.wikipedia.org/wiki/Hwair [LATIN CAPITAL LETTER HWAIR] -"\u01F6" => "HV" - -# ⒣ [PARENTHESIZED LATIN SMALL LETTER H] -"\u24A3" => "(h)" - -# ƕ [LATIN SMALL LETTER HV] -"\u0195" => "hv" - -# Ì [LATIN CAPITAL LETTER I WITH GRAVE] -"\u00CC" => "I" - -# Í [LATIN CAPITAL LETTER I WITH ACUTE] -"\u00CD" => "I" - -# Î [LATIN CAPITAL LETTER I WITH CIRCUMFLEX] -"\u00CE" => "I" - -# Ï [LATIN CAPITAL LETTER I WITH DIAERESIS] -"\u00CF" => "I" - -# Ĩ [LATIN CAPITAL LETTER I WITH TILDE] -"\u0128" => "I" - -# Ī [LATIN CAPITAL LETTER I WITH MACRON] -"\u012A" => "I" - -# Ĭ [LATIN CAPITAL LETTER I WITH BREVE] -"\u012C" => "I" - -# Į [LATIN CAPITAL LETTER I WITH OGONEK] -"\u012E" => "I" - -# İ [LATIN CAPITAL LETTER I WITH DOT ABOVE] -"\u0130" => "I" - -# Ɩ [LATIN CAPITAL LETTER IOTA] -"\u0196" => "I" - -# Ɨ [LATIN CAPITAL LETTER I WITH STROKE] -"\u0197" => "I" - -# Ǐ [LATIN CAPITAL LETTER I WITH CARON] -"\u01CF" => "I" - -# Ȉ [LATIN CAPITAL LETTER I WITH DOUBLE GRAVE] -"\u0208" => "I" - -# Ȋ [LATIN CAPITAL LETTER I WITH INVERTED BREVE] -"\u020A" => "I" - -# ɪ [LATIN LETTER SMALL CAPITAL I] -"\u026A" => "I" - -# ᵻ [LATIN SMALL CAPITAL LETTER I WITH STROKE] -"\u1D7B" => "I" - -# Ḭ [LATIN CAPITAL LETTER I WITH TILDE BELOW] -"\u1E2C" => "I" - -# Ḯ [LATIN CAPITAL LETTER I WITH DIAERESIS AND ACUTE] -"\u1E2E" => "I" - -# Ỉ [LATIN CAPITAL LETTER I WITH HOOK ABOVE] -"\u1EC8" => "I" - -# Ị [LATIN CAPITAL LETTER I WITH DOT BELOW] -"\u1ECA" => "I" - -# Ⓘ [CIRCLED LATIN CAPITAL LETTER I] -"\u24BE" => "I" - -# ꟾ [LATIN EPIGRAPHIC LETTER I LONGA] -"\uA7FE" => "I" - -# I [FULLWIDTH LATIN CAPITAL LETTER I] -"\uFF29" => "I" - -# ì [LATIN SMALL LETTER I WITH GRAVE] -"\u00EC" => "i" - -# í [LATIN SMALL LETTER I WITH ACUTE] -"\u00ED" => "i" - -# î [LATIN SMALL LETTER I WITH CIRCUMFLEX] -"\u00EE" => "i" - -# ï [LATIN SMALL LETTER I WITH DIAERESIS] -"\u00EF" => "i" - -# ĩ [LATIN SMALL LETTER I WITH TILDE] -"\u0129" => "i" - -# ī [LATIN SMALL LETTER I WITH MACRON] -"\u012B" => "i" - -# ĭ [LATIN SMALL LETTER I WITH BREVE] -"\u012D" => "i" - -# į [LATIN SMALL LETTER I WITH OGONEK] -"\u012F" => "i" - -# ı [LATIN SMALL LETTER DOTLESS I] -"\u0131" => "i" - -# ǐ [LATIN SMALL LETTER I WITH CARON] -"\u01D0" => "i" - -# ȉ [LATIN SMALL LETTER I WITH DOUBLE GRAVE] -"\u0209" => "i" - -# ȋ [LATIN SMALL LETTER I WITH INVERTED BREVE] -"\u020B" => "i" - -# ɨ [LATIN SMALL LETTER I WITH STROKE] -"\u0268" => "i" - -# ᴉ [LATIN SMALL LETTER TURNED I] -"\u1D09" => "i" - -# ᵢ [LATIN SUBSCRIPT SMALL LETTER I] -"\u1D62" => "i" - -# ᵼ [LATIN SMALL LETTER IOTA WITH STROKE] -"\u1D7C" => "i" - -# ᶖ [LATIN SMALL LETTER I WITH RETROFLEX HOOK] -"\u1D96" => "i" - -# ḭ [LATIN SMALL LETTER I WITH TILDE BELOW] -"\u1E2D" => "i" - -# ḯ [LATIN SMALL LETTER I WITH DIAERESIS AND ACUTE] -"\u1E2F" => "i" - -# ỉ [LATIN SMALL LETTER I WITH HOOK ABOVE] -"\u1EC9" => "i" - -# ị [LATIN SMALL LETTER I WITH DOT BELOW] -"\u1ECB" => "i" - -# ⁱ [SUPERSCRIPT LATIN SMALL LETTER I] -"\u2071" => "i" - -# ⓘ [CIRCLED LATIN SMALL LETTER I] -"\u24D8" => "i" - -# i [FULLWIDTH LATIN SMALL LETTER I] -"\uFF49" => "i" - -# IJ [LATIN CAPITAL LIGATURE IJ] -"\u0132" => "IJ" - -# ⒤ [PARENTHESIZED LATIN SMALL LETTER I] -"\u24A4" => "(i)" - -# ij [LATIN SMALL LIGATURE IJ] -"\u0133" => "ij" - -# Ĵ [LATIN CAPITAL LETTER J WITH CIRCUMFLEX] -"\u0134" => "J" - -# Ɉ [LATIN CAPITAL LETTER J WITH STROKE] -"\u0248" => "J" - -# ᴊ [LATIN LETTER SMALL CAPITAL J] -"\u1D0A" => "J" - -# Ⓙ [CIRCLED LATIN CAPITAL LETTER J] -"\u24BF" => "J" - -# J [FULLWIDTH LATIN CAPITAL LETTER J] -"\uFF2A" => "J" - -# ĵ [LATIN SMALL LETTER J WITH CIRCUMFLEX] -"\u0135" => "j" - -# ǰ [LATIN SMALL LETTER J WITH CARON] -"\u01F0" => "j" - -# ȷ [LATIN SMALL LETTER DOTLESS J] -"\u0237" => "j" - -# ɉ [LATIN SMALL LETTER J WITH STROKE] -"\u0249" => "j" - -# ɟ [LATIN SMALL LETTER DOTLESS J WITH STROKE] -"\u025F" => "j" - -# ʄ [LATIN SMALL LETTER DOTLESS J WITH STROKE AND HOOK] -"\u0284" => "j" - -# ʝ [LATIN SMALL LETTER J WITH CROSSED-TAIL] -"\u029D" => "j" - -# ⓙ [CIRCLED LATIN SMALL LETTER J] -"\u24D9" => "j" - -# ⱼ [LATIN SUBSCRIPT SMALL LETTER J] -"\u2C7C" => "j" - -# j [FULLWIDTH LATIN SMALL LETTER J] -"\uFF4A" => "j" - -# ⒥ [PARENTHESIZED LATIN SMALL LETTER J] -"\u24A5" => "(j)" - -# Ķ [LATIN CAPITAL LETTER K WITH CEDILLA] -"\u0136" => "K" - -# Ƙ [LATIN CAPITAL LETTER K WITH HOOK] -"\u0198" => "K" - -# Ǩ [LATIN CAPITAL LETTER K WITH CARON] -"\u01E8" => "K" - -# ᴋ [LATIN LETTER SMALL CAPITAL K] -"\u1D0B" => "K" - -# Ḱ [LATIN CAPITAL LETTER K WITH ACUTE] -"\u1E30" => "K" - -# Ḳ [LATIN CAPITAL LETTER K WITH DOT BELOW] -"\u1E32" => "K" - -# Ḵ [LATIN CAPITAL LETTER K WITH LINE BELOW] -"\u1E34" => "K" - -# Ⓚ [CIRCLED LATIN CAPITAL LETTER K] -"\u24C0" => "K" - -# Ⱪ [LATIN CAPITAL LETTER K WITH DESCENDER] -"\u2C69" => "K" - -# Ꝁ [LATIN CAPITAL LETTER K WITH STROKE] -"\uA740" => "K" - -# Ꝃ [LATIN CAPITAL LETTER K WITH DIAGONAL STROKE] -"\uA742" => "K" - -# Ꝅ [LATIN CAPITAL LETTER K WITH STROKE AND DIAGONAL STROKE] -"\uA744" => "K" - -# K [FULLWIDTH LATIN CAPITAL LETTER K] -"\uFF2B" => "K" - -# ķ [LATIN SMALL LETTER K WITH CEDILLA] -"\u0137" => "k" - -# ƙ [LATIN SMALL LETTER K WITH HOOK] -"\u0199" => "k" - -# ǩ [LATIN SMALL LETTER K WITH CARON] -"\u01E9" => "k" - -# ʞ [LATIN SMALL LETTER TURNED K] -"\u029E" => "k" - -# ᶄ [LATIN SMALL LETTER K WITH PALATAL HOOK] -"\u1D84" => "k" - -# ḱ [LATIN SMALL LETTER K WITH ACUTE] -"\u1E31" => "k" - -# ḳ [LATIN SMALL LETTER K WITH DOT BELOW] -"\u1E33" => "k" - -# ḵ [LATIN SMALL LETTER K WITH LINE BELOW] -"\u1E35" => "k" - -# ⓚ [CIRCLED LATIN SMALL LETTER K] -"\u24DA" => "k" - -# ⱪ [LATIN SMALL LETTER K WITH DESCENDER] -"\u2C6A" => "k" - -# ꝁ [LATIN SMALL LETTER K WITH STROKE] -"\uA741" => "k" - -# ꝃ [LATIN SMALL LETTER K WITH DIAGONAL STROKE] -"\uA743" => "k" - -# ꝅ [LATIN SMALL LETTER K WITH STROKE AND DIAGONAL STROKE] -"\uA745" => "k" - -# k [FULLWIDTH LATIN SMALL LETTER K] -"\uFF4B" => "k" - -# ⒦ [PARENTHESIZED LATIN SMALL LETTER K] -"\u24A6" => "(k)" - -# Ĺ [LATIN CAPITAL LETTER L WITH ACUTE] -"\u0139" => "L" - -# Ļ [LATIN CAPITAL LETTER L WITH CEDILLA] -"\u013B" => "L" - -# Ľ [LATIN CAPITAL LETTER L WITH CARON] -"\u013D" => "L" - -# Ŀ [LATIN CAPITAL LETTER L WITH MIDDLE DOT] -"\u013F" => "L" - -# Ł [LATIN CAPITAL LETTER L WITH STROKE] -"\u0141" => "L" - -# Ƚ [LATIN CAPITAL LETTER L WITH BAR] -"\u023D" => "L" - -# ʟ [LATIN LETTER SMALL CAPITAL L] -"\u029F" => "L" - -# ᴌ [LATIN LETTER SMALL CAPITAL L WITH STROKE] -"\u1D0C" => "L" - -# Ḷ [LATIN CAPITAL LETTER L WITH DOT BELOW] -"\u1E36" => "L" - -# Ḹ [LATIN CAPITAL LETTER L WITH DOT BELOW AND MACRON] -"\u1E38" => "L" - -# Ḻ [LATIN CAPITAL LETTER L WITH LINE BELOW] -"\u1E3A" => "L" - -# Ḽ [LATIN CAPITAL LETTER L WITH CIRCUMFLEX BELOW] -"\u1E3C" => "L" - -# Ⓛ [CIRCLED LATIN CAPITAL LETTER L] -"\u24C1" => "L" - -# Ⱡ [LATIN CAPITAL LETTER L WITH DOUBLE BAR] -"\u2C60" => "L" - -# Ɫ [LATIN CAPITAL LETTER L WITH MIDDLE TILDE] -"\u2C62" => "L" - -# Ꝇ [LATIN CAPITAL LETTER BROKEN L] -"\uA746" => "L" - -# Ꝉ [LATIN CAPITAL LETTER L WITH HIGH STROKE] -"\uA748" => "L" - -# Ꞁ [LATIN CAPITAL LETTER TURNED L] -"\uA780" => "L" - -# L [FULLWIDTH LATIN CAPITAL LETTER L] -"\uFF2C" => "L" - -# ĺ [LATIN SMALL LETTER L WITH ACUTE] -"\u013A" => "l" - -# ļ [LATIN SMALL LETTER L WITH CEDILLA] -"\u013C" => "l" - -# ľ [LATIN SMALL LETTER L WITH CARON] -"\u013E" => "l" - -# ŀ [LATIN SMALL LETTER L WITH MIDDLE DOT] -"\u0140" => "l" - -# ł [LATIN SMALL LETTER L WITH STROKE] -"\u0142" => "l" - -# ƚ [LATIN SMALL LETTER L WITH BAR] -"\u019A" => "l" - -# ȴ [LATIN SMALL LETTER L WITH CURL] -"\u0234" => "l" - -# ɫ [LATIN SMALL LETTER L WITH MIDDLE TILDE] -"\u026B" => "l" - -# ɬ [LATIN SMALL LETTER L WITH BELT] -"\u026C" => "l" - -# ɭ [LATIN SMALL LETTER L WITH RETROFLEX HOOK] -"\u026D" => "l" - -# ᶅ [LATIN SMALL LETTER L WITH PALATAL HOOK] -"\u1D85" => "l" - -# ḷ [LATIN SMALL LETTER L WITH DOT BELOW] -"\u1E37" => "l" - -# ḹ [LATIN SMALL LETTER L WITH DOT BELOW AND MACRON] -"\u1E39" => "l" - -# ḻ [LATIN SMALL LETTER L WITH LINE BELOW] -"\u1E3B" => "l" - -# ḽ [LATIN SMALL LETTER L WITH CIRCUMFLEX BELOW] -"\u1E3D" => "l" - -# ⓛ [CIRCLED LATIN SMALL LETTER L] -"\u24DB" => "l" - -# ⱡ [LATIN SMALL LETTER L WITH DOUBLE BAR] -"\u2C61" => "l" - -# ꝇ [LATIN SMALL LETTER BROKEN L] -"\uA747" => "l" - -# ꝉ [LATIN SMALL LETTER L WITH HIGH STROKE] -"\uA749" => "l" - -# ꞁ [LATIN SMALL LETTER TURNED L] -"\uA781" => "l" - -# l [FULLWIDTH LATIN SMALL LETTER L] -"\uFF4C" => "l" - -# LJ [LATIN CAPITAL LETTER LJ] -"\u01C7" => "LJ" - -# Ỻ [LATIN CAPITAL LETTER MIDDLE-WELSH LL] -"\u1EFA" => "LL" - -# Lj [LATIN CAPITAL LETTER L WITH SMALL LETTER J] -"\u01C8" => "Lj" - -# ⒧ [PARENTHESIZED LATIN SMALL LETTER L] -"\u24A7" => "(l)" - -# lj [LATIN SMALL LETTER LJ] -"\u01C9" => "lj" - -# ỻ [LATIN SMALL LETTER MIDDLE-WELSH LL] -"\u1EFB" => "ll" - -# ʪ [LATIN SMALL LETTER LS DIGRAPH] -"\u02AA" => "ls" - -# ʫ [LATIN SMALL LETTER LZ DIGRAPH] -"\u02AB" => "lz" - -# Ɯ [LATIN CAPITAL LETTER TURNED M] -"\u019C" => "M" - -# ᴍ [LATIN LETTER SMALL CAPITAL M] -"\u1D0D" => "M" - -# Ḿ [LATIN CAPITAL LETTER M WITH ACUTE] -"\u1E3E" => "M" - -# Ṁ [LATIN CAPITAL LETTER M WITH DOT ABOVE] -"\u1E40" => "M" - -# Ṃ [LATIN CAPITAL LETTER M WITH DOT BELOW] -"\u1E42" => "M" - -# Ⓜ [CIRCLED LATIN CAPITAL LETTER M] -"\u24C2" => "M" - -# Ɱ [LATIN CAPITAL LETTER M WITH HOOK] -"\u2C6E" => "M" - -# ꟽ [LATIN EPIGRAPHIC LETTER INVERTED M] -"\uA7FD" => "M" - -# ꟿ [LATIN EPIGRAPHIC LETTER ARCHAIC M] -"\uA7FF" => "M" - -# M [FULLWIDTH LATIN CAPITAL LETTER M] -"\uFF2D" => "M" - -# ɯ [LATIN SMALL LETTER TURNED M] -"\u026F" => "m" - -# ɰ [LATIN SMALL LETTER TURNED M WITH LONG LEG] -"\u0270" => "m" - -# ɱ [LATIN SMALL LETTER M WITH HOOK] -"\u0271" => "m" - -# ᵯ [LATIN SMALL LETTER M WITH MIDDLE TILDE] -"\u1D6F" => "m" - -# ᶆ [LATIN SMALL LETTER M WITH PALATAL HOOK] -"\u1D86" => "m" - -# ḿ [LATIN SMALL LETTER M WITH ACUTE] -"\u1E3F" => "m" - -# ṁ [LATIN SMALL LETTER M WITH DOT ABOVE] -"\u1E41" => "m" - -# ṃ [LATIN SMALL LETTER M WITH DOT BELOW] -"\u1E43" => "m" - -# ⓜ [CIRCLED LATIN SMALL LETTER M] -"\u24DC" => "m" - -# m [FULLWIDTH LATIN SMALL LETTER M] -"\uFF4D" => "m" - -# ⒨ [PARENTHESIZED LATIN SMALL LETTER M] -"\u24A8" => "(m)" - -# Ñ [LATIN CAPITAL LETTER N WITH TILDE] -"\u00D1" => "N" - -# Ń [LATIN CAPITAL LETTER N WITH ACUTE] -"\u0143" => "N" - -# Ņ [LATIN CAPITAL LETTER N WITH CEDILLA] -"\u0145" => "N" - -# Ň [LATIN CAPITAL LETTER N WITH CARON] -"\u0147" => "N" - -# Ŋ http://en.wikipedia.org/wiki/Eng_(letter) [LATIN CAPITAL LETTER ENG] -"\u014A" => "N" - -# Ɲ [LATIN CAPITAL LETTER N WITH LEFT HOOK] -"\u019D" => "N" - -# Ǹ [LATIN CAPITAL LETTER N WITH GRAVE] -"\u01F8" => "N" - -# Ƞ [LATIN CAPITAL LETTER N WITH LONG RIGHT LEG] -"\u0220" => "N" - -# ɴ [LATIN LETTER SMALL CAPITAL N] -"\u0274" => "N" - -# ᴎ [LATIN LETTER SMALL CAPITAL REVERSED N] -"\u1D0E" => "N" - -# Ṅ [LATIN CAPITAL LETTER N WITH DOT ABOVE] -"\u1E44" => "N" - -# Ṇ [LATIN CAPITAL LETTER N WITH DOT BELOW] -"\u1E46" => "N" - -# Ṉ [LATIN CAPITAL LETTER N WITH LINE BELOW] -"\u1E48" => "N" - -# Ṋ [LATIN CAPITAL LETTER N WITH CIRCUMFLEX BELOW] -"\u1E4A" => "N" - -# Ⓝ [CIRCLED LATIN CAPITAL LETTER N] -"\u24C3" => "N" - -# N [FULLWIDTH LATIN CAPITAL LETTER N] -"\uFF2E" => "N" - -# ñ [LATIN SMALL LETTER N WITH TILDE] -"\u00F1" => "n" - -# ń [LATIN SMALL LETTER N WITH ACUTE] -"\u0144" => "n" - -# ņ [LATIN SMALL LETTER N WITH CEDILLA] -"\u0146" => "n" - -# ň [LATIN SMALL LETTER N WITH CARON] -"\u0148" => "n" - -# ʼn [LATIN SMALL LETTER N PRECEDED BY APOSTROPHE] -"\u0149" => "n" - -# ŋ http://en.wikipedia.org/wiki/Eng_(letter) [LATIN SMALL LETTER ENG] -"\u014B" => "n" - -# ƞ [LATIN SMALL LETTER N WITH LONG RIGHT LEG] -"\u019E" => "n" - -# ǹ [LATIN SMALL LETTER N WITH GRAVE] -"\u01F9" => "n" - -# ȵ [LATIN SMALL LETTER N WITH CURL] -"\u0235" => "n" - -# ɲ [LATIN SMALL LETTER N WITH LEFT HOOK] -"\u0272" => "n" - -# ɳ [LATIN SMALL LETTER N WITH RETROFLEX HOOK] -"\u0273" => "n" - -# ᵰ [LATIN SMALL LETTER N WITH MIDDLE TILDE] -"\u1D70" => "n" - -# ᶇ [LATIN SMALL LETTER N WITH PALATAL HOOK] -"\u1D87" => "n" - -# ṅ [LATIN SMALL LETTER N WITH DOT ABOVE] -"\u1E45" => "n" - -# ṇ [LATIN SMALL LETTER N WITH DOT BELOW] -"\u1E47" => "n" - -# ṉ [LATIN SMALL LETTER N WITH LINE BELOW] -"\u1E49" => "n" - -# ṋ [LATIN SMALL LETTER N WITH CIRCUMFLEX BELOW] -"\u1E4B" => "n" - -# ⁿ [SUPERSCRIPT LATIN SMALL LETTER N] -"\u207F" => "n" - -# ⓝ [CIRCLED LATIN SMALL LETTER N] -"\u24DD" => "n" - -# n [FULLWIDTH LATIN SMALL LETTER N] -"\uFF4E" => "n" - -# NJ [LATIN CAPITAL LETTER NJ] -"\u01CA" => "NJ" - -# Nj [LATIN CAPITAL LETTER N WITH SMALL LETTER J] -"\u01CB" => "Nj" - -# ⒩ [PARENTHESIZED LATIN SMALL LETTER N] -"\u24A9" => "(n)" - -# nj [LATIN SMALL LETTER NJ] -"\u01CC" => "nj" - -# Ò [LATIN CAPITAL LETTER O WITH GRAVE] -"\u00D2" => "O" - -# Ó [LATIN CAPITAL LETTER O WITH ACUTE] -"\u00D3" => "O" - -# Ô [LATIN CAPITAL LETTER O WITH CIRCUMFLEX] -"\u00D4" => "O" - -# Õ [LATIN CAPITAL LETTER O WITH TILDE] -"\u00D5" => "O" - -# Ö [LATIN CAPITAL LETTER O WITH DIAERESIS] -"\u00D6" => "O" - -# Ø [LATIN CAPITAL LETTER O WITH STROKE] -"\u00D8" => "O" - -# Ō [LATIN CAPITAL LETTER O WITH MACRON] -"\u014C" => "O" - -# Ŏ [LATIN CAPITAL LETTER O WITH BREVE] -"\u014E" => "O" - -# Ő [LATIN CAPITAL LETTER O WITH DOUBLE ACUTE] -"\u0150" => "O" - -# Ɔ [LATIN CAPITAL LETTER OPEN O] -"\u0186" => "O" - -# Ɵ [LATIN CAPITAL LETTER O WITH MIDDLE TILDE] -"\u019F" => "O" - -# Ơ [LATIN CAPITAL LETTER O WITH HORN] -"\u01A0" => "O" - -# Ǒ [LATIN CAPITAL LETTER O WITH CARON] -"\u01D1" => "O" - -# Ǫ [LATIN CAPITAL LETTER O WITH OGONEK] -"\u01EA" => "O" - -# Ǭ [LATIN CAPITAL LETTER O WITH OGONEK AND MACRON] -"\u01EC" => "O" - -# Ǿ [LATIN CAPITAL LETTER O WITH STROKE AND ACUTE] -"\u01FE" => "O" - -# Ȍ [LATIN CAPITAL LETTER O WITH DOUBLE GRAVE] -"\u020C" => "O" - -# Ȏ [LATIN CAPITAL LETTER O WITH INVERTED BREVE] -"\u020E" => "O" - -# Ȫ [LATIN CAPITAL LETTER O WITH DIAERESIS AND MACRON] -"\u022A" => "O" - -# Ȭ [LATIN CAPITAL LETTER O WITH TILDE AND MACRON] -"\u022C" => "O" - -# Ȯ [LATIN CAPITAL LETTER O WITH DOT ABOVE] -"\u022E" => "O" - -# Ȱ [LATIN CAPITAL LETTER O WITH DOT ABOVE AND MACRON] -"\u0230" => "O" - -# ᴏ [LATIN LETTER SMALL CAPITAL O] -"\u1D0F" => "O" - -# ᴐ [LATIN LETTER SMALL CAPITAL OPEN O] -"\u1D10" => "O" - -# Ṍ [LATIN CAPITAL LETTER O WITH TILDE AND ACUTE] -"\u1E4C" => "O" - -# Ṏ [LATIN CAPITAL LETTER O WITH TILDE AND DIAERESIS] -"\u1E4E" => "O" - -# Ṑ [LATIN CAPITAL LETTER O WITH MACRON AND GRAVE] -"\u1E50" => "O" - -# Ṓ [LATIN CAPITAL LETTER O WITH MACRON AND ACUTE] -"\u1E52" => "O" - -# Ọ [LATIN CAPITAL LETTER O WITH DOT BELOW] -"\u1ECC" => "O" - -# Ỏ [LATIN CAPITAL LETTER O WITH HOOK ABOVE] -"\u1ECE" => "O" - -# Ố [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND ACUTE] -"\u1ED0" => "O" - -# Ồ [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND GRAVE] -"\u1ED2" => "O" - -# Ổ [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND HOOK ABOVE] -"\u1ED4" => "O" - -# Ỗ [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND TILDE] -"\u1ED6" => "O" - -# Ộ [LATIN CAPITAL LETTER O WITH CIRCUMFLEX AND DOT BELOW] -"\u1ED8" => "O" - -# Ớ [LATIN CAPITAL LETTER O WITH HORN AND ACUTE] -"\u1EDA" => "O" - -# Ờ [LATIN CAPITAL LETTER O WITH HORN AND GRAVE] -"\u1EDC" => "O" - -# Ở [LATIN CAPITAL LETTER O WITH HORN AND HOOK ABOVE] -"\u1EDE" => "O" - -# Ỡ [LATIN CAPITAL LETTER O WITH HORN AND TILDE] -"\u1EE0" => "O" - -# Ợ [LATIN CAPITAL LETTER O WITH HORN AND DOT BELOW] -"\u1EE2" => "O" - -# Ⓞ [CIRCLED LATIN CAPITAL LETTER O] -"\u24C4" => "O" - -# Ꝋ [LATIN CAPITAL LETTER O WITH LONG STROKE OVERLAY] -"\uA74A" => "O" - -# Ꝍ [LATIN CAPITAL LETTER O WITH LOOP] -"\uA74C" => "O" - -# O [FULLWIDTH LATIN CAPITAL LETTER O] -"\uFF2F" => "O" - -# ò [LATIN SMALL LETTER O WITH GRAVE] -"\u00F2" => "o" - -# ó [LATIN SMALL LETTER O WITH ACUTE] -"\u00F3" => "o" - -# ô [LATIN SMALL LETTER O WITH CIRCUMFLEX] -"\u00F4" => "o" - -# õ [LATIN SMALL LETTER O WITH TILDE] -"\u00F5" => "o" - -# ö [LATIN SMALL LETTER O WITH DIAERESIS] -"\u00F6" => "o" - -# ø [LATIN SMALL LETTER O WITH STROKE] -"\u00F8" => "o" - -# ō [LATIN SMALL LETTER O WITH MACRON] -"\u014D" => "o" - -# ŏ [LATIN SMALL LETTER O WITH BREVE] -"\u014F" => "o" - -# ő [LATIN SMALL LETTER O WITH DOUBLE ACUTE] -"\u0151" => "o" - -# ơ [LATIN SMALL LETTER O WITH HORN] -"\u01A1" => "o" - -# ǒ [LATIN SMALL LETTER O WITH CARON] -"\u01D2" => "o" - -# ǫ [LATIN SMALL LETTER O WITH OGONEK] -"\u01EB" => "o" - -# ǭ [LATIN SMALL LETTER O WITH OGONEK AND MACRON] -"\u01ED" => "o" - -# ǿ [LATIN SMALL LETTER O WITH STROKE AND ACUTE] -"\u01FF" => "o" - -# ȍ [LATIN SMALL LETTER O WITH DOUBLE GRAVE] -"\u020D" => "o" - -# ȏ [LATIN SMALL LETTER O WITH INVERTED BREVE] -"\u020F" => "o" - -# ȫ [LATIN SMALL LETTER O WITH DIAERESIS AND MACRON] -"\u022B" => "o" - -# ȭ [LATIN SMALL LETTER O WITH TILDE AND MACRON] -"\u022D" => "o" - -# ȯ [LATIN SMALL LETTER O WITH DOT ABOVE] -"\u022F" => "o" - -# ȱ [LATIN SMALL LETTER O WITH DOT ABOVE AND MACRON] -"\u0231" => "o" - -# ɔ [LATIN SMALL LETTER OPEN O] -"\u0254" => "o" - -# ɵ [LATIN SMALL LETTER BARRED O] -"\u0275" => "o" - -# ᴖ [LATIN SMALL LETTER TOP HALF O] -"\u1D16" => "o" - -# ᴗ [LATIN SMALL LETTER BOTTOM HALF O] -"\u1D17" => "o" - -# ᶗ [LATIN SMALL LETTER OPEN O WITH RETROFLEX HOOK] -"\u1D97" => "o" - -# ṍ [LATIN SMALL LETTER O WITH TILDE AND ACUTE] -"\u1E4D" => "o" - -# ṏ [LATIN SMALL LETTER O WITH TILDE AND DIAERESIS] -"\u1E4F" => "o" - -# ṑ [LATIN SMALL LETTER O WITH MACRON AND GRAVE] -"\u1E51" => "o" - -# ṓ [LATIN SMALL LETTER O WITH MACRON AND ACUTE] -"\u1E53" => "o" - -# ọ [LATIN SMALL LETTER O WITH DOT BELOW] -"\u1ECD" => "o" - -# ỏ [LATIN SMALL LETTER O WITH HOOK ABOVE] -"\u1ECF" => "o" - -# ố [LATIN SMALL LETTER O WITH CIRCUMFLEX AND ACUTE] -"\u1ED1" => "o" - -# ồ [LATIN SMALL LETTER O WITH CIRCUMFLEX AND GRAVE] -"\u1ED3" => "o" - -# ổ [LATIN SMALL LETTER O WITH CIRCUMFLEX AND HOOK ABOVE] -"\u1ED5" => "o" - -# ỗ [LATIN SMALL LETTER O WITH CIRCUMFLEX AND TILDE] -"\u1ED7" => "o" - -# ộ [LATIN SMALL LETTER O WITH CIRCUMFLEX AND DOT BELOW] -"\u1ED9" => "o" - -# ớ [LATIN SMALL LETTER O WITH HORN AND ACUTE] -"\u1EDB" => "o" - -# ờ [LATIN SMALL LETTER O WITH HORN AND GRAVE] -"\u1EDD" => "o" - -# ở [LATIN SMALL LETTER O WITH HORN AND HOOK ABOVE] -"\u1EDF" => "o" - -# ỡ [LATIN SMALL LETTER O WITH HORN AND TILDE] -"\u1EE1" => "o" - -# ợ [LATIN SMALL LETTER O WITH HORN AND DOT BELOW] -"\u1EE3" => "o" - -# ₒ [LATIN SUBSCRIPT SMALL LETTER O] -"\u2092" => "o" - -# ⓞ [CIRCLED LATIN SMALL LETTER O] -"\u24DE" => "o" - -# ⱺ [LATIN SMALL LETTER O WITH LOW RING INSIDE] -"\u2C7A" => "o" - -# ꝋ [LATIN SMALL LETTER O WITH LONG STROKE OVERLAY] -"\uA74B" => "o" - -# ꝍ [LATIN SMALL LETTER O WITH LOOP] -"\uA74D" => "o" - -# o [FULLWIDTH LATIN SMALL LETTER O] -"\uFF4F" => "o" - -# Œ [LATIN CAPITAL LIGATURE OE] -"\u0152" => "OE" - -# ɶ [LATIN LETTER SMALL CAPITAL OE] -"\u0276" => "OE" - -# Ꝏ [LATIN CAPITAL LETTER OO] -"\uA74E" => "OO" - -# Ȣ http://en.wikipedia.org/wiki/OU [LATIN CAPITAL LETTER OU] -"\u0222" => "OU" - -# ᴕ [LATIN LETTER SMALL CAPITAL OU] -"\u1D15" => "OU" - -# ⒪ [PARENTHESIZED LATIN SMALL LETTER O] -"\u24AA" => "(o)" - -# œ [LATIN SMALL LIGATURE OE] -"\u0153" => "oe" - -# ᴔ [LATIN SMALL LETTER TURNED OE] -"\u1D14" => "oe" - -# ꝏ [LATIN SMALL LETTER OO] -"\uA74F" => "oo" - -# ȣ http://en.wikipedia.org/wiki/OU [LATIN SMALL LETTER OU] -"\u0223" => "ou" - -# Ƥ [LATIN CAPITAL LETTER P WITH HOOK] -"\u01A4" => "P" - -# ᴘ [LATIN LETTER SMALL CAPITAL P] -"\u1D18" => "P" - -# Ṕ [LATIN CAPITAL LETTER P WITH ACUTE] -"\u1E54" => "P" - -# Ṗ [LATIN CAPITAL LETTER P WITH DOT ABOVE] -"\u1E56" => "P" - -# Ⓟ [CIRCLED LATIN CAPITAL LETTER P] -"\u24C5" => "P" - -# Ᵽ [LATIN CAPITAL LETTER P WITH STROKE] -"\u2C63" => "P" - -# Ꝑ [LATIN CAPITAL LETTER P WITH STROKE THROUGH DESCENDER] -"\uA750" => "P" - -# Ꝓ [LATIN CAPITAL LETTER P WITH FLOURISH] -"\uA752" => "P" - -# Ꝕ [LATIN CAPITAL LETTER P WITH SQUIRREL TAIL] -"\uA754" => "P" - -# P [FULLWIDTH LATIN CAPITAL LETTER P] -"\uFF30" => "P" - -# ƥ [LATIN SMALL LETTER P WITH HOOK] -"\u01A5" => "p" - -# ᵱ [LATIN SMALL LETTER P WITH MIDDLE TILDE] -"\u1D71" => "p" - -# ᵽ [LATIN SMALL LETTER P WITH STROKE] -"\u1D7D" => "p" - -# ᶈ [LATIN SMALL LETTER P WITH PALATAL HOOK] -"\u1D88" => "p" - -# ṕ [LATIN SMALL LETTER P WITH ACUTE] -"\u1E55" => "p" - -# ṗ [LATIN SMALL LETTER P WITH DOT ABOVE] -"\u1E57" => "p" - -# ⓟ [CIRCLED LATIN SMALL LETTER P] -"\u24DF" => "p" - -# ꝑ [LATIN SMALL LETTER P WITH STROKE THROUGH DESCENDER] -"\uA751" => "p" - -# ꝓ [LATIN SMALL LETTER P WITH FLOURISH] -"\uA753" => "p" - -# ꝕ [LATIN SMALL LETTER P WITH SQUIRREL TAIL] -"\uA755" => "p" - -# ꟼ [LATIN EPIGRAPHIC LETTER REVERSED P] -"\uA7FC" => "p" - -# p [FULLWIDTH LATIN SMALL LETTER P] -"\uFF50" => "p" - -# ⒫ [PARENTHESIZED LATIN SMALL LETTER P] -"\u24AB" => "(p)" - -# Ɋ [LATIN CAPITAL LETTER SMALL Q WITH HOOK TAIL] -"\u024A" => "Q" - -# Ⓠ [CIRCLED LATIN CAPITAL LETTER Q] -"\u24C6" => "Q" - -# Ꝗ [LATIN CAPITAL LETTER Q WITH STROKE THROUGH DESCENDER] -"\uA756" => "Q" - -# Ꝙ [LATIN CAPITAL LETTER Q WITH DIAGONAL STROKE] -"\uA758" => "Q" - -# Q [FULLWIDTH LATIN CAPITAL LETTER Q] -"\uFF31" => "Q" - -# ĸ http://en.wikipedia.org/wiki/Kra_(letter) [LATIN SMALL LETTER KRA] -"\u0138" => "q" - -# ɋ [LATIN SMALL LETTER Q WITH HOOK TAIL] -"\u024B" => "q" - -# ʠ [LATIN SMALL LETTER Q WITH HOOK] -"\u02A0" => "q" - -# ⓠ [CIRCLED LATIN SMALL LETTER Q] -"\u24E0" => "q" - -# ꝗ [LATIN SMALL LETTER Q WITH STROKE THROUGH DESCENDER] -"\uA757" => "q" - -# ꝙ [LATIN SMALL LETTER Q WITH DIAGONAL STROKE] -"\uA759" => "q" - -# q [FULLWIDTH LATIN SMALL LETTER Q] -"\uFF51" => "q" - -# ⒬ [PARENTHESIZED LATIN SMALL LETTER Q] -"\u24AC" => "(q)" - -# ȹ [LATIN SMALL LETTER QP DIGRAPH] -"\u0239" => "qp" - -# Ŕ [LATIN CAPITAL LETTER R WITH ACUTE] -"\u0154" => "R" - -# Ŗ [LATIN CAPITAL LETTER R WITH CEDILLA] -"\u0156" => "R" - -# Ř [LATIN CAPITAL LETTER R WITH CARON] -"\u0158" => "R" - -# Ȓ [LATIN CAPITAL LETTER R WITH DOUBLE GRAVE] -"\u0210" => "R" - -# Ȓ [LATIN CAPITAL LETTER R WITH INVERTED BREVE] -"\u0212" => "R" - -# Ɍ [LATIN CAPITAL LETTER R WITH STROKE] -"\u024C" => "R" - -# ʀ [LATIN LETTER SMALL CAPITAL R] -"\u0280" => "R" - -# ʁ [LATIN LETTER SMALL CAPITAL INVERTED R] -"\u0281" => "R" - -# ᴙ [LATIN LETTER SMALL CAPITAL REVERSED R] -"\u1D19" => "R" - -# ᴚ [LATIN LETTER SMALL CAPITAL TURNED R] -"\u1D1A" => "R" - -# Ṙ [LATIN CAPITAL LETTER R WITH DOT ABOVE] -"\u1E58" => "R" - -# Ṛ [LATIN CAPITAL LETTER R WITH DOT BELOW] -"\u1E5A" => "R" - -# Ṝ [LATIN CAPITAL LETTER R WITH DOT BELOW AND MACRON] -"\u1E5C" => "R" - -# Ṟ [LATIN CAPITAL LETTER R WITH LINE BELOW] -"\u1E5E" => "R" - -# Ⓡ [CIRCLED LATIN CAPITAL LETTER R] -"\u24C7" => "R" - -# Ɽ [LATIN CAPITAL LETTER R WITH TAIL] -"\u2C64" => "R" - -# Ꝛ [LATIN CAPITAL LETTER R ROTUNDA] -"\uA75A" => "R" - -# Ꞃ [LATIN CAPITAL LETTER INSULAR R] -"\uA782" => "R" - -# R [FULLWIDTH LATIN CAPITAL LETTER R] -"\uFF32" => "R" - -# ŕ [LATIN SMALL LETTER R WITH ACUTE] -"\u0155" => "r" - -# ŗ [LATIN SMALL LETTER R WITH CEDILLA] -"\u0157" => "r" - -# ř [LATIN SMALL LETTER R WITH CARON] -"\u0159" => "r" - -# ȑ [LATIN SMALL LETTER R WITH DOUBLE GRAVE] -"\u0211" => "r" - -# ȓ [LATIN SMALL LETTER R WITH INVERTED BREVE] -"\u0213" => "r" - -# ɍ [LATIN SMALL LETTER R WITH STROKE] -"\u024D" => "r" - -# ɼ [LATIN SMALL LETTER R WITH LONG LEG] -"\u027C" => "r" - -# ɽ [LATIN SMALL LETTER R WITH TAIL] -"\u027D" => "r" - -# ɾ [LATIN SMALL LETTER R WITH FISHHOOK] -"\u027E" => "r" - -# ɿ [LATIN SMALL LETTER REVERSED R WITH FISHHOOK] -"\u027F" => "r" - -# ᵣ [LATIN SUBSCRIPT SMALL LETTER R] -"\u1D63" => "r" - -# ᵲ [LATIN SMALL LETTER R WITH MIDDLE TILDE] -"\u1D72" => "r" - -# ᵳ [LATIN SMALL LETTER R WITH FISHHOOK AND MIDDLE TILDE] -"\u1D73" => "r" - -# ᶉ [LATIN SMALL LETTER R WITH PALATAL HOOK] -"\u1D89" => "r" - -# ṙ [LATIN SMALL LETTER R WITH DOT ABOVE] -"\u1E59" => "r" - -# ṛ [LATIN SMALL LETTER R WITH DOT BELOW] -"\u1E5B" => "r" - -# ṝ [LATIN SMALL LETTER R WITH DOT BELOW AND MACRON] -"\u1E5D" => "r" - -# ṟ [LATIN SMALL LETTER R WITH LINE BELOW] -"\u1E5F" => "r" - -# ⓡ [CIRCLED LATIN SMALL LETTER R] -"\u24E1" => "r" - -# ꝛ [LATIN SMALL LETTER R ROTUNDA] -"\uA75B" => "r" - -# ꞃ [LATIN SMALL LETTER INSULAR R] -"\uA783" => "r" - -# r [FULLWIDTH LATIN SMALL LETTER R] -"\uFF52" => "r" - -# ⒭ [PARENTHESIZED LATIN SMALL LETTER R] -"\u24AD" => "(r)" - -# Ś [LATIN CAPITAL LETTER S WITH ACUTE] -"\u015A" => "S" - -# Ŝ [LATIN CAPITAL LETTER S WITH CIRCUMFLEX] -"\u015C" => "S" - -# Ş [LATIN CAPITAL LETTER S WITH CEDILLA] -"\u015E" => "S" - -# Š [LATIN CAPITAL LETTER S WITH CARON] -"\u0160" => "S" - -# Ș [LATIN CAPITAL LETTER S WITH COMMA BELOW] -"\u0218" => "S" - -# Ṡ [LATIN CAPITAL LETTER S WITH DOT ABOVE] -"\u1E60" => "S" - -# Ṣ [LATIN CAPITAL LETTER S WITH DOT BELOW] -"\u1E62" => "S" - -# Ṥ [LATIN CAPITAL LETTER S WITH ACUTE AND DOT ABOVE] -"\u1E64" => "S" - -# Ṧ [LATIN CAPITAL LETTER S WITH CARON AND DOT ABOVE] -"\u1E66" => "S" - -# Ṩ [LATIN CAPITAL LETTER S WITH DOT BELOW AND DOT ABOVE] -"\u1E68" => "S" - -# Ⓢ [CIRCLED LATIN CAPITAL LETTER S] -"\u24C8" => "S" - -# ꜱ [LATIN LETTER SMALL CAPITAL S] -"\uA731" => "S" - -# ꞅ [LATIN SMALL LETTER INSULAR S] -"\uA785" => "S" - -# S [FULLWIDTH LATIN CAPITAL LETTER S] -"\uFF33" => "S" - -# ś [LATIN SMALL LETTER S WITH ACUTE] -"\u015B" => "s" - -# ŝ [LATIN SMALL LETTER S WITH CIRCUMFLEX] -"\u015D" => "s" - -# ş [LATIN SMALL LETTER S WITH CEDILLA] -"\u015F" => "s" - -# š [LATIN SMALL LETTER S WITH CARON] -"\u0161" => "s" - -# ſ http://en.wikipedia.org/wiki/Long_S [LATIN SMALL LETTER LONG S] -"\u017F" => "s" - -# ș [LATIN SMALL LETTER S WITH COMMA BELOW] -"\u0219" => "s" - -# ȿ [LATIN SMALL LETTER S WITH SWASH TAIL] -"\u023F" => "s" - -# ʂ [LATIN SMALL LETTER S WITH HOOK] -"\u0282" => "s" - -# ᵴ [LATIN SMALL LETTER S WITH MIDDLE TILDE] -"\u1D74" => "s" - -# ᶊ [LATIN SMALL LETTER S WITH PALATAL HOOK] -"\u1D8A" => "s" - -# ṡ [LATIN SMALL LETTER S WITH DOT ABOVE] -"\u1E61" => "s" - -# ṣ [LATIN SMALL LETTER S WITH DOT BELOW] -"\u1E63" => "s" - -# ṥ [LATIN SMALL LETTER S WITH ACUTE AND DOT ABOVE] -"\u1E65" => "s" - -# ṧ [LATIN SMALL LETTER S WITH CARON AND DOT ABOVE] -"\u1E67" => "s" - -# ṩ [LATIN SMALL LETTER S WITH DOT BELOW AND DOT ABOVE] -"\u1E69" => "s" - -# ẜ [LATIN SMALL LETTER LONG S WITH DIAGONAL STROKE] -"\u1E9C" => "s" - -# ẝ [LATIN SMALL LETTER LONG S WITH HIGH STROKE] -"\u1E9D" => "s" - -# ⓢ [CIRCLED LATIN SMALL LETTER S] -"\u24E2" => "s" - -# Ꞅ [LATIN CAPITAL LETTER INSULAR S] -"\uA784" => "s" - -# s [FULLWIDTH LATIN SMALL LETTER S] -"\uFF53" => "s" - -# ẞ [LATIN CAPITAL LETTER SHARP S] -"\u1E9E" => "SS" - -# ⒮ [PARENTHESIZED LATIN SMALL LETTER S] -"\u24AE" => "(s)" - -# ß [LATIN SMALL LETTER SHARP S] -"\u00DF" => "ss" - -# st [LATIN SMALL LIGATURE ST] -"\uFB06" => "st" - -# Ţ [LATIN CAPITAL LETTER T WITH CEDILLA] -"\u0162" => "T" - -# Ť [LATIN CAPITAL LETTER T WITH CARON] -"\u0164" => "T" - -# Ŧ [LATIN CAPITAL LETTER T WITH STROKE] -"\u0166" => "T" - -# Ƭ [LATIN CAPITAL LETTER T WITH HOOK] -"\u01AC" => "T" - -# Ʈ [LATIN CAPITAL LETTER T WITH RETROFLEX HOOK] -"\u01AE" => "T" - -# Ț [LATIN CAPITAL LETTER T WITH COMMA BELOW] -"\u021A" => "T" - -# Ⱦ [LATIN CAPITAL LETTER T WITH DIAGONAL STROKE] -"\u023E" => "T" - -# ᴛ [LATIN LETTER SMALL CAPITAL T] -"\u1D1B" => "T" - -# Ṫ [LATIN CAPITAL LETTER T WITH DOT ABOVE] -"\u1E6A" => "T" - -# Ṭ [LATIN CAPITAL LETTER T WITH DOT BELOW] -"\u1E6C" => "T" - -# Ṯ [LATIN CAPITAL LETTER T WITH LINE BELOW] -"\u1E6E" => "T" - -# Ṱ [LATIN CAPITAL LETTER T WITH CIRCUMFLEX BELOW] -"\u1E70" => "T" - -# Ⓣ [CIRCLED LATIN CAPITAL LETTER T] -"\u24C9" => "T" - -# Ꞇ [LATIN CAPITAL LETTER INSULAR T] -"\uA786" => "T" - -# T [FULLWIDTH LATIN CAPITAL LETTER T] -"\uFF34" => "T" - -# ţ [LATIN SMALL LETTER T WITH CEDILLA] -"\u0163" => "t" - -# ť [LATIN SMALL LETTER T WITH CARON] -"\u0165" => "t" - -# ŧ [LATIN SMALL LETTER T WITH STROKE] -"\u0167" => "t" - -# ƫ [LATIN SMALL LETTER T WITH PALATAL HOOK] -"\u01AB" => "t" - -# ƭ [LATIN SMALL LETTER T WITH HOOK] -"\u01AD" => "t" - -# ț [LATIN SMALL LETTER T WITH COMMA BELOW] -"\u021B" => "t" - -# ȶ [LATIN SMALL LETTER T WITH CURL] -"\u0236" => "t" - -# ʇ [LATIN SMALL LETTER TURNED T] -"\u0287" => "t" - -# ʈ [LATIN SMALL LETTER T WITH RETROFLEX HOOK] -"\u0288" => "t" - -# ᵵ [LATIN SMALL LETTER T WITH MIDDLE TILDE] -"\u1D75" => "t" - -# ṫ [LATIN SMALL LETTER T WITH DOT ABOVE] -"\u1E6B" => "t" - -# ṭ [LATIN SMALL LETTER T WITH DOT BELOW] -"\u1E6D" => "t" - -# ṯ [LATIN SMALL LETTER T WITH LINE BELOW] -"\u1E6F" => "t" - -# ṱ [LATIN SMALL LETTER T WITH CIRCUMFLEX BELOW] -"\u1E71" => "t" - -# ẗ [LATIN SMALL LETTER T WITH DIAERESIS] -"\u1E97" => "t" - -# ⓣ [CIRCLED LATIN SMALL LETTER T] -"\u24E3" => "t" - -# ⱦ [LATIN SMALL LETTER T WITH DIAGONAL STROKE] -"\u2C66" => "t" - -# t [FULLWIDTH LATIN SMALL LETTER T] -"\uFF54" => "t" - -# Þ [LATIN CAPITAL LETTER THORN] -"\u00DE" => "TH" - -# Ꝧ [LATIN CAPITAL LETTER THORN WITH STROKE THROUGH DESCENDER] -"\uA766" => "TH" - -# Ꜩ [LATIN CAPITAL LETTER TZ] -"\uA728" => "TZ" - -# ⒯ [PARENTHESIZED LATIN SMALL LETTER T] -"\u24AF" => "(t)" - -# ʨ [LATIN SMALL LETTER TC DIGRAPH WITH CURL] -"\u02A8" => "tc" - -# þ [LATIN SMALL LETTER THORN] -"\u00FE" => "th" - -# ᵺ [LATIN SMALL LETTER TH WITH STRIKETHROUGH] -"\u1D7A" => "th" - -# ꝧ [LATIN SMALL LETTER THORN WITH STROKE THROUGH DESCENDER] -"\uA767" => "th" - -# ʦ [LATIN SMALL LETTER TS DIGRAPH] -"\u02A6" => "ts" - -# ꜩ [LATIN SMALL LETTER TZ] -"\uA729" => "tz" - -# Ù [LATIN CAPITAL LETTER U WITH GRAVE] -"\u00D9" => "U" - -# Ú [LATIN CAPITAL LETTER U WITH ACUTE] -"\u00DA" => "U" - -# Û [LATIN CAPITAL LETTER U WITH CIRCUMFLEX] -"\u00DB" => "U" - -# Ü [LATIN CAPITAL LETTER U WITH DIAERESIS] -"\u00DC" => "U" - -# Ũ [LATIN CAPITAL LETTER U WITH TILDE] -"\u0168" => "U" - -# Ū [LATIN CAPITAL LETTER U WITH MACRON] -"\u016A" => "U" - -# Ŭ [LATIN CAPITAL LETTER U WITH BREVE] -"\u016C" => "U" - -# Ů [LATIN CAPITAL LETTER U WITH RING ABOVE] -"\u016E" => "U" - -# Ű [LATIN CAPITAL LETTER U WITH DOUBLE ACUTE] -"\u0170" => "U" - -# Ų [LATIN CAPITAL LETTER U WITH OGONEK] -"\u0172" => "U" - -# Ư [LATIN CAPITAL LETTER U WITH HORN] -"\u01AF" => "U" - -# Ǔ [LATIN CAPITAL LETTER U WITH CARON] -"\u01D3" => "U" - -# Ǖ [LATIN CAPITAL LETTER U WITH DIAERESIS AND MACRON] -"\u01D5" => "U" - -# Ǘ [LATIN CAPITAL LETTER U WITH DIAERESIS AND ACUTE] -"\u01D7" => "U" - -# Ǚ [LATIN CAPITAL LETTER U WITH DIAERESIS AND CARON] -"\u01D9" => "U" - -# Ǜ [LATIN CAPITAL LETTER U WITH DIAERESIS AND GRAVE] -"\u01DB" => "U" - -# Ȕ [LATIN CAPITAL LETTER U WITH DOUBLE GRAVE] -"\u0214" => "U" - -# Ȗ [LATIN CAPITAL LETTER U WITH INVERTED BREVE] -"\u0216" => "U" - -# Ʉ [LATIN CAPITAL LETTER U BAR] -"\u0244" => "U" - -# ᴜ [LATIN LETTER SMALL CAPITAL U] -"\u1D1C" => "U" - -# ᵾ [LATIN SMALL CAPITAL LETTER U WITH STROKE] -"\u1D7E" => "U" - -# Ṳ [LATIN CAPITAL LETTER U WITH DIAERESIS BELOW] -"\u1E72" => "U" - -# Ṵ [LATIN CAPITAL LETTER U WITH TILDE BELOW] -"\u1E74" => "U" - -# Ṷ [LATIN CAPITAL LETTER U WITH CIRCUMFLEX BELOW] -"\u1E76" => "U" - -# Ṹ [LATIN CAPITAL LETTER U WITH TILDE AND ACUTE] -"\u1E78" => "U" - -# Ṻ [LATIN CAPITAL LETTER U WITH MACRON AND DIAERESIS] -"\u1E7A" => "U" - -# Ụ [LATIN CAPITAL LETTER U WITH DOT BELOW] -"\u1EE4" => "U" - -# Ủ [LATIN CAPITAL LETTER U WITH HOOK ABOVE] -"\u1EE6" => "U" - -# Ứ [LATIN CAPITAL LETTER U WITH HORN AND ACUTE] -"\u1EE8" => "U" - -# Ừ [LATIN CAPITAL LETTER U WITH HORN AND GRAVE] -"\u1EEA" => "U" - -# Ử [LATIN CAPITAL LETTER U WITH HORN AND HOOK ABOVE] -"\u1EEC" => "U" - -# Ữ [LATIN CAPITAL LETTER U WITH HORN AND TILDE] -"\u1EEE" => "U" - -# Ự [LATIN CAPITAL LETTER U WITH HORN AND DOT BELOW] -"\u1EF0" => "U" - -# Ⓤ [CIRCLED LATIN CAPITAL LETTER U] -"\u24CA" => "U" - -# U [FULLWIDTH LATIN CAPITAL LETTER U] -"\uFF35" => "U" - -# ù [LATIN SMALL LETTER U WITH GRAVE] -"\u00F9" => "u" - -# ú [LATIN SMALL LETTER U WITH ACUTE] -"\u00FA" => "u" - -# û [LATIN SMALL LETTER U WITH CIRCUMFLEX] -"\u00FB" => "u" - -# ü [LATIN SMALL LETTER U WITH DIAERESIS] -"\u00FC" => "u" - -# ũ [LATIN SMALL LETTER U WITH TILDE] -"\u0169" => "u" - -# ū [LATIN SMALL LETTER U WITH MACRON] -"\u016B" => "u" - -# ŭ [LATIN SMALL LETTER U WITH BREVE] -"\u016D" => "u" - -# ů [LATIN SMALL LETTER U WITH RING ABOVE] -"\u016F" => "u" - -# ű [LATIN SMALL LETTER U WITH DOUBLE ACUTE] -"\u0171" => "u" - -# ų [LATIN SMALL LETTER U WITH OGONEK] -"\u0173" => "u" - -# ư [LATIN SMALL LETTER U WITH HORN] -"\u01B0" => "u" - -# ǔ [LATIN SMALL LETTER U WITH CARON] -"\u01D4" => "u" - -# ǖ [LATIN SMALL LETTER U WITH DIAERESIS AND MACRON] -"\u01D6" => "u" - -# ǘ [LATIN SMALL LETTER U WITH DIAERESIS AND ACUTE] -"\u01D8" => "u" - -# ǚ [LATIN SMALL LETTER U WITH DIAERESIS AND CARON] -"\u01DA" => "u" - -# ǜ [LATIN SMALL LETTER U WITH DIAERESIS AND GRAVE] -"\u01DC" => "u" - -# ȕ [LATIN SMALL LETTER U WITH DOUBLE GRAVE] -"\u0215" => "u" - -# ȗ [LATIN SMALL LETTER U WITH INVERTED BREVE] -"\u0217" => "u" - -# ʉ [LATIN SMALL LETTER U BAR] -"\u0289" => "u" - -# ᵤ [LATIN SUBSCRIPT SMALL LETTER U] -"\u1D64" => "u" - -# ᶙ [LATIN SMALL LETTER U WITH RETROFLEX HOOK] -"\u1D99" => "u" - -# ṳ [LATIN SMALL LETTER U WITH DIAERESIS BELOW] -"\u1E73" => "u" - -# ṵ [LATIN SMALL LETTER U WITH TILDE BELOW] -"\u1E75" => "u" - -# ṷ [LATIN SMALL LETTER U WITH CIRCUMFLEX BELOW] -"\u1E77" => "u" - -# ṹ [LATIN SMALL LETTER U WITH TILDE AND ACUTE] -"\u1E79" => "u" - -# ṻ [LATIN SMALL LETTER U WITH MACRON AND DIAERESIS] -"\u1E7B" => "u" - -# ụ [LATIN SMALL LETTER U WITH DOT BELOW] -"\u1EE5" => "u" - -# ủ [LATIN SMALL LETTER U WITH HOOK ABOVE] -"\u1EE7" => "u" - -# ứ [LATIN SMALL LETTER U WITH HORN AND ACUTE] -"\u1EE9" => "u" - -# ừ [LATIN SMALL LETTER U WITH HORN AND GRAVE] -"\u1EEB" => "u" - -# ử [LATIN SMALL LETTER U WITH HORN AND HOOK ABOVE] -"\u1EED" => "u" - -# ữ [LATIN SMALL LETTER U WITH HORN AND TILDE] -"\u1EEF" => "u" - -# ự [LATIN SMALL LETTER U WITH HORN AND DOT BELOW] -"\u1EF1" => "u" - -# ⓤ [CIRCLED LATIN SMALL LETTER U] -"\u24E4" => "u" - -# u [FULLWIDTH LATIN SMALL LETTER U] -"\uFF55" => "u" - -# ⒰ [PARENTHESIZED LATIN SMALL LETTER U] -"\u24B0" => "(u)" - -# ᵫ [LATIN SMALL LETTER UE] -"\u1D6B" => "ue" - -# Ʋ [LATIN CAPITAL LETTER V WITH HOOK] -"\u01B2" => "V" - -# Ʌ [LATIN CAPITAL LETTER TURNED V] -"\u0245" => "V" - -# ᴠ [LATIN LETTER SMALL CAPITAL V] -"\u1D20" => "V" - -# Ṽ [LATIN CAPITAL LETTER V WITH TILDE] -"\u1E7C" => "V" - -# Ṿ [LATIN CAPITAL LETTER V WITH DOT BELOW] -"\u1E7E" => "V" - -# Ỽ [LATIN CAPITAL LETTER MIDDLE-WELSH V] -"\u1EFC" => "V" - -# Ⓥ [CIRCLED LATIN CAPITAL LETTER V] -"\u24CB" => "V" - -# Ꝟ [LATIN CAPITAL LETTER V WITH DIAGONAL STROKE] -"\uA75E" => "V" - -# Ꝩ [LATIN CAPITAL LETTER VEND] -"\uA768" => "V" - -# V [FULLWIDTH LATIN CAPITAL LETTER V] -"\uFF36" => "V" - -# ʋ [LATIN SMALL LETTER V WITH HOOK] -"\u028B" => "v" - -# ʌ [LATIN SMALL LETTER TURNED V] -"\u028C" => "v" - -# ᵥ [LATIN SUBSCRIPT SMALL LETTER V] -"\u1D65" => "v" - -# ᶌ [LATIN SMALL LETTER V WITH PALATAL HOOK] -"\u1D8C" => "v" - -# ṽ [LATIN SMALL LETTER V WITH TILDE] -"\u1E7D" => "v" - -# ṿ [LATIN SMALL LETTER V WITH DOT BELOW] -"\u1E7F" => "v" - -# ⓥ [CIRCLED LATIN SMALL LETTER V] -"\u24E5" => "v" - -# ⱱ [LATIN SMALL LETTER V WITH RIGHT HOOK] -"\u2C71" => "v" - -# ⱴ [LATIN SMALL LETTER V WITH CURL] -"\u2C74" => "v" - -# ꝟ [LATIN SMALL LETTER V WITH DIAGONAL STROKE] -"\uA75F" => "v" - -# v [FULLWIDTH LATIN SMALL LETTER V] -"\uFF56" => "v" - -# Ꝡ [LATIN CAPITAL LETTER VY] -"\uA760" => "VY" - -# ⒱ [PARENTHESIZED LATIN SMALL LETTER V] -"\u24B1" => "(v)" - -# ꝡ [LATIN SMALL LETTER VY] -"\uA761" => "vy" - -# Ŵ [LATIN CAPITAL LETTER W WITH CIRCUMFLEX] -"\u0174" => "W" - -# Ƿ http://en.wikipedia.org/wiki/Wynn [LATIN CAPITAL LETTER WYNN] -"\u01F7" => "W" - -# ᴡ [LATIN LETTER SMALL CAPITAL W] -"\u1D21" => "W" - -# Ẁ [LATIN CAPITAL LETTER W WITH GRAVE] -"\u1E80" => "W" - -# Ẃ [LATIN CAPITAL LETTER W WITH ACUTE] -"\u1E82" => "W" - -# Ẅ [LATIN CAPITAL LETTER W WITH DIAERESIS] -"\u1E84" => "W" - -# Ẇ [LATIN CAPITAL LETTER W WITH DOT ABOVE] -"\u1E86" => "W" - -# Ẉ [LATIN CAPITAL LETTER W WITH DOT BELOW] -"\u1E88" => "W" - -# Ⓦ [CIRCLED LATIN CAPITAL LETTER W] -"\u24CC" => "W" - -# Ⱳ [LATIN CAPITAL LETTER W WITH HOOK] -"\u2C72" => "W" - -# W [FULLWIDTH LATIN CAPITAL LETTER W] -"\uFF37" => "W" - -# ŵ [LATIN SMALL LETTER W WITH CIRCUMFLEX] -"\u0175" => "w" - -# ƿ http://en.wikipedia.org/wiki/Wynn [LATIN LETTER WYNN] -"\u01BF" => "w" - -# ʍ [LATIN SMALL LETTER TURNED W] -"\u028D" => "w" - -# ẁ [LATIN SMALL LETTER W WITH GRAVE] -"\u1E81" => "w" - -# ẃ [LATIN SMALL LETTER W WITH ACUTE] -"\u1E83" => "w" - -# ẅ [LATIN SMALL LETTER W WITH DIAERESIS] -"\u1E85" => "w" - -# ẇ [LATIN SMALL LETTER W WITH DOT ABOVE] -"\u1E87" => "w" - -# ẉ [LATIN SMALL LETTER W WITH DOT BELOW] -"\u1E89" => "w" - -# ẘ [LATIN SMALL LETTER W WITH RING ABOVE] -"\u1E98" => "w" - -# ⓦ [CIRCLED LATIN SMALL LETTER W] -"\u24E6" => "w" - -# ⱳ [LATIN SMALL LETTER W WITH HOOK] -"\u2C73" => "w" - -# w [FULLWIDTH LATIN SMALL LETTER W] -"\uFF57" => "w" - -# ⒲ [PARENTHESIZED LATIN SMALL LETTER W] -"\u24B2" => "(w)" - -# Ẋ [LATIN CAPITAL LETTER X WITH DOT ABOVE] -"\u1E8A" => "X" - -# Ẍ [LATIN CAPITAL LETTER X WITH DIAERESIS] -"\u1E8C" => "X" - -# Ⓧ [CIRCLED LATIN CAPITAL LETTER X] -"\u24CD" => "X" - -# X [FULLWIDTH LATIN CAPITAL LETTER X] -"\uFF38" => "X" - -# ᶍ [LATIN SMALL LETTER X WITH PALATAL HOOK] -"\u1D8D" => "x" - -# ẋ [LATIN SMALL LETTER X WITH DOT ABOVE] -"\u1E8B" => "x" - -# ẍ [LATIN SMALL LETTER X WITH DIAERESIS] -"\u1E8D" => "x" - -# ₓ [LATIN SUBSCRIPT SMALL LETTER X] -"\u2093" => "x" - -# ⓧ [CIRCLED LATIN SMALL LETTER X] -"\u24E7" => "x" - -# x [FULLWIDTH LATIN SMALL LETTER X] -"\uFF58" => "x" - -# ⒳ [PARENTHESIZED LATIN SMALL LETTER X] -"\u24B3" => "(x)" - -# Ý [LATIN CAPITAL LETTER Y WITH ACUTE] -"\u00DD" => "Y" - -# Ŷ [LATIN CAPITAL LETTER Y WITH CIRCUMFLEX] -"\u0176" => "Y" - -# Ÿ [LATIN CAPITAL LETTER Y WITH DIAERESIS] -"\u0178" => "Y" - -# Ƴ [LATIN CAPITAL LETTER Y WITH HOOK] -"\u01B3" => "Y" - -# Ȳ [LATIN CAPITAL LETTER Y WITH MACRON] -"\u0232" => "Y" - -# Ɏ [LATIN CAPITAL LETTER Y WITH STROKE] -"\u024E" => "Y" - -# ʏ [LATIN LETTER SMALL CAPITAL Y] -"\u028F" => "Y" - -# Ẏ [LATIN CAPITAL LETTER Y WITH DOT ABOVE] -"\u1E8E" => "Y" - -# Ỳ [LATIN CAPITAL LETTER Y WITH GRAVE] -"\u1EF2" => "Y" - -# Ỵ [LATIN CAPITAL LETTER Y WITH DOT BELOW] -"\u1EF4" => "Y" - -# Ỷ [LATIN CAPITAL LETTER Y WITH HOOK ABOVE] -"\u1EF6" => "Y" - -# Ỹ [LATIN CAPITAL LETTER Y WITH TILDE] -"\u1EF8" => "Y" - -# Ỿ [LATIN CAPITAL LETTER Y WITH LOOP] -"\u1EFE" => "Y" - -# Ⓨ [CIRCLED LATIN CAPITAL LETTER Y] -"\u24CE" => "Y" - -# Y [FULLWIDTH LATIN CAPITAL LETTER Y] -"\uFF39" => "Y" - -# ý [LATIN SMALL LETTER Y WITH ACUTE] -"\u00FD" => "y" - -# ÿ [LATIN SMALL LETTER Y WITH DIAERESIS] -"\u00FF" => "y" - -# ŷ [LATIN SMALL LETTER Y WITH CIRCUMFLEX] -"\u0177" => "y" - -# ƴ [LATIN SMALL LETTER Y WITH HOOK] -"\u01B4" => "y" - -# ȳ [LATIN SMALL LETTER Y WITH MACRON] -"\u0233" => "y" - -# ɏ [LATIN SMALL LETTER Y WITH STROKE] -"\u024F" => "y" - -# ʎ [LATIN SMALL LETTER TURNED Y] -"\u028E" => "y" - -# ẏ [LATIN SMALL LETTER Y WITH DOT ABOVE] -"\u1E8F" => "y" - -# ẙ [LATIN SMALL LETTER Y WITH RING ABOVE] -"\u1E99" => "y" - -# ỳ [LATIN SMALL LETTER Y WITH GRAVE] -"\u1EF3" => "y" - -# ỵ [LATIN SMALL LETTER Y WITH DOT BELOW] -"\u1EF5" => "y" - -# ỷ [LATIN SMALL LETTER Y WITH HOOK ABOVE] -"\u1EF7" => "y" - -# ỹ [LATIN SMALL LETTER Y WITH TILDE] -"\u1EF9" => "y" - -# ỿ [LATIN SMALL LETTER Y WITH LOOP] -"\u1EFF" => "y" - -# ⓨ [CIRCLED LATIN SMALL LETTER Y] -"\u24E8" => "y" - -# y [FULLWIDTH LATIN SMALL LETTER Y] -"\uFF59" => "y" - -# ⒴ [PARENTHESIZED LATIN SMALL LETTER Y] -"\u24B4" => "(y)" - -# Ź [LATIN CAPITAL LETTER Z WITH ACUTE] -"\u0179" => "Z" - -# Ż [LATIN CAPITAL LETTER Z WITH DOT ABOVE] -"\u017B" => "Z" - -# Ž [LATIN CAPITAL LETTER Z WITH CARON] -"\u017D" => "Z" - -# Ƶ [LATIN CAPITAL LETTER Z WITH STROKE] -"\u01B5" => "Z" - -# Ȝ http://en.wikipedia.org/wiki/Yogh [LATIN CAPITAL LETTER YOGH] -"\u021C" => "Z" - -# Ȥ [LATIN CAPITAL LETTER Z WITH HOOK] -"\u0224" => "Z" - -# ᴢ [LATIN LETTER SMALL CAPITAL Z] -"\u1D22" => "Z" - -# Ẑ [LATIN CAPITAL LETTER Z WITH CIRCUMFLEX] -"\u1E90" => "Z" - -# Ẓ [LATIN CAPITAL LETTER Z WITH DOT BELOW] -"\u1E92" => "Z" - -# Ẕ [LATIN CAPITAL LETTER Z WITH LINE BELOW] -"\u1E94" => "Z" - -# Ⓩ [CIRCLED LATIN CAPITAL LETTER Z] -"\u24CF" => "Z" - -# Ⱬ [LATIN CAPITAL LETTER Z WITH DESCENDER] -"\u2C6B" => "Z" - -# Ꝣ [LATIN CAPITAL LETTER VISIGOTHIC Z] -"\uA762" => "Z" - -# Z [FULLWIDTH LATIN CAPITAL LETTER Z] -"\uFF3A" => "Z" - -# ź [LATIN SMALL LETTER Z WITH ACUTE] -"\u017A" => "z" - -# ż [LATIN SMALL LETTER Z WITH DOT ABOVE] -"\u017C" => "z" - -# ž [LATIN SMALL LETTER Z WITH CARON] -"\u017E" => "z" - -# ƶ [LATIN SMALL LETTER Z WITH STROKE] -"\u01B6" => "z" - -# ȝ http://en.wikipedia.org/wiki/Yogh [LATIN SMALL LETTER YOGH] -"\u021D" => "z" - -# ȥ [LATIN SMALL LETTER Z WITH HOOK] -"\u0225" => "z" - -# ɀ [LATIN SMALL LETTER Z WITH SWASH TAIL] -"\u0240" => "z" - -# ʐ [LATIN SMALL LETTER Z WITH RETROFLEX HOOK] -"\u0290" => "z" - -# ʑ [LATIN SMALL LETTER Z WITH CURL] -"\u0291" => "z" - -# ᵶ [LATIN SMALL LETTER Z WITH MIDDLE TILDE] -"\u1D76" => "z" - -# ᶎ [LATIN SMALL LETTER Z WITH PALATAL HOOK] -"\u1D8E" => "z" - -# ẑ [LATIN SMALL LETTER Z WITH CIRCUMFLEX] -"\u1E91" => "z" - -# ẓ [LATIN SMALL LETTER Z WITH DOT BELOW] -"\u1E93" => "z" - -# ẕ [LATIN SMALL LETTER Z WITH LINE BELOW] -"\u1E95" => "z" - -# ⓩ [CIRCLED LATIN SMALL LETTER Z] -"\u24E9" => "z" - -# ⱬ [LATIN SMALL LETTER Z WITH DESCENDER] -"\u2C6C" => "z" - -# ꝣ [LATIN SMALL LETTER VISIGOTHIC Z] -"\uA763" => "z" - -# z [FULLWIDTH LATIN SMALL LETTER Z] -"\uFF5A" => "z" - -# ⒵ [PARENTHESIZED LATIN SMALL LETTER Z] -"\u24B5" => "(z)" - -# ⁰ [SUPERSCRIPT ZERO] -"\u2070" => "0" - -# ₀ [SUBSCRIPT ZERO] -"\u2080" => "0" - -# ⓪ [CIRCLED DIGIT ZERO] -"\u24EA" => "0" - -# ⓿ [NEGATIVE CIRCLED DIGIT ZERO] -"\u24FF" => "0" - -# 0 [FULLWIDTH DIGIT ZERO] -"\uFF10" => "0" - -# ¹ [SUPERSCRIPT ONE] -"\u00B9" => "1" - -# ₁ [SUBSCRIPT ONE] -"\u2081" => "1" - -# ① [CIRCLED DIGIT ONE] -"\u2460" => "1" - -# ⓵ [DOUBLE CIRCLED DIGIT ONE] -"\u24F5" => "1" - -# ❶ [DINGBAT NEGATIVE CIRCLED DIGIT ONE] -"\u2776" => "1" - -# ➀ [DINGBAT CIRCLED SANS-SERIF DIGIT ONE] -"\u2780" => "1" - -# ➊ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT ONE] -"\u278A" => "1" - -# 1 [FULLWIDTH DIGIT ONE] -"\uFF11" => "1" - -# ⒈ [DIGIT ONE FULL STOP] -"\u2488" => "1." - -# ⑴ [PARENTHESIZED DIGIT ONE] -"\u2474" => "(1)" - -# ² [SUPERSCRIPT TWO] -"\u00B2" => "2" - -# ₂ [SUBSCRIPT TWO] -"\u2082" => "2" - -# ② [CIRCLED DIGIT TWO] -"\u2461" => "2" - -# ⓶ [DOUBLE CIRCLED DIGIT TWO] -"\u24F6" => "2" - -# ❷ [DINGBAT NEGATIVE CIRCLED DIGIT TWO] -"\u2777" => "2" - -# ➁ [DINGBAT CIRCLED SANS-SERIF DIGIT TWO] -"\u2781" => "2" - -# ➋ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT TWO] -"\u278B" => "2" - -# 2 [FULLWIDTH DIGIT TWO] -"\uFF12" => "2" - -# ⒉ [DIGIT TWO FULL STOP] -"\u2489" => "2." - -# ⑵ [PARENTHESIZED DIGIT TWO] -"\u2475" => "(2)" - -# ³ [SUPERSCRIPT THREE] -"\u00B3" => "3" - -# ₃ [SUBSCRIPT THREE] -"\u2083" => "3" - -# ③ [CIRCLED DIGIT THREE] -"\u2462" => "3" - -# ⓷ [DOUBLE CIRCLED DIGIT THREE] -"\u24F7" => "3" - -# ❸ [DINGBAT NEGATIVE CIRCLED DIGIT THREE] -"\u2778" => "3" - -# ➂ [DINGBAT CIRCLED SANS-SERIF DIGIT THREE] -"\u2782" => "3" - -# ➌ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT THREE] -"\u278C" => "3" - -# 3 [FULLWIDTH DIGIT THREE] -"\uFF13" => "3" - -# ⒊ [DIGIT THREE FULL STOP] -"\u248A" => "3." - -# ⑶ [PARENTHESIZED DIGIT THREE] -"\u2476" => "(3)" - -# ⁴ [SUPERSCRIPT FOUR] -"\u2074" => "4" - -# ₄ [SUBSCRIPT FOUR] -"\u2084" => "4" - -# ④ [CIRCLED DIGIT FOUR] -"\u2463" => "4" - -# ⓸ [DOUBLE CIRCLED DIGIT FOUR] -"\u24F8" => "4" - -# ❹ [DINGBAT NEGATIVE CIRCLED DIGIT FOUR] -"\u2779" => "4" - -# ➃ [DINGBAT CIRCLED SANS-SERIF DIGIT FOUR] -"\u2783" => "4" - -# ➍ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT FOUR] -"\u278D" => "4" - -# 4 [FULLWIDTH DIGIT FOUR] -"\uFF14" => "4" - -# ⒋ [DIGIT FOUR FULL STOP] -"\u248B" => "4." - -# ⑷ [PARENTHESIZED DIGIT FOUR] -"\u2477" => "(4)" - -# ⁵ [SUPERSCRIPT FIVE] -"\u2075" => "5" - -# ₅ [SUBSCRIPT FIVE] -"\u2085" => "5" - -# ⑤ [CIRCLED DIGIT FIVE] -"\u2464" => "5" - -# ⓹ [DOUBLE CIRCLED DIGIT FIVE] -"\u24F9" => "5" - -# ❺ [DINGBAT NEGATIVE CIRCLED DIGIT FIVE] -"\u277A" => "5" - -# ➄ [DINGBAT CIRCLED SANS-SERIF DIGIT FIVE] -"\u2784" => "5" - -# ➎ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT FIVE] -"\u278E" => "5" - -# 5 [FULLWIDTH DIGIT FIVE] -"\uFF15" => "5" - -# ⒌ [DIGIT FIVE FULL STOP] -"\u248C" => "5." - -# ⑸ [PARENTHESIZED DIGIT FIVE] -"\u2478" => "(5)" - -# ⁶ [SUPERSCRIPT SIX] -"\u2076" => "6" - -# ₆ [SUBSCRIPT SIX] -"\u2086" => "6" - -# ⑥ [CIRCLED DIGIT SIX] -"\u2465" => "6" - -# ⓺ [DOUBLE CIRCLED DIGIT SIX] -"\u24FA" => "6" - -# ❻ [DINGBAT NEGATIVE CIRCLED DIGIT SIX] -"\u277B" => "6" - -# ➅ [DINGBAT CIRCLED SANS-SERIF DIGIT SIX] -"\u2785" => "6" - -# ➏ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT SIX] -"\u278F" => "6" - -# 6 [FULLWIDTH DIGIT SIX] -"\uFF16" => "6" - -# ⒍ [DIGIT SIX FULL STOP] -"\u248D" => "6." - -# ⑹ [PARENTHESIZED DIGIT SIX] -"\u2479" => "(6)" - -# ⁷ [SUPERSCRIPT SEVEN] -"\u2077" => "7" - -# ₇ [SUBSCRIPT SEVEN] -"\u2087" => "7" - -# ⑦ [CIRCLED DIGIT SEVEN] -"\u2466" => "7" - -# ⓻ [DOUBLE CIRCLED DIGIT SEVEN] -"\u24FB" => "7" - -# ❼ [DINGBAT NEGATIVE CIRCLED DIGIT SEVEN] -"\u277C" => "7" - -# ➆ [DINGBAT CIRCLED SANS-SERIF DIGIT SEVEN] -"\u2786" => "7" - -# ➐ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT SEVEN] -"\u2790" => "7" - -# 7 [FULLWIDTH DIGIT SEVEN] -"\uFF17" => "7" - -# ⒎ [DIGIT SEVEN FULL STOP] -"\u248E" => "7." - -# ⑺ [PARENTHESIZED DIGIT SEVEN] -"\u247A" => "(7)" - -# ⁸ [SUPERSCRIPT EIGHT] -"\u2078" => "8" - -# ₈ [SUBSCRIPT EIGHT] -"\u2088" => "8" - -# ⑧ [CIRCLED DIGIT EIGHT] -"\u2467" => "8" - -# ⓼ [DOUBLE CIRCLED DIGIT EIGHT] -"\u24FC" => "8" - -# ❽ [DINGBAT NEGATIVE CIRCLED DIGIT EIGHT] -"\u277D" => "8" - -# ➇ [DINGBAT CIRCLED SANS-SERIF DIGIT EIGHT] -"\u2787" => "8" - -# ➑ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT EIGHT] -"\u2791" => "8" - -# 8 [FULLWIDTH DIGIT EIGHT] -"\uFF18" => "8" - -# ⒏ [DIGIT EIGHT FULL STOP] -"\u248F" => "8." - -# ⑻ [PARENTHESIZED DIGIT EIGHT] -"\u247B" => "(8)" - -# ⁹ [SUPERSCRIPT NINE] -"\u2079" => "9" - -# ₉ [SUBSCRIPT NINE] -"\u2089" => "9" - -# ⑨ [CIRCLED DIGIT NINE] -"\u2468" => "9" - -# ⓽ [DOUBLE CIRCLED DIGIT NINE] -"\u24FD" => "9" - -# ❾ [DINGBAT NEGATIVE CIRCLED DIGIT NINE] -"\u277E" => "9" - -# ➈ [DINGBAT CIRCLED SANS-SERIF DIGIT NINE] -"\u2788" => "9" - -# ➒ [DINGBAT NEGATIVE CIRCLED SANS-SERIF DIGIT NINE] -"\u2792" => "9" - -# 9 [FULLWIDTH DIGIT NINE] -"\uFF19" => "9" - -# ⒐ [DIGIT NINE FULL STOP] -"\u2490" => "9." - -# ⑼ [PARENTHESIZED DIGIT NINE] -"\u247C" => "(9)" - -# ⑩ [CIRCLED NUMBER TEN] -"\u2469" => "10" - -# ⓾ [DOUBLE CIRCLED NUMBER TEN] -"\u24FE" => "10" - -# ❿ [DINGBAT NEGATIVE CIRCLED NUMBER TEN] -"\u277F" => "10" - -# ➉ [DINGBAT CIRCLED SANS-SERIF NUMBER TEN] -"\u2789" => "10" - -# ➓ [DINGBAT NEGATIVE CIRCLED SANS-SERIF NUMBER TEN] -"\u2793" => "10" - -# ⒑ [NUMBER TEN FULL STOP] -"\u2491" => "10." - -# ⑽ [PARENTHESIZED NUMBER TEN] -"\u247D" => "(10)" - -# ⑪ [CIRCLED NUMBER ELEVEN] -"\u246A" => "11" - -# ⓫ [NEGATIVE CIRCLED NUMBER ELEVEN] -"\u24EB" => "11" - -# ⒒ [NUMBER ELEVEN FULL STOP] -"\u2492" => "11." - -# ⑾ [PARENTHESIZED NUMBER ELEVEN] -"\u247E" => "(11)" - -# ⑫ [CIRCLED NUMBER TWELVE] -"\u246B" => "12" - -# ⓬ [NEGATIVE CIRCLED NUMBER TWELVE] -"\u24EC" => "12" - -# ⒓ [NUMBER TWELVE FULL STOP] -"\u2493" => "12." - -# ⑿ [PARENTHESIZED NUMBER TWELVE] -"\u247F" => "(12)" - -# ⑬ [CIRCLED NUMBER THIRTEEN] -"\u246C" => "13" - -# ⓭ [NEGATIVE CIRCLED NUMBER THIRTEEN] -"\u24ED" => "13" - -# ⒔ [NUMBER THIRTEEN FULL STOP] -"\u2494" => "13." - -# ⒀ [PARENTHESIZED NUMBER THIRTEEN] -"\u2480" => "(13)" - -# ⑭ [CIRCLED NUMBER FOURTEEN] -"\u246D" => "14" - -# ⓮ [NEGATIVE CIRCLED NUMBER FOURTEEN] -"\u24EE" => "14" - -# ⒕ [NUMBER FOURTEEN FULL STOP] -"\u2495" => "14." - -# ⒁ [PARENTHESIZED NUMBER FOURTEEN] -"\u2481" => "(14)" - -# ⑮ [CIRCLED NUMBER FIFTEEN] -"\u246E" => "15" - -# ⓯ [NEGATIVE CIRCLED NUMBER FIFTEEN] -"\u24EF" => "15" - -# ⒖ [NUMBER FIFTEEN FULL STOP] -"\u2496" => "15." - -# ⒂ [PARENTHESIZED NUMBER FIFTEEN] -"\u2482" => "(15)" - -# ⑯ [CIRCLED NUMBER SIXTEEN] -"\u246F" => "16" - -# ⓰ [NEGATIVE CIRCLED NUMBER SIXTEEN] -"\u24F0" => "16" - -# ⒗ [NUMBER SIXTEEN FULL STOP] -"\u2497" => "16." - -# ⒃ [PARENTHESIZED NUMBER SIXTEEN] -"\u2483" => "(16)" - -# ⑰ [CIRCLED NUMBER SEVENTEEN] -"\u2470" => "17" - -# ⓱ [NEGATIVE CIRCLED NUMBER SEVENTEEN] -"\u24F1" => "17" - -# ⒘ [NUMBER SEVENTEEN FULL STOP] -"\u2498" => "17." - -# ⒄ [PARENTHESIZED NUMBER SEVENTEEN] -"\u2484" => "(17)" - -# ⑱ [CIRCLED NUMBER EIGHTEEN] -"\u2471" => "18" - -# ⓲ [NEGATIVE CIRCLED NUMBER EIGHTEEN] -"\u24F2" => "18" - -# ⒙ [NUMBER EIGHTEEN FULL STOP] -"\u2499" => "18." - -# ⒅ [PARENTHESIZED NUMBER EIGHTEEN] -"\u2485" => "(18)" - -# ⑲ [CIRCLED NUMBER NINETEEN] -"\u2472" => "19" - -# ⓳ [NEGATIVE CIRCLED NUMBER NINETEEN] -"\u24F3" => "19" - -# ⒚ [NUMBER NINETEEN FULL STOP] -"\u249A" => "19." - -# ⒆ [PARENTHESIZED NUMBER NINETEEN] -"\u2486" => "(19)" - -# ⑳ [CIRCLED NUMBER TWENTY] -"\u2473" => "20" - -# ⓴ [NEGATIVE CIRCLED NUMBER TWENTY] -"\u24F4" => "20" - -# ⒛ [NUMBER TWENTY FULL STOP] -"\u249B" => "20." - -# ⒇ [PARENTHESIZED NUMBER TWENTY] -"\u2487" => "(20)" - -# « [LEFT-POINTING DOUBLE ANGLE QUOTATION MARK] -"\u00AB" => "\"" - -# » [RIGHT-POINTING DOUBLE ANGLE QUOTATION MARK] -"\u00BB" => "\"" - -# “ [LEFT DOUBLE QUOTATION MARK] -"\u201C" => "\"" - -# ” [RIGHT DOUBLE QUOTATION MARK] -"\u201D" => "\"" - -# „ [DOUBLE LOW-9 QUOTATION MARK] -"\u201E" => "\"" - -# ″ [DOUBLE PRIME] -"\u2033" => "\"" - -# ‶ [REVERSED DOUBLE PRIME] -"\u2036" => "\"" - -# ❝ [HEAVY DOUBLE TURNED COMMA QUOTATION MARK ORNAMENT] -"\u275D" => "\"" - -# ❞ [HEAVY DOUBLE COMMA QUOTATION MARK ORNAMENT] -"\u275E" => "\"" - -# ❮ [HEAVY LEFT-POINTING ANGLE QUOTATION MARK ORNAMENT] -"\u276E" => "\"" - -# ❯ [HEAVY RIGHT-POINTING ANGLE QUOTATION MARK ORNAMENT] -"\u276F" => "\"" - -# " [FULLWIDTH QUOTATION MARK] -"\uFF02" => "\"" - -# ‘ [LEFT SINGLE QUOTATION MARK] -"\u2018" => "\'" - -# ’ [RIGHT SINGLE QUOTATION MARK] -"\u2019" => "\'" - -# ‚ [SINGLE LOW-9 QUOTATION MARK] -"\u201A" => "\'" - -# ‛ [SINGLE HIGH-REVERSED-9 QUOTATION MARK] -"\u201B" => "\'" - -# ′ [PRIME] -"\u2032" => "\'" - -# ‵ [REVERSED PRIME] -"\u2035" => "\'" - -# ‹ [SINGLE LEFT-POINTING ANGLE QUOTATION MARK] -"\u2039" => "\'" - -# › [SINGLE RIGHT-POINTING ANGLE QUOTATION MARK] -"\u203A" => "\'" - -# ❛ [HEAVY SINGLE TURNED COMMA QUOTATION MARK ORNAMENT] -"\u275B" => "\'" - -# ❜ [HEAVY SINGLE COMMA QUOTATION MARK ORNAMENT] -"\u275C" => "\'" - -# ' [FULLWIDTH APOSTROPHE] -"\uFF07" => "\'" - -# ‐ [HYPHEN] -"\u2010" => "-" - -# ‑ [NON-BREAKING HYPHEN] -"\u2011" => "-" - -# ‒ [FIGURE DASH] -"\u2012" => "-" - -# – [EN DASH] -"\u2013" => "-" - -# — [EM DASH] -"\u2014" => "-" - -# ⁻ [SUPERSCRIPT MINUS] -"\u207B" => "-" - -# ₋ [SUBSCRIPT MINUS] -"\u208B" => "-" - -# - [FULLWIDTH HYPHEN-MINUS] -"\uFF0D" => "-" - -# ⁅ [LEFT SQUARE BRACKET WITH QUILL] -"\u2045" => "[" - -# ❲ [LIGHT LEFT TORTOISE SHELL BRACKET ORNAMENT] -"\u2772" => "[" - -# [ [FULLWIDTH LEFT SQUARE BRACKET] -"\uFF3B" => "[" - -# ⁆ [RIGHT SQUARE BRACKET WITH QUILL] -"\u2046" => "]" - -# ❳ [LIGHT RIGHT TORTOISE SHELL BRACKET ORNAMENT] -"\u2773" => "]" - -# ] [FULLWIDTH RIGHT SQUARE BRACKET] -"\uFF3D" => "]" - -# ⁽ [SUPERSCRIPT LEFT PARENTHESIS] -"\u207D" => "(" - -# ₍ [SUBSCRIPT LEFT PARENTHESIS] -"\u208D" => "(" - -# ❨ [MEDIUM LEFT PARENTHESIS ORNAMENT] -"\u2768" => "(" - -# ❪ [MEDIUM FLATTENED LEFT PARENTHESIS ORNAMENT] -"\u276A" => "(" - -# ( [FULLWIDTH LEFT PARENTHESIS] -"\uFF08" => "(" - -# ⸨ [LEFT DOUBLE PARENTHESIS] -"\u2E28" => "((" - -# ⁾ [SUPERSCRIPT RIGHT PARENTHESIS] -"\u207E" => ")" - -# ₎ [SUBSCRIPT RIGHT PARENTHESIS] -"\u208E" => ")" - -# ❩ [MEDIUM RIGHT PARENTHESIS ORNAMENT] -"\u2769" => ")" - -# ❫ [MEDIUM FLATTENED RIGHT PARENTHESIS ORNAMENT] -"\u276B" => ")" - -# ) [FULLWIDTH RIGHT PARENTHESIS] -"\uFF09" => ")" - -# ⸩ [RIGHT DOUBLE PARENTHESIS] -"\u2E29" => "))" - -# ❬ [MEDIUM LEFT-POINTING ANGLE BRACKET ORNAMENT] -"\u276C" => "<" - -# ❰ [HEAVY LEFT-POINTING ANGLE BRACKET ORNAMENT] -"\u2770" => "<" - -# < [FULLWIDTH LESS-THAN SIGN] -"\uFF1C" => "<" - -# ❭ [MEDIUM RIGHT-POINTING ANGLE BRACKET ORNAMENT] -"\u276D" => ">" - -# ❱ [HEAVY RIGHT-POINTING ANGLE BRACKET ORNAMENT] -"\u2771" => ">" - -# > [FULLWIDTH GREATER-THAN SIGN] -"\uFF1E" => ">" - -# ❴ [MEDIUM LEFT CURLY BRACKET ORNAMENT] -"\u2774" => "{" - -# { [FULLWIDTH LEFT CURLY BRACKET] -"\uFF5B" => "{" - -# ❵ [MEDIUM RIGHT CURLY BRACKET ORNAMENT] -"\u2775" => "}" - -# } [FULLWIDTH RIGHT CURLY BRACKET] -"\uFF5D" => "}" - -# ⁺ [SUPERSCRIPT PLUS SIGN] -"\u207A" => "+" - -# ₊ [SUBSCRIPT PLUS SIGN] -"\u208A" => "+" - -# + [FULLWIDTH PLUS SIGN] -"\uFF0B" => "+" - -# ⁼ [SUPERSCRIPT EQUALS SIGN] -"\u207C" => "=" - -# ₌ [SUBSCRIPT EQUALS SIGN] -"\u208C" => "=" - -# = [FULLWIDTH EQUALS SIGN] -"\uFF1D" => "=" - -# ! [FULLWIDTH EXCLAMATION MARK] -"\uFF01" => "!" - -# ‼ [DOUBLE EXCLAMATION MARK] -"\u203C" => "!!" - -# ⁉ [EXCLAMATION QUESTION MARK] -"\u2049" => "!?" - -# # [FULLWIDTH NUMBER SIGN] -"\uFF03" => "#" - -# $ [FULLWIDTH DOLLAR SIGN] -"\uFF04" => "$" - -# ⁒ [COMMERCIAL MINUS SIGN] -"\u2052" => "%" - -# % [FULLWIDTH PERCENT SIGN] -"\uFF05" => "%" - -# & [FULLWIDTH AMPERSAND] -"\uFF06" => "&" - -# ⁎ [LOW ASTERISK] -"\u204E" => "*" - -# * [FULLWIDTH ASTERISK] -"\uFF0A" => "*" - -# , [FULLWIDTH COMMA] -"\uFF0C" => "," - -# . [FULLWIDTH FULL STOP] -"\uFF0E" => "." - -# ⁄ [FRACTION SLASH] -"\u2044" => "/" - -# / [FULLWIDTH SOLIDUS] -"\uFF0F" => "/" - -# : [FULLWIDTH COLON] -"\uFF1A" => ":" - -# ⁏ [REVERSED SEMICOLON] -"\u204F" => ";" - -# ; [FULLWIDTH SEMICOLON] -"\uFF1B" => ";" - -# ? [FULLWIDTH QUESTION MARK] -"\uFF1F" => "?" - -# ⁇ [DOUBLE QUESTION MARK] -"\u2047" => "??" - -# ⁈ [QUESTION EXCLAMATION MARK] -"\u2048" => "?!" - -# @ [FULLWIDTH COMMERCIAL AT] -"\uFF20" => "@" - -# \ [FULLWIDTH REVERSE SOLIDUS] -"\uFF3C" => "\\" - -# ‸ [CARET] -"\u2038" => "^" - -# ^ [FULLWIDTH CIRCUMFLEX ACCENT] -"\uFF3E" => "^" - -# _ [FULLWIDTH LOW LINE] -"\uFF3F" => "_" - -# ⁓ [SWUNG DASH] -"\u2053" => "~" - -# ~ [FULLWIDTH TILDE] -"\uFF5E" => "~" - -################################################################ -# Below is the Perl script used to generate the above mappings # -# from ASCIIFoldingFilter.java: # -################################################################ -# -# #!/usr/bin/perl -# -# use warnings; -# use strict; -# -# my @source_chars = (); -# my @source_char_descriptions = (); -# my $target = ''; -# -# while (<>) { -# if (/case\s+'(\\u[A-F0-9]+)':\s*\/\/\s*(.*)/i) { -# push @source_chars, $1; -# push @source_char_descriptions, $2; -# next; -# } -# if (/output\[[^\]]+\]\s*=\s*'(\\'|\\\\|.)'/) { -# $target .= $1; -# next; -# } -# if (/break;/) { -# $target = "\\\"" if ($target eq '"'); -# for my $source_char_num (0..$#source_chars) { -# print "# $source_char_descriptions[$source_char_num]\n"; -# print "\"$source_chars[$source_char_num]\" => \"$target\"\n\n"; -# } -# @source_chars = (); -# @source_char_descriptions = (); -# $target = ''; -# } -# } diff --git a/solr/example/example-DIH/solr/rss/conf/mapping-ISOLatin1Accent.txt b/solr/example/example-DIH/solr/rss/conf/mapping-ISOLatin1Accent.txt deleted file mode 100644 index ede7742581b..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/mapping-ISOLatin1Accent.txt +++ /dev/null @@ -1,246 +0,0 @@ -# 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. - -# Syntax: -# "source" => "target" -# "source".length() > 0 (source cannot be empty.) -# "target".length() >= 0 (target can be empty.) - -# example: -# "À" => "A" -# "\u00C0" => "A" -# "\u00C0" => "\u0041" -# "ß" => "ss" -# "\t" => " " -# "\n" => "" - -# À => A -"\u00C0" => "A" - -# Á => A -"\u00C1" => "A" - -#  => A -"\u00C2" => "A" - -# à => A -"\u00C3" => "A" - -# Ä => A -"\u00C4" => "A" - -# Å => A -"\u00C5" => "A" - -# Æ => AE -"\u00C6" => "AE" - -# Ç => C -"\u00C7" => "C" - -# È => E -"\u00C8" => "E" - -# É => E -"\u00C9" => "E" - -# Ê => E -"\u00CA" => "E" - -# Ë => E -"\u00CB" => "E" - -# Ì => I -"\u00CC" => "I" - -# Í => I -"\u00CD" => "I" - -# Î => I -"\u00CE" => "I" - -# Ï => I -"\u00CF" => "I" - -# IJ => IJ -"\u0132" => "IJ" - -# Ð => D -"\u00D0" => "D" - -# Ñ => N -"\u00D1" => "N" - -# Ò => O -"\u00D2" => "O" - -# Ó => O -"\u00D3" => "O" - -# Ô => O -"\u00D4" => "O" - -# Õ => O -"\u00D5" => "O" - -# Ö => O -"\u00D6" => "O" - -# Ø => O -"\u00D8" => "O" - -# Œ => OE -"\u0152" => "OE" - -# Þ -"\u00DE" => "TH" - -# Ù => U -"\u00D9" => "U" - -# Ú => U -"\u00DA" => "U" - -# Û => U -"\u00DB" => "U" - -# Ü => U -"\u00DC" => "U" - -# Ý => Y -"\u00DD" => "Y" - -# Ÿ => Y -"\u0178" => "Y" - -# à => a -"\u00E0" => "a" - -# á => a -"\u00E1" => "a" - -# â => a -"\u00E2" => "a" - -# ã => a -"\u00E3" => "a" - -# ä => a -"\u00E4" => "a" - -# å => a -"\u00E5" => "a" - -# æ => ae -"\u00E6" => "ae" - -# ç => c -"\u00E7" => "c" - -# è => e -"\u00E8" => "e" - -# é => e -"\u00E9" => "e" - -# ê => e -"\u00EA" => "e" - -# ë => e -"\u00EB" => "e" - -# ì => i -"\u00EC" => "i" - -# í => i -"\u00ED" => "i" - -# î => i -"\u00EE" => "i" - -# ï => i -"\u00EF" => "i" - -# ij => ij -"\u0133" => "ij" - -# ð => d -"\u00F0" => "d" - -# ñ => n -"\u00F1" => "n" - -# ò => o -"\u00F2" => "o" - -# ó => o -"\u00F3" => "o" - -# ô => o -"\u00F4" => "o" - -# õ => o -"\u00F5" => "o" - -# ö => o -"\u00F6" => "o" - -# ø => o -"\u00F8" => "o" - -# œ => oe -"\u0153" => "oe" - -# ß => ss -"\u00DF" => "ss" - -# þ => th -"\u00FE" => "th" - -# ù => u -"\u00F9" => "u" - -# ú => u -"\u00FA" => "u" - -# û => u -"\u00FB" => "u" - -# ü => u -"\u00FC" => "u" - -# ý => y -"\u00FD" => "y" - -# ÿ => y -"\u00FF" => "y" - -# ff => ff -"\uFB00" => "ff" - -# fi => fi -"\uFB01" => "fi" - -# fl => fl -"\uFB02" => "fl" - -# ffi => ffi -"\uFB03" => "ffi" - -# ffl => ffl -"\uFB04" => "ffl" - -# ſt => ft -"\uFB05" => "ft" - -# st => st -"\uFB06" => "st" diff --git a/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml b/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml deleted file mode 100644 index 704325babcb..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/rss-data-config.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/solrconfig.xml b/solr/example/example-DIH/solr/rss/conf/solrconfig.xml deleted file mode 100644 index eae60a79956..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/solrconfig.xml +++ /dev/null @@ -1,1396 +0,0 @@ - - - - - - - - - 7.0.0 - - - - - - - - - - - - - - - - - - - - ${solr.data.dir:} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ${solr.lock.type:native} - - - - - - - - - - - - - true - - - - - - - - - - - - - - - - ${solr.ulog.dir:} - - - - - ${solr.autoCommit.maxTime:15000} - false - - - - - - ${solr.autoSoftCommit.maxTime:-1} - - - - - - - - - - - - - - - - 1024 - - - - - - - - - - - - - - - - - - - - - - - - - true - - - - - - 20 - - - 200 - - - - - - - - - - - - static firstSearcher warming in solrconfig.xml - - - - - - false - - - - - - - - - - - - - - - - - - - - - - rss-data-config.xml - - - - - - - - explicit - 10 - text - - - - - - - - - - - - - - explicit - json - true - text - - - - - - - explicit - - - velocity - browse - layout - - - edismax - *:* - 10 - *,score - - - on - 1 - - - - - - text - - - - - - - true - ignored_ - - - true - links - ignored_ - - - - - - - - - text_general - - - - - - default - text - solr.DirectSolrSpellChecker - - internal - - 0.5 - - 2 - - 1 - - 5 - - 4 - - 0.01 - - - - - - wordbreak - solr.WordBreakSolrSpellChecker - name - true - true - 10 - - - - - - - - - - - - - - - - text - - default - wordbreak - on - true - 10 - 5 - 5 - true - true - 10 - 5 - - - spellcheck - - - - - - mySuggester - FuzzyLookupFactory - DocumentDictionaryFactory - cat - price - string - - - - - - true - 10 - - - suggest - - - - - - - - - text - true - - - tvComponent - - - - - - - - - - true - false - - - terms - - - - - - - - string - elevate.xml - - - - - - explicit - text - - - elevator - - - - - - - - - - - 100 - - - - - - - - 70 - - 0.5 - - [-\w ,/\n\"']{20,200} - - - - - - - ]]> - ]]> - - - - - - - - - - - - - - - - - - - - - - - - ,, - ,, - ,, - ,, - ,]]> - ]]> - - - - - - 10 - .,!? - - - - - - - WORD - - - en - US - - - - - - - - - - - - - - - - - - - - - - text/plain; charset=UTF-8 - - - - - ${velocity.template.base.dir:} - - - - - 5 - - - - - - - - - - - - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/spellings.txt b/solr/example/example-DIH/solr/rss/conf/spellings.txt deleted file mode 100644 index d7ede6f5611..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/spellings.txt +++ /dev/null @@ -1,2 +0,0 @@ -pizza -history \ No newline at end of file diff --git a/solr/example/example-DIH/solr/rss/conf/stopwords.txt b/solr/example/example-DIH/solr/rss/conf/stopwords.txt deleted file mode 100644 index ae1e83eeb3d..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/stopwords.txt +++ /dev/null @@ -1,14 +0,0 @@ -# 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. diff --git a/solr/example/example-DIH/solr/rss/conf/update-script.js b/solr/example/example-DIH/solr/rss/conf/update-script.js deleted file mode 100644 index 49b07f9b71e..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/update-script.js +++ /dev/null @@ -1,53 +0,0 @@ -/* - This is a basic skeleton JavaScript update processor. - - In order for this to be executed, it must be properly wired into solrconfig.xml; by default it is commented out in - the example solrconfig.xml and must be uncommented to be enabled. - - See http://wiki.apache.org/solr/ScriptUpdateProcessor for more details. -*/ - -function processAdd(cmd) { - - doc = cmd.solrDoc; // org.apache.solr.common.SolrInputDocument - id = doc.getFieldValue("id"); - logger.info("update-script#processAdd: id=" + id); - -// Set a field value: -// doc.setField("foo_s", "whatever"); - -// Get a configuration parameter: -// config_param = params.get('config_param'); // "params" only exists if processor configured with - -// Get a request parameter: -// some_param = req.getParams().get("some_param") - -// Add a field of field names that match a pattern: -// - Potentially useful to determine the fields/attributes represented in a result set, via faceting on field_name_ss -// field_names = doc.getFieldNames().toArray(); -// for(i=0; i < field_names.length; i++) { -// field_name = field_names[i]; -// if (/attr_.*/.test(field_name)) { doc.addField("attribute_ss", field_names[i]); } -// } - -} - -function processDelete(cmd) { - // no-op -} - -function processMergeIndexes(cmd) { - // no-op -} - -function processCommit(cmd) { - // no-op -} - -function processRollback(cmd) { - // no-op -} - -function finish() { - // no-op -} diff --git a/solr/example/example-DIH/solr/rss/conf/xslt/example.xsl b/solr/example/example-DIH/solr/rss/conf/xslt/example.xsl deleted file mode 100644 index b8992700828..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/xslt/example.xsl +++ /dev/null @@ -1,132 +0,0 @@ - - - - - - - - - - - - - - - <xsl:value-of select="$title"/> - - - -

    -
    - This has been formatted by the sample "example.xsl" transform - - use your own XSLT to get a nicer page -
    - - - -
    - - - -
    - - - - -
    -
    -
    - - - - - - - - - - - - - - javascript:toggle("");? -
    - - exp - - - - - -
    - - -
    - - - - - - - -
      - -
    • -
      -
    - - -
    - - - - - - - - - - - - - - - - - - - - -
    diff --git a/solr/example/example-DIH/solr/rss/conf/xslt/example_atom.xsl b/solr/example/example-DIH/solr/rss/conf/xslt/example_atom.xsl deleted file mode 100644 index b6c23151dc4..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/xslt/example_atom.xsl +++ /dev/null @@ -1,67 +0,0 @@ - - - - - - - - - - - - - - Example Solr Atom 1.0 Feed - - This has been formatted by the sample "example_atom.xsl" transform - - use your own XSLT to get a nicer Atom feed. - - - Apache Solr - solr-user@lucene.apache.org - - - - - - tag:localhost,2007:example - - - - - - - - - <xsl:value-of select="str[@name='name']"/> - - tag:localhost,2007: - - - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/xslt/example_rss.xsl b/solr/example/example-DIH/solr/rss/conf/xslt/example_rss.xsl deleted file mode 100644 index c8ab5bfb1ec..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/xslt/example_rss.xsl +++ /dev/null @@ -1,66 +0,0 @@ - - - - - - - - - - - - - Example Solr RSS 2.0 Feed - http://localhost:8983/solr - - This has been formatted by the sample "example_rss.xsl" transform - - use your own XSLT to get a nicer RSS feed. - - en-us - http://localhost:8983/solr - - - - - - - - - - - <xsl:value-of select="str[@name='name']"/> - - http://localhost:8983/solr/select?q=id: - - - - - - - http://localhost:8983/solr/select?q=id: - - - - diff --git a/solr/example/example-DIH/solr/rss/conf/xslt/luke.xsl b/solr/example/example-DIH/solr/rss/conf/xslt/luke.xsl deleted file mode 100644 index 05fb5bfeee2..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/xslt/luke.xsl +++ /dev/null @@ -1,337 +0,0 @@ - - - - - - - - - Solr Luke Request Handler Response - - - - - - - - - <xsl:value-of select="$title"/> - - - - - -

    - -

    -
    - -
    - -

    Index Statistics

    - -
    - -

    Field Statistics

    - - - -

    Document statistics

    - - - - -
    - - - - - -
    - -
    - - -
    - -
    - -
    -
    -
    - - - - - - - - - - - - - - - - - - - - - -
    -

    - -

    - -
    - -
    -
    -
    - - -
    - - 50 - 800 - 160 - blue - -
    -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - -
    - background-color: ; width: px; height: px; -
    -
    - -
    -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
      - -
    • - -
    • -
      -
    - - -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - 1 - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    diff --git a/solr/example/example-DIH/solr/rss/conf/xslt/updateXml.xsl b/solr/example/example-DIH/solr/rss/conf/xslt/updateXml.xsl deleted file mode 100644 index a96e1d02448..00000000000 --- a/solr/example/example-DIH/solr/rss/conf/xslt/updateXml.xsl +++ /dev/null @@ -1,70 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - From b02626de5071c543eb6e8deea450266218238c9e Mon Sep 17 00:00:00 2001 From: Alexandre Rafalovitch Date: Sat, 1 Apr 2017 19:06:50 -0400 Subject: [PATCH 101/563] SOLR-9601: DIH Tika example is now minimal Only keep definitions and files required to show Tika-extraction in DIH --- solr/CHANGES.txt | 2 + .../solr/tika/conf/admin-extra.html | 24 - .../tika/conf/admin-extra.menu-bottom.html | 25 - .../solr/tika/conf/admin-extra.menu-top.html | 25 - .../example-DIH/solr/tika/conf/managed-schema | 916 +---------- .../example-DIH/solr/tika/conf/solrconfig.xml | 1364 +---------------- .../solr/tika/conf/tika-data-config.xml | 33 +- 7 files changed, 80 insertions(+), 2309 deletions(-) delete mode 100644 solr/example/example-DIH/solr/tika/conf/admin-extra.html delete mode 100644 solr/example/example-DIH/solr/tika/conf/admin-extra.menu-bottom.html delete mode 100644 solr/example/example-DIH/solr/tika/conf/admin-extra.menu-top.html diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 3187dc3c7bc..99edab464c5 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -182,6 +182,8 @@ Other Changes * SOLR-7383: New DataImportHandler 'atom' example, replacing broken 'rss' example (Alexandre Rafalovitch) +* SOLR-9601: Redone DataImportHandler 'tika' example, removing all unused and irrelevant definitions (Alexandre Rafalovitch) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/example/example-DIH/solr/tika/conf/admin-extra.html b/solr/example/example-DIH/solr/tika/conf/admin-extra.html deleted file mode 100644 index fecab20513d..00000000000 --- a/solr/example/example-DIH/solr/tika/conf/admin-extra.html +++ /dev/null @@ -1,24 +0,0 @@ - - - diff --git a/solr/example/example-DIH/solr/tika/conf/admin-extra.menu-bottom.html b/solr/example/example-DIH/solr/tika/conf/admin-extra.menu-bottom.html deleted file mode 100644 index 3359a460a48..00000000000 --- a/solr/example/example-DIH/solr/tika/conf/admin-extra.menu-bottom.html +++ /dev/null @@ -1,25 +0,0 @@ - - - - diff --git a/solr/example/example-DIH/solr/tika/conf/admin-extra.menu-top.html b/solr/example/example-DIH/solr/tika/conf/admin-extra.menu-top.html deleted file mode 100644 index 0886cee37a4..00000000000 --- a/solr/example/example-DIH/solr/tika/conf/admin-extra.menu-top.html +++ /dev/null @@ -1,25 +0,0 @@ - - - - diff --git a/solr/example/example-DIH/solr/tika/conf/managed-schema b/solr/example/example-DIH/solr/tika/conf/managed-schema index 58b2a80b6e1..6d506f7ec64 100644 --- a/solr/example/example-DIH/solr/tika/conf/managed-schema +++ b/solr/example/example-DIH/solr/tika/conf/managed-schema @@ -16,897 +16,39 @@ limitations under the License. --> - - - + id + + + + + + + + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + - - - + + + + + + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + \ No newline at end of file diff --git a/solr/example/example-DIH/solr/tika/conf/solrconfig.xml b/solr/example/example-DIH/solr/tika/conf/solrconfig.xml index ac0c22a6d3a..38d5d8b81e0 100644 --- a/solr/example/example-DIH/solr/tika/conf/solrconfig.xml +++ b/solr/example/example-DIH/solr/tika/conf/solrconfig.xml @@ -17,705 +17,37 @@ --> + - - 7.0.0 - - - - - - - - - - - - - - - - - - - - ${solr.data.dir:} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ${solr.lock.type:native} - - - - - - - - - - - - - true - - - - - - - - - - - - - - - - - - - ${solr.autoCommit.maxTime:15000} - false - - - - - - ${solr.autoSoftCommit.maxTime:-1} - - - - - - - - - - - - - - - - 1024 - - - - - - - - - - - - - - - - - - - - - - - - - true - - - - - - 20 - - - 200 - - - - - - - - - - - - static firstSearcher warming in solrconfig.xml - - - - - - false - - - - - - - - - - - - - - - - - - + + + + + + + explicit + text + + @@ -723,650 +55,4 @@ - - - - - explicit - 10 - text - - - - - - - - - - - - - - explicit - json - true - text - - - - - - - explicit - - - velocity - browse - layout - - - edismax - *:* - 10 - *,score - - - on - 1 - - - - - - text - - - - - - - true - ignored_ - - - true - links - ignored_ - - - - - - - - text_general - - - - - - default - text - solr.DirectSolrSpellChecker - - internal - - 0.5 - - 2 - - 1 - - 5 - - 4 - - 0.01 - - - - - - wordbreak - solr.WordBreakSolrSpellChecker - name - true - true - 10 - - - - - - - - - - - - - - - - text - - default - wordbreak - on - true - 10 - 5 - 5 - true - true - 10 - 5 - - - spellcheck - - - - - - mySuggester - FuzzyLookupFactory - DocumentDictionaryFactory - cat - price - string - - - - - - true - 10 - - - suggest - - - - - - - - - text - true - - - tvComponent - - - - - - - - - - true - false - - - terms - - - - - - - - - - - - 100 - - - - - - - - 70 - - 0.5 - - [-\w ,/\n\"']{20,200} - - - - - - - ]]> - ]]> - - - - - - - - - - - - - - - - - - - - - - - - ,, - ,, - ,, - ,, - ,]]> - ]]> - - - - - - 10 - .,!? - - - - - - - WORD - - - en - US - - - - - - - - - - - - - - - - - - - - - - text/plain; charset=UTF-8 - - - - - ${velocity.template.base.dir:} - - - - - 5 - - - - - - - - - - - - - diff --git a/solr/example/example-DIH/solr/tika/conf/tika-data-config.xml b/solr/example/example-DIH/solr/tika/conf/tika-data-config.xml index 0d9e7685c44..5286fc418f7 100644 --- a/solr/example/example-DIH/solr/tika/conf/tika-data-config.xml +++ b/solr/example/example-DIH/solr/tika/conf/tika-data-config.xml @@ -1,11 +1,26 @@ - - - - - - - - + + + + + + + + + + + + + + + + + + + From 674ce4e89393efe3147629e76f053c9901c182dc Mon Sep 17 00:00:00 2001 From: Dennis Gove Date: Thu, 23 Mar 2017 20:08:11 -0400 Subject: [PATCH 102/563] SOLR-10356: Adds basic math streaming evaluators --- solr/CHANGES.txt | 2 + .../apache/solr/handler/StreamHandler.java | 79 ++++++++- .../solrj/io/eval/ArcCosineEvaluator.java | 60 +++++++ .../solrj/io/eval/ArcSineEvaluator.java | 60 +++++++ .../solrj/io/eval/ArcTangentEvaluator.java | 60 +++++++ .../solrj/io/eval/BooleanEvaluator.java | 7 - .../solrj/io/eval/CeilingEvaluator.java | 61 +++++++ .../solrj/io/eval/CoalesceEvaluator.java | 52 ++++++ .../solrj/io/eval/ComplexEvaluator.java | 6 + .../solrj/io/eval/ConditionalEvaluator.java | 6 - .../client/solrj/io/eval/CosineEvaluator.java | 60 +++++++ .../solrj/io/eval/CubedRootEvaluator.java | 60 +++++++ .../client/solrj/io/eval/FloorEvaluator.java | 61 +++++++ .../io/eval/HyperbolicCosineEvaluator.java | 60 +++++++ .../io/eval/HyperbolicSineEvaluator.java | 60 +++++++ .../io/eval/HyperbolicTangentEvaluator.java | 60 +++++++ .../client/solrj/io/eval/ModuloEvaluator.java | 78 +++++++++ .../client/solrj/io/eval/NumberEvaluator.java | 6 - .../client/solrj/io/eval/PowerEvaluator.java | 61 +++++++ .../client/solrj/io/eval/RoundEvaluator.java | 60 +++++++ .../client/solrj/io/eval/SineEvaluator.java | 60 +++++++ .../solrj/io/eval/SquareRootEvaluator.java | 60 +++++++ .../solrj/io/eval/TangentEvaluator.java | 60 +++++++ .../stream/eval/ArcCosineEvaluatorTest.java | 91 ++++++++++ .../io/stream/eval/ArcSineEvaluatorTest.java | 91 ++++++++++ .../stream/eval/ArcTangentEvaluatorTest.java | 91 ++++++++++ .../io/stream/eval/CeilingEvaluatorTest.java | 96 ++++++++++ .../io/stream/eval/CoalesceEvaluatorTest.java | 112 ++++++++++++ .../io/stream/eval/CosineEvaluatorTest.java | 91 ++++++++++ .../stream/eval/CubedRootEvaluatorTest.java | 91 ++++++++++ .../io/stream/eval/FloorEvaluatorTest.java | 96 ++++++++++ .../eval/HyperbolicCosineEvaluatorTest.java | 91 ++++++++++ .../eval/HyperbolicSineEvaluatorTest.java | 91 ++++++++++ .../eval/HyperbolicTangentEvaluatorTest.java | 91 ++++++++++ .../io/stream/eval/ModuloEvaluatorTest.java | 164 ++++++++++++++++++ .../io/stream/eval/PowerEvaluatorTest.java | 119 +++++++++++++ .../io/stream/eval/RoundEvaluatorTest.java | 95 ++++++++++ .../io/stream/eval/SineEvaluatorTest.java | 91 ++++++++++ .../stream/eval/SquareRootEvaluatorTest.java | 91 ++++++++++ .../io/stream/eval/TangentEvaluatorTest.java | 91 ++++++++++ 40 files changed, 2799 insertions(+), 23 deletions(-) create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcCosineEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcSineEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcTangentEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CeilingEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CoalesceEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CosineEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CubedRootEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FloorEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicCosineEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicSineEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicTangentEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ModuloEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PowerEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RoundEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SineEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SquareRootEvaluator.java create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TangentEvaluator.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcCosineEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcSineEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcTangentEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CeilingEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CoalesceEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CosineEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CubedRootEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/FloorEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicCosineEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicSineEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicTangentEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ModuloEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/PowerEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/RoundEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SineEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SquareRootEvaluatorTest.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TangentEvaluatorTest.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 99edab464c5..1c3aaf725d4 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -61,6 +61,8 @@ New Features * SOLR-10292: Adds CartesianProductStream which turns a single tuple with a multi-valued field into N tuples, one for each value in the multi-valued field. (Dennis Gove) +* SOLR-10356: Adds basic math Streaming Evaluators (Dennis Gove) + Bug Fixes ---------------------- * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509. diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java index 3ede7320f89..b508754b20a 100644 --- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java @@ -16,6 +16,9 @@ */ package org.apache.solr.handler; +import static org.apache.solr.common.params.CommonParams.ID; +import static org.apache.solr.common.params.CommonParams.SORT; + import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.ArrayList; @@ -33,27 +36,80 @@ import org.apache.solr.client.solrj.io.comp.StreamComparator; import org.apache.solr.client.solrj.io.eval.AbsoluteValueEvaluator; import org.apache.solr.client.solrj.io.eval.AddEvaluator; import org.apache.solr.client.solrj.io.eval.AndEvaluator; +import org.apache.solr.client.solrj.io.eval.ArcCosineEvaluator; +import org.apache.solr.client.solrj.io.eval.ArcSineEvaluator; +import org.apache.solr.client.solrj.io.eval.ArcTangentEvaluator; +import org.apache.solr.client.solrj.io.eval.CeilingEvaluator; +import org.apache.solr.client.solrj.io.eval.CoalesceEvaluator; +import org.apache.solr.client.solrj.io.eval.CosineEvaluator; +import org.apache.solr.client.solrj.io.eval.CubedRootEvaluator; import org.apache.solr.client.solrj.io.eval.DivideEvaluator; import org.apache.solr.client.solrj.io.eval.EqualsEvaluator; import org.apache.solr.client.solrj.io.eval.ExclusiveOrEvaluator; +import org.apache.solr.client.solrj.io.eval.FloorEvaluator; import org.apache.solr.client.solrj.io.eval.GreaterThanEqualToEvaluator; import org.apache.solr.client.solrj.io.eval.GreaterThanEvaluator; +import org.apache.solr.client.solrj.io.eval.HyperbolicCosineEvaluator; +import org.apache.solr.client.solrj.io.eval.HyperbolicSineEvaluator; +import org.apache.solr.client.solrj.io.eval.HyperbolicTangentEvaluator; import org.apache.solr.client.solrj.io.eval.IfThenElseEvaluator; import org.apache.solr.client.solrj.io.eval.LessThanEqualToEvaluator; import org.apache.solr.client.solrj.io.eval.LessThanEvaluator; +import org.apache.solr.client.solrj.io.eval.ModuloEvaluator; import org.apache.solr.client.solrj.io.eval.MultiplyEvaluator; import org.apache.solr.client.solrj.io.eval.NaturalLogEvaluator; import org.apache.solr.client.solrj.io.eval.NotEvaluator; import org.apache.solr.client.solrj.io.eval.OrEvaluator; +import org.apache.solr.client.solrj.io.eval.PowerEvaluator; import org.apache.solr.client.solrj.io.eval.RawValueEvaluator; +import org.apache.solr.client.solrj.io.eval.RoundEvaluator; +import org.apache.solr.client.solrj.io.eval.SineEvaluator; +import org.apache.solr.client.solrj.io.eval.SquareRootEvaluator; import org.apache.solr.client.solrj.io.eval.SubtractEvaluator; +import org.apache.solr.client.solrj.io.eval.TangentEvaluator; import org.apache.solr.client.solrj.io.graph.GatherNodesStream; import org.apache.solr.client.solrj.io.graph.ShortestPathStream; import org.apache.solr.client.solrj.io.ops.ConcatOperation; import org.apache.solr.client.solrj.io.ops.DistinctOperation; import org.apache.solr.client.solrj.io.ops.GroupOperation; import org.apache.solr.client.solrj.io.ops.ReplaceOperation; -import org.apache.solr.client.solrj.io.stream.*; +import org.apache.solr.client.solrj.io.stream.CartesianProductStream; +import org.apache.solr.client.solrj.io.stream.CloudSolrStream; +import org.apache.solr.client.solrj.io.stream.CommitStream; +import org.apache.solr.client.solrj.io.stream.ComplementStream; +import org.apache.solr.client.solrj.io.stream.DaemonStream; +import org.apache.solr.client.solrj.io.stream.ExceptionStream; +import org.apache.solr.client.solrj.io.stream.ExecutorStream; +import org.apache.solr.client.solrj.io.stream.FacetStream; +import org.apache.solr.client.solrj.io.stream.FeaturesSelectionStream; +import org.apache.solr.client.solrj.io.stream.FetchStream; +import org.apache.solr.client.solrj.io.stream.HashJoinStream; +import org.apache.solr.client.solrj.io.stream.HavingStream; +import org.apache.solr.client.solrj.io.stream.InnerJoinStream; +import org.apache.solr.client.solrj.io.stream.IntersectStream; +import org.apache.solr.client.solrj.io.stream.JDBCStream; +import org.apache.solr.client.solrj.io.stream.LeftOuterJoinStream; +import org.apache.solr.client.solrj.io.stream.MergeStream; +import org.apache.solr.client.solrj.io.stream.ModelStream; +import org.apache.solr.client.solrj.io.stream.NullStream; +import org.apache.solr.client.solrj.io.stream.OuterHashJoinStream; +import org.apache.solr.client.solrj.io.stream.ParallelStream; +import org.apache.solr.client.solrj.io.stream.PriorityStream; +import org.apache.solr.client.solrj.io.stream.RandomStream; +import org.apache.solr.client.solrj.io.stream.RankStream; +import org.apache.solr.client.solrj.io.stream.ReducerStream; +import org.apache.solr.client.solrj.io.stream.RollupStream; +import org.apache.solr.client.solrj.io.stream.ScoreNodesStream; +import org.apache.solr.client.solrj.io.stream.SelectStream; +import org.apache.solr.client.solrj.io.stream.SignificantTermsStream; +import org.apache.solr.client.solrj.io.stream.SortStream; +import org.apache.solr.client.solrj.io.stream.StatsStream; +import org.apache.solr.client.solrj.io.stream.StreamContext; +import org.apache.solr.client.solrj.io.stream.TextLogitStream; +import org.apache.solr.client.solrj.io.stream.TopicStream; +import org.apache.solr.client.solrj.io.stream.TupleStream; +import org.apache.solr.client.solrj.io.stream.UniqueStream; +import org.apache.solr.client.solrj.io.stream.UpdateStream; import org.apache.solr.client.solrj.io.stream.expr.Explanation; import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType; import org.apache.solr.client.solrj.io.stream.expr.Expressible; @@ -80,9 +136,6 @@ import org.apache.solr.util.plugin.SolrCoreAware; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.solr.common.params.CommonParams.ID; -import static org.apache.solr.common.params.CommonParams.SORT; - public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, PermissionNameProvider { static SolrClientCache clientCache = new SolrClientCache(); @@ -207,6 +260,24 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, .withFunctionName("mult", MultiplyEvaluator.class) .withFunctionName("sub", SubtractEvaluator.class) .withFunctionName("log", NaturalLogEvaluator.class) + .withFunctionName("pow", PowerEvaluator.class) + .withFunctionName("mod", ModuloEvaluator.class) + .withFunctionName("ceil", CeilingEvaluator.class) + .withFunctionName("floor", FloorEvaluator.class) + .withFunctionName("sin", SineEvaluator.class) + .withFunctionName("asin", ArcSineEvaluator.class) + .withFunctionName("sinh", HyperbolicSineEvaluator.class) + .withFunctionName("cos", CosineEvaluator.class) + .withFunctionName("acos", ArcCosineEvaluator.class) + .withFunctionName("cosh", HyperbolicCosineEvaluator.class) + .withFunctionName("tan", TangentEvaluator.class) + .withFunctionName("atan", ArcTangentEvaluator.class) + .withFunctionName("tanh", HyperbolicTangentEvaluator.class) + .withFunctionName("round", RoundEvaluator.class) + .withFunctionName("sqrt", SquareRootEvaluator.class) + .withFunctionName("cbrt", CubedRootEvaluator.class) + .withFunctionName("coalesce", CoalesceEvaluator.class) + // Conditional Stream Evaluators .withFunctionName("if", IfThenElseEvaluator.class) .withFunctionName("analyze", AnalyzeEvaluator.class) diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcCosineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcCosineEvaluator.java new file mode 100644 index 00000000000..0c8e383786a --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcCosineEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class ArcCosineEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public ArcCosineEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.acos(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcSineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcSineEvaluator.java new file mode 100644 index 00000000000..ed95165dfaa --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcSineEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class ArcSineEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public ArcSineEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.asin(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcTangentEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcTangentEvaluator.java new file mode 100644 index 00000000000..9325b410ffb --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ArcTangentEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class ArcTangentEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public ArcTangentEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.atan(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java index f02f1fac65e..908562f8220 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/BooleanEvaluator.java @@ -24,13 +24,11 @@ import java.util.ArrayList; import java.util.List; import org.apache.solr.client.solrj.io.Tuple; -import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; public abstract class BooleanEvaluator extends ComplexEvaluator { protected static final long serialVersionUID = 1L; - protected StreamContext streamContext; public BooleanEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ super(expression, factory); @@ -48,11 +46,6 @@ public abstract class BooleanEvaluator extends ComplexEvaluator { return results; } - public void setStreamContext(StreamContext streamContext) { - this.streamContext = streamContext; - } - - public interface Checker { default boolean isNullAllowed(){ return false; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CeilingEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CeilingEvaluator.java new file mode 100644 index 00000000000..e2ccc8f1deb --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CeilingEvaluator.java @@ -0,0 +1,61 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class CeilingEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public CeilingEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return normalizeType(results.get(0).setScale(0, RoundingMode.CEILING)); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CoalesceEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CoalesceEvaluator.java new file mode 100644 index 00000000000..8a6eda40974 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CoalesceEvaluator.java @@ -0,0 +1,52 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class CoalesceEvaluator extends ComplexEvaluator { + protected static final long serialVersionUID = 1L; + + public CoalesceEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(subEvaluators.size() < 1){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at least one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Object evaluate(Tuple tuple) throws IOException { + + for(StreamEvaluator evaluator : subEvaluators){ + Object result = evaluator.evaluate(tuple); + if(null != result){ + return result; + } + } + + return null; + } +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java index 1e56d12bfcc..59a4653ef5e 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ComplexEvaluator.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Locale; import java.util.UUID; +import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.client.solrj.io.stream.expr.Explanation; import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType; import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; @@ -31,6 +32,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; public abstract class ComplexEvaluator implements StreamEvaluator { protected static final long serialVersionUID = 1L; + protected StreamContext streamContext; protected UUID nodeId = UUID.randomUUID(); @@ -96,4 +98,8 @@ public abstract class ComplexEvaluator implements StreamEvaluator { .withImplementingClass(getClass().getName()) .withExpression(toExpression(factory).toString()); } + + public void setStreamContext(StreamContext context) { + this.streamContext = context; + } } diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java index 61265441f8d..025bfae7d98 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ConditionalEvaluator.java @@ -24,13 +24,11 @@ import java.util.ArrayList; import java.util.List; import org.apache.solr.client.solrj.io.Tuple; -import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; public abstract class ConditionalEvaluator extends ComplexEvaluator { protected static final long serialVersionUID = 1L; - protected StreamContext streamContext; public ConditionalEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ super(expression, factory); @@ -45,10 +43,6 @@ public abstract class ConditionalEvaluator extends ComplexEvaluator { return results; } - public void setStreamContext(StreamContext streamContext) { - this.streamContext = streamContext; - } - public interface Checker { default boolean isNullAllowed(){ return false; diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CosineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CosineEvaluator.java new file mode 100644 index 00000000000..6adbb81f3cc --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CosineEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class CosineEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public CosineEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.cos(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CubedRootEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CubedRootEvaluator.java new file mode 100644 index 00000000000..4cd927789ad --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/CubedRootEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class CubedRootEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public CubedRootEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.cbrt(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FloorEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FloorEvaluator.java new file mode 100644 index 00000000000..0191a8e4802 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/FloorEvaluator.java @@ -0,0 +1,61 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class FloorEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public FloorEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return normalizeType(results.get(0).setScale(0, RoundingMode.FLOOR)); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicCosineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicCosineEvaluator.java new file mode 100644 index 00000000000..4e973a4e789 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicCosineEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class HyperbolicCosineEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public HyperbolicCosineEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.cosh(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicSineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicSineEvaluator.java new file mode 100644 index 00000000000..5bf4a38c76c --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicSineEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class HyperbolicSineEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public HyperbolicSineEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.sinh(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicTangentEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicTangentEvaluator.java new file mode 100644 index 00000000000..89aacd19d23 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/HyperbolicTangentEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class HyperbolicTangentEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public HyperbolicTangentEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.tanh(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ModuloEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ModuloEvaluator.java new file mode 100644 index 00000000000..928754b6360 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ModuloEvaluator.java @@ -0,0 +1,78 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.MathContext; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class ModuloEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public ModuloEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(2 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting two values but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // two found in the constructor could become != 2 + if(2 != results.size()){ + String message = null; + if(1 == results.size()){ + message = String.format(Locale.ROOT,"%s(...) only works with a 2 values (numerator,denominator) but 1 was provided", constructingFactory.getFunctionName(getClass())); + } + else{ + message = String.format(Locale.ROOT,"%s(...) only works with a 2 values (numerator,denominator) but %d were provided", constructingFactory.getFunctionName(getClass()), results.size()); + } + throw new IOException(message); + } + + BigDecimal numerator = results.get(0); + BigDecimal denominator = results.get(1); + + if(null == numerator){ + throw new IOException(String.format(Locale.ROOT,"Unable to %s(...) with a null numerator", constructingFactory.getFunctionName(getClass()))); + } + + if(null == denominator){ + throw new IOException(String.format(Locale.ROOT,"Unable to %s(...) with a null denominator", constructingFactory.getFunctionName(getClass()))); + } + + if(0 == denominator.compareTo(BigDecimal.ZERO)){ + throw new IOException(String.format(Locale.ROOT,"Unable to %s(...) with a 0 denominator", constructingFactory.getFunctionName(getClass()))); + } + + return normalizeType(numerator.remainder(denominator, MathContext.DECIMAL64)); + } +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java index 283c7b196e8..f4491fd45ff 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/NumberEvaluator.java @@ -26,13 +26,11 @@ import java.util.List; import java.util.Locale; import org.apache.solr.client.solrj.io.Tuple; -import org.apache.solr.client.solrj.io.stream.StreamContext; import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; public abstract class NumberEvaluator extends ComplexEvaluator { protected static final long serialVersionUID = 1L; - protected StreamContext streamContext; public NumberEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ super(expression, factory); @@ -40,10 +38,6 @@ public abstract class NumberEvaluator extends ComplexEvaluator { // restrict result to a Number public abstract Number evaluate(Tuple tuple) throws IOException; - - public void setStreamContext(StreamContext context) { - this.streamContext = context; - } public List evaluateAll(final Tuple tuple) throws IOException { // evaluate each and confirm they are all either null or numeric diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PowerEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PowerEvaluator.java new file mode 100644 index 00000000000..a8245b6e15e --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PowerEvaluator.java @@ -0,0 +1,61 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class PowerEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public PowerEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(2 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting exactly two values but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + if(results.stream().anyMatch(item -> null == item)){ + return null; + } + + BigDecimal value = results.get(0); + BigDecimal exponent = results.get(1); + + double result = Math.pow(value.doubleValue(), exponent.doubleValue()); + if(Double.isNaN(result)){ + return result; + } + + return normalizeType(BigDecimal.valueOf(result)); + } +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RoundEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RoundEvaluator.java new file mode 100644 index 00000000000..a34cdf4037a --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RoundEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class RoundEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public RoundEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.round(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SineEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SineEvaluator.java new file mode 100644 index 00000000000..1e2fbb5a6bb --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SineEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class SineEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public SineEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.sin(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SquareRootEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SquareRootEvaluator.java new file mode 100644 index 00000000000..74b9d81989c --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/SquareRootEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class SquareRootEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public SquareRootEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.sqrt(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TangentEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TangentEvaluator.java new file mode 100644 index 00000000000..d2a0476d1a2 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/TangentEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.Locale; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class TangentEvaluator extends NumberEvaluator { + protected static final long serialVersionUID = 1L; + + public TangentEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + super(expression, factory); + + if(1 != subEvaluators.size()){ + throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting one value but found %d",expression,subEvaluators.size())); + } + } + + @Override + public Number evaluate(Tuple tuple) throws IOException { + + List results = evaluateAll(tuple); + + // we're still doing these checks because if we ever add an array-flatten evaluator, + // one found in the constructor could become != 1 + if(1 != results.size()){ + throw new IOException(String.format(Locale.ROOT,"%s(...) only works with a 1 value but %d were provided", constructingFactory.getFunctionName(getClass()), results.size())); + } + + if(null == results.get(0)){ + return null; + } + + return Math.tan(results.get(0).doubleValue()); + } + +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcCosineEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcCosineEvaluatorTest.java new file mode 100644 index 00000000000..6a99a1cff0e --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcCosineEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.ArcCosineEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class ArcCosineEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public ArcCosineEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("acos", ArcCosineEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("acos(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.acos(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("acos()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("acos(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("acos(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcSineEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcSineEvaluatorTest.java new file mode 100644 index 00000000000..79e934bda7c --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcSineEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.ArcSineEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class ArcSineEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public ArcSineEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("asin", ArcSineEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("asin(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.asin(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("asin()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("asin(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("asin(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcTangentEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcTangentEvaluatorTest.java new file mode 100644 index 00000000000..7af225bd704 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ArcTangentEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.ArcTangentEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class ArcTangentEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public ArcTangentEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("atan", ArcTangentEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("atan(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.atan(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("atan()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("atan(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("atan(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CeilingEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CeilingEvaluatorTest.java new file mode 100644 index 00000000000..03395d2242e --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CeilingEvaluatorTest.java @@ -0,0 +1,96 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.CeilingEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class CeilingEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public CeilingEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("ceil", CeilingEvaluator.class); + values = new HashMap(); + } + + @Test + public void ceilingOneField() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("ceil(a)"); + Object result; + + values.clear(); + values.put("a", 1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(1L, result); + + values.clear(); + values.put("a", 1.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(2L, result); + + values.clear(); + values.put("a", -1.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(-1L, result); + } + + @Test(expected = IOException.class) + public void ceilNoField() throws Exception{ + factory.constructEvaluator("ceil()"); + } + + @Test(expected = IOException.class) + public void ceilTwoFields() throws Exception{ + factory.constructEvaluator("ceil(a,b)"); + } + + @Test + public void ceilNoValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("ceil(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void ceilNullValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("ceil(a)"); + + values.clear(); + values.put("a", null); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CoalesceEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CoalesceEvaluatorTest.java new file mode 100644 index 00000000000..79f46e729d9 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CoalesceEvaluatorTest.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for multitional 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.client.solrj.io.stream.eval; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.CoalesceEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class CoalesceEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public CoalesceEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("coalesce", CoalesceEvaluator.class); + values = new HashMap(); + } + + @Test + public void twoFieldsWithValues() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("coalesce(a,b)"); + Object result; + + values.clear(); + values.put("a", null); + values.put("b", 2); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertEquals(2, result); + + values.clear(); + values.put("a", 1.1); + values.put("b", null); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertEquals(1.1D, result); + + values.clear(); + values.put("a", "foo"); + values.put("b", 2.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertEquals("foo", result); + + values.clear(); + values.put("a", true); + values.put("b", 2.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertEquals(true, result); + + + values.clear(); + values.put("a", null); + values.put("b", false); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertEquals(false, result); + + values.clear(); + values.put("a", null); + values.put("b", null); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertNull(result); + } + + + @Test + public void twoFieldsWithMissingField() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("coalesce(a,b)"); + Object result; + + values.clear(); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertNull(result); + + } + + @Test + public void manyFieldsWithSubcoalesces() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("coalesce(a,b,coalesce(c,d))"); + Object result; + + values.clear(); + values.put("a", 1); + values.put("b", null); + values.put("c", null); + values.put("d", 4); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertEquals(1, result); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CosineEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CosineEvaluatorTest.java new file mode 100644 index 00000000000..6bb69137340 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CosineEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.CosineEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class CosineEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public CosineEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("cos", CosineEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("cos(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.cos(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("cos()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("cos(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("cos(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CubedRootEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CubedRootEvaluatorTest.java new file mode 100644 index 00000000000..0a7f3de8db9 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/CubedRootEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.CubedRootEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class CubedRootEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public CubedRootEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("cbrt", CubedRootEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("cbrt(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.cbrt(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("cbrt()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("cbrt(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("cbrt(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/FloorEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/FloorEvaluatorTest.java new file mode 100644 index 00000000000..0fbf16d909a --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/FloorEvaluatorTest.java @@ -0,0 +1,96 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.FloorEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class FloorEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public FloorEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("floor", FloorEvaluator.class); + values = new HashMap(); + } + + @Test + public void floorOneField() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("floor(a)"); + Object result; + + values.clear(); + values.put("a", 1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(1L, result); + + values.clear(); + values.put("a", 1.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(1L, result); + + values.clear(); + values.put("a", -1.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(-2L, result); + } + + @Test(expected = IOException.class) + public void floorNoField() throws Exception{ + factory.constructEvaluator("floor()"); + } + + @Test(expected = IOException.class) + public void floorTwoFields() throws Exception{ + factory.constructEvaluator("floor(a,b)"); + } + + @Test + public void floorNoValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("floor(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void floorNullValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("floor(a)"); + + values.clear(); + values.put("a", null); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicCosineEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicCosineEvaluatorTest.java new file mode 100644 index 00000000000..7847f30d846 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicCosineEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.HyperbolicCosineEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class HyperbolicCosineEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public HyperbolicCosineEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("cosh", HyperbolicCosineEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("cosh(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.cosh(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("cosh()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("cosh(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("cosh(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicSineEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicSineEvaluatorTest.java new file mode 100644 index 00000000000..22733cc9e1f --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicSineEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.HyperbolicSineEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class HyperbolicSineEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public HyperbolicSineEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("sinh", HyperbolicSineEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("sinh(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.sinh(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("sinh()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("sinh(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("sinh(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicTangentEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicTangentEvaluatorTest.java new file mode 100644 index 00000000000..e526a086201 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/HyperbolicTangentEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.HyperbolicTangentEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class HyperbolicTangentEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public HyperbolicTangentEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("tanh", HyperbolicTangentEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("tanh(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.tanh(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("tanh()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("tanh(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("tanh(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ModuloEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ModuloEvaluatorTest.java new file mode 100644 index 00000000000..436763b931d --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/ModuloEvaluatorTest.java @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for multitional 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.ModuloEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class ModuloEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public ModuloEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("mod", ModuloEvaluator.class); + values = new HashMap(); + } + + @Test + public void modTwoFieldsWithValues() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("mod(a,b)"); + Object result; + + values.clear(); + values.put("a", 1); + values.put("b", 2); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(Long.valueOf(1 % 2), result); + + values.clear(); + values.put("a", 1.1); + values.put("b", 2); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(1.1 % 2, result); + + values.clear(); + values.put("a", 1.1); + values.put("b", 2.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(1.1 % 2.1, result); + } + + @Test(expected = IOException.class) + public void modOneField() throws Exception{ + factory.constructEvaluator("mod(a)"); + } + + @Test(expected = IOException.class) + public void modTwoFieldWithNulls() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("mod(a,b)"); + + values.clear(); + evaluator.evaluate(new Tuple(values)); + } + + @Test(expected = IOException.class) + public void modTwoFieldsWithNullDenominator() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("mod(a,b)"); + + values.clear(); + values.put("a", 1); + evaluator.evaluate(new Tuple(values)); + } + + @Test(expected = IOException.class) + public void modTwoFieldsWithNullNumerator() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("mod(a,b)"); + + values.clear(); + values.put("b", 1); + evaluator.evaluate(new Tuple(values)); + } + + + @Test(expected = IOException.class) + public void modTwoFieldsWithMissingDenominator() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("mod(a,b)"); + + values.clear(); + values.put("a", 1); + evaluator.evaluate(new Tuple(values)); + } + + @Test(expected = IOException.class) + public void modTwoFieldsWithMissingNumerator() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("mod(a,b)"); + + values.clear(); + values.put("b", 1); + evaluator.evaluate(new Tuple(values)); + } + + + @Test(expected = IOException.class) + public void modManyFieldsWithValues() throws Exception{ + factory.constructEvaluator("mod(a,b,c,d)"); + } + + @Test + public void modManyFieldsWithSubmods() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("mod(a,mod(b,c))"); + Object result; + + values.clear(); + values.put("a", 1); + values.put("b", 2); + values.put("c", 9); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(Long.valueOf(1 % (2 % 9)), result); + } + + @Test(expected = IOException.class) + public void modByZero() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("mod(a,b)"); + + values.clear(); + values.put("a", 1); + values.put("b", 0); + evaluator.evaluate(new Tuple(values)); + } + + @Test + public void modZeroByValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("mod(a,b)"); + Object result; + + values.clear(); + values.put("a", 0); + values.put("b", 2); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(0L, result); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/PowerEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/PowerEvaluatorTest.java new file mode 100644 index 00000000000..5efa7a4adf7 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/PowerEvaluatorTest.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for multitional 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.PowerEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class PowerEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public PowerEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("pow", PowerEvaluator.class); + values = new HashMap(); + } + + @Test + public void powTwoFieldsWithValues() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("pow(a,b)"); + Object result; + + values.clear(); + values.put("a", 2); + values.put("b", 5); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Number); + Assert.assertEquals(BigDecimal.valueOf(Math.pow(2, 5)), BigDecimal.valueOf(result instanceof Long ? (long)result : (double)result)); + + values.clear(); + values.put("a", 1.1); + values.put("b", 2); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Number); + Assert.assertEquals(Math.pow(1.1, 2), result); + + values.clear(); + values.put("a", 1.1); + values.put("b", 2.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Number); + Assert.assertEquals(Math.pow(1.1, 2.1), result); + + values.clear(); + values.put("a", -1.1); + values.put("b", 2.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(Double.isNaN((double)result)); + + values.clear(); + values.put("a", 1.1); + values.put("b", -2.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Number); + Assert.assertEquals(Math.pow(1.1, -2.1), result); + + values.clear(); + values.put("a", -1.1); + values.put("b", -2.1); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(Double.isNaN((double)result)); + } + + @Test(expected = IOException.class) + public void powOneField() throws Exception{ + factory.constructEvaluator("pow(a)"); + } + + @Test + public void powTwoFieldWithNulls() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("pow(a,b)"); + + values.clear(); + Assert.assertNull(evaluator.evaluate(new Tuple(values))); + } + + @Test + public void powManyFieldsWithSubpows() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("pow(a,pow(b,c))"); + Object result; + + values.clear(); + values.put("a", 8); + values.put("b", 2); + values.put("c", 3); + result = evaluator.evaluate(new Tuple(values)); + Assert.assertTrue(result instanceof Number); + Assert.assertEquals(BigDecimal.valueOf(Math.pow(8, Math.pow(2, 3))), BigDecimal.valueOf(result instanceof Long ? (long)result : (double)result)); + } + +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/RoundEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/RoundEvaluatorTest.java new file mode 100644 index 00000000000..8851b3c621c --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/RoundEvaluatorTest.java @@ -0,0 +1,95 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.RoundEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class RoundEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public RoundEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("round", RoundEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("round(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Long); + Assert.assertEquals(Math.round(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45.555555D); + test(12.4D); + test(-.4D); + test(-0D); + test(-0.0235D); + test(-12.44444446D); + test(-45.23D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("round()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("round(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("round(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SineEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SineEvaluatorTest.java new file mode 100644 index 00000000000..8f8a9eb9e88 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SineEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.SineEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class SineEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public SineEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("sin", SineEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("sin(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.sin(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("sin()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("sin(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("sin(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SquareRootEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SquareRootEvaluatorTest.java new file mode 100644 index 00000000000..733a8f7701d --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/SquareRootEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.SquareRootEvaluator; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class SquareRootEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public SquareRootEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("sqrt", SquareRootEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("sqrt(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.sqrt(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("sqrt()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("sqrt(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("sqrt(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TangentEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TangentEvaluatorTest.java new file mode 100644 index 00000000000..ab0fbef09d3 --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/TangentEvaluatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.eval.TangentEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class TangentEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public TangentEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("tan", TangentEvaluator.class); + values = new HashMap(); + } + + private void test(Double value) throws IOException{ + StreamEvaluator evaluator = factory.constructEvaluator("tan(a)"); + + values.clear(); + values.put("a", value); + Object result = evaluator.evaluate(new Tuple(values)); + + if(null == value){ + Assert.assertNull(result); + } + else{ + Assert.assertTrue(result instanceof Double); + Assert.assertEquals(Math.tan(value), result); + } + } + + @Test + public void oneField() throws Exception{ + test(90D); + test(45D); + test(12.4D); + test(-45D); + } + + @Test(expected = IOException.class) + public void noField() throws Exception{ + factory.constructEvaluator("tan()"); + } + + @Test(expected = IOException.class) + public void twoFields() throws Exception{ + factory.constructEvaluator("tan(a,b)"); + } + + @Test + public void noValue() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("tan(a)"); + + values.clear(); + Object result = evaluator.evaluate(new Tuple(values)); + assertNull(result); + } + @Test + public void nullValue() throws Exception{ + test(null); + } +} From ef821834d15194c2c8b626d494b5119dd42b4f9f Mon Sep 17 00:00:00 2001 From: Dennis Gove Date: Fri, 31 Mar 2017 20:52:42 -0400 Subject: [PATCH 103/563] SOLR-10393: Adds UUID Streaming Evaluator --- solr/CHANGES.txt | 2 + .../apache/solr/handler/StreamHandler.java | 2 + .../client/solrj/io/eval/UuidEvaluator.java | 57 +++++++++++++++++++ .../io/stream/eval/UuidEvaluatorTest.java | 52 +++++++++++++++++ 4 files changed, 113 insertions(+) create mode 100644 solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/UuidEvaluator.java create mode 100644 solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/UuidEvaluatorTest.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 1c3aaf725d4..e30824f962e 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -63,6 +63,8 @@ New Features * SOLR-10356: Adds basic math Streaming Evaluators (Dennis Gove) +* SOLR-10393: Adds UUID Streaming Evaluator (Dennis Gove) + Bug Fixes ---------------------- * SOLR-9262: Connection and read timeouts are being ignored by UpdateShardHandler after SOLR-4509. diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java index b508754b20a..599924ed358 100644 --- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java @@ -67,6 +67,7 @@ import org.apache.solr.client.solrj.io.eval.SineEvaluator; import org.apache.solr.client.solrj.io.eval.SquareRootEvaluator; import org.apache.solr.client.solrj.io.eval.SubtractEvaluator; import org.apache.solr.client.solrj.io.eval.TangentEvaluator; +import org.apache.solr.client.solrj.io.eval.UuidEvaluator; import org.apache.solr.client.solrj.io.graph.GatherNodesStream; import org.apache.solr.client.solrj.io.graph.ShortestPathStream; import org.apache.solr.client.solrj.io.ops.ConcatOperation; @@ -277,6 +278,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, .withFunctionName("sqrt", SquareRootEvaluator.class) .withFunctionName("cbrt", CubedRootEvaluator.class) .withFunctionName("coalesce", CoalesceEvaluator.class) + .withFunctionName("uuid", UuidEvaluator.class) // Conditional Stream Evaluators .withFunctionName("if", IfThenElseEvaluator.class) diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/UuidEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/UuidEvaluator.java new file mode 100644 index 00000000000..88acee45f37 --- /dev/null +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/UuidEvaluator.java @@ -0,0 +1,57 @@ +/* + * 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.client.solrj.io.eval; + +import java.io.IOException; +import java.util.UUID; + +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.stream.expr.Explanation; +import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; +import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParameter; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; + +public class UuidEvaluator extends SimpleEvaluator { + protected static final long serialVersionUID = 1L; + + public UuidEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{ + // no parameters are used + } + + @Override + public UUID evaluate(Tuple tuple) throws IOException { + return UUID.randomUUID(); + } + + @Override + public StreamExpressionParameter toExpression(StreamFactory factory) throws IOException { + return new StreamExpression(factory.getFunctionName(getClass())); + } + + @Override + public Explanation toExplanation(StreamFactory factory) throws IOException { + return new Explanation(nodeId.toString()) + .withExpressionType(ExpressionType.EVALUATOR) + .withImplementingClass(getClass().getName()) + .withExpression(toExpression(factory).toString()); + } + +} diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/UuidEvaluatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/UuidEvaluatorTest.java new file mode 100644 index 00000000000..682b600493c --- /dev/null +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/eval/UuidEvaluatorTest.java @@ -0,0 +1,52 @@ +/* + * 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.client.solrj.io.stream.eval; + +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +import org.apache.lucene.util.LuceneTestCase; +import org.apache.solr.client.solrj.io.Tuple; +import org.apache.solr.client.solrj.io.eval.StreamEvaluator; +import org.apache.solr.client.solrj.io.eval.UuidEvaluator; +import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.junit.Test; + +import junit.framework.Assert; + +public class UuidEvaluatorTest extends LuceneTestCase { + + StreamFactory factory; + Map values; + + public UuidEvaluatorTest() { + super(); + + factory = new StreamFactory() + .withFunctionName("uuid", UuidEvaluator.class); + values = new HashMap(); + } + + @Test + public void isUuidType() throws Exception{ + StreamEvaluator evaluator = factory.constructEvaluator("uuid()"); + + Assert.assertTrue(evaluator.evaluate(null) instanceof UUID); + Assert.assertTrue(evaluator.evaluate(new Tuple(values)) instanceof UUID); + } +} From 99af830223a4ef69387586531e56338590659f7c Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Sun, 2 Apr 2017 16:25:09 -0400 Subject: [PATCH 104/563] switch to advanceExact --- .../apache/lucene/search/DiversifiedTopDocsCollector.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/lucene/misc/src/java/org/apache/lucene/search/DiversifiedTopDocsCollector.java b/lucene/misc/src/java/org/apache/lucene/search/DiversifiedTopDocsCollector.java index 6b756224439..1a087d47cf5 100644 --- a/lucene/misc/src/java/org/apache/lucene/search/DiversifiedTopDocsCollector.java +++ b/lucene/misc/src/java/org/apache/lucene/search/DiversifiedTopDocsCollector.java @@ -124,10 +124,7 @@ public abstract class DiversifiedTopDocsCollector extends // a leaf reader value when looking up keys int leafDocID = addition.doc - docBase; long value; - if (keys.docID() < leafDocID) { - keys.advance(leafDocID); - } - if (keys.docID() == leafDocID) { + if (keys.advanceExact(leafDocID)) { value = keys.longValue(); } else { value = 0; From e875f135bee21484386160b258b0eb6f0d2b7738 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Mon, 3 Apr 2017 12:10:09 +0100 Subject: [PATCH 105/563] SOLR-10383: reduce code duplication in TestOriginalScoreFeature --- .../ltr/feature/TestOriginalScoreFeature.java | 113 +++++++++--------- 1 file changed, 56 insertions(+), 57 deletions(-) diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java index d6512246896..4eb9bea280f 100644 --- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java +++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java @@ -53,51 +53,10 @@ public class TestOriginalScoreFeature extends TestRerankBase { @Test public void testOriginalScore() throws Exception { loadFeature("score", OriginalScoreFeature.class.getCanonicalName(), "{}"); - loadModel("originalScore", LinearModel.class.getCanonicalName(), new String[] {"score"}, "{\"weights\":{\"score\":1.0}}"); - final SolrQuery query = new SolrQuery(); - query.setQuery("title:w1"); - query.add("fl", "*, score"); - query.add("rows", "4"); - query.add("wt", "json"); - - // Normal term match - assertJQ("/query" + query.toQueryString(), "/response/numFound/==4"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='7'"); - - final String res = restTestHarness.query("/query" + query.toQueryString()); - final Map jsonParse = (Map) ObjectBuilder - .fromJSON(res); - final String doc0Score = ((Double) ((Map) ((ArrayList) ((Map) jsonParse - .get("response")).get("docs")).get(0)).get("score")).toString(); - final String doc1Score = ((Double) ((Map) ((ArrayList) ((Map) jsonParse - .get("response")).get("docs")).get(1)).get("score")).toString(); - final String doc2Score = ((Double) ((Map) ((ArrayList) ((Map) jsonParse - .get("response")).get("docs")).get(2)).get("score")).toString(); - final String doc3Score = ((Double) ((Map) ((ArrayList) ((Map) jsonParse - .get("response")).get("docs")).get(3)).get("score")).toString(); - - query.add("fl", "[fv]"); - query.add("rq", "{!ltr model=originalScore reRankDocs=4}"); - - assertJQ("/query" + query.toQueryString(), "/response/numFound/==4"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/score==" - + doc0Score); - assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/score==" - + doc1Score); - assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/score==" - + doc2Score); - assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='7'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/score==" - + doc3Score); + implTestOriginalScoreResponseDocsCheck("originalScore", "score", null, null); } @Test @@ -111,12 +70,29 @@ public class TestOriginalScoreFeature extends TestRerankBase { new String[] {"origScore"}, "store2", "{\"weights\":{\"origScore\":1.0}}"); + implTestOriginalScoreResponseDocsCheck("origScore", "origScore", "c2", "2.0"); + } + + public static void implTestOriginalScoreResponseDocsCheck(String modelName, + String origScoreFeatureName, + String nonScoringFeatureName, String nonScoringFeatureValue) throws Exception { + final SolrQuery query = new SolrQuery(); query.setQuery("title:w1"); - query.add("fl", "*, score, fv:[fv]"); + query.add("fl", "*, score"); query.add("rows", "4"); query.add("wt", "json"); - query.add("rq", "{!ltr model=origScore reRankDocs=4}"); + + final int doc0Id = 1; + final int doc1Id = 8; + final int doc2Id = 6; + final int doc3Id = 7; + + assertJQ("/query" + query.toQueryString(), "/response/numFound/==4"); + assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='"+doc0Id+"'"); + assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='"+doc1Id+"'"); + assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='"+doc2Id+"'"); + assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='"+doc3Id+"'"); final String res = restTestHarness.query("/query" + query.toQueryString()); final Map jsonParse = (Map) ObjectBuilder @@ -130,20 +106,43 @@ public class TestOriginalScoreFeature extends TestRerankBase { final String doc3Score = ((Double) ((Map) ((ArrayList) ((Map) jsonParse .get("response")).get("docs")).get(3)).get("score")).toString(); - assertJQ("/query" + query.toQueryString(), "/response/numFound/==4"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='1'"); - assertJQ("/query" + query.toQueryString(), - "/response/docs/[0]/fv=='" + FeatureLoggerTestUtils.toFeatureVector("origScore", doc0Score, "c2", "2.0")+"'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='8'"); + final boolean debugQuery = false; - assertJQ("/query" + query.toQueryString(), - "/response/docs/[1]/fv=='" + FeatureLoggerTestUtils.toFeatureVector("origScore", doc1Score, "c2", "2.0")+"'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='6'"); - assertJQ("/query" + query.toQueryString(), - "/response/docs/[2]/fv=='" + FeatureLoggerTestUtils.toFeatureVector("origScore", doc2Score, "c2", "2.0")+"'"); - assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='7'"); - assertJQ("/query" + query.toQueryString(), - "/response/docs/[3]/fv=='" + FeatureLoggerTestUtils.toFeatureVector("origScore", doc3Score, "c2", "2.0")+"'"); + query.remove("fl"); + query.add("fl", "*, score, fv:[fv]"); + query.add("rq", "{!ltr model="+modelName+" reRankDocs=4}"); + + assertJQ("/query" + query.toQueryString(), "/response/numFound/==4"); + assertJQ("/query" + query.toQueryString(), "/response/docs/[0]/id=='"+doc0Id+"'"); + assertJQ("/query" + query.toQueryString(), "/response/docs/[1]/id=='"+doc1Id+"'"); + assertJQ("/query" + query.toQueryString(), "/response/docs/[2]/id=='"+doc2Id+"'"); + assertJQ("/query" + query.toQueryString(), "/response/docs/[3]/id=='"+doc3Id+"'"); + + implTestOriginalScoreResponseDocsCheck(modelName, query, 0, doc0Id, origScoreFeatureName, doc0Score, + nonScoringFeatureName, nonScoringFeatureValue, debugQuery); + implTestOriginalScoreResponseDocsCheck(modelName, query, 1, doc1Id, origScoreFeatureName, doc1Score, + nonScoringFeatureName, nonScoringFeatureValue, debugQuery); + implTestOriginalScoreResponseDocsCheck(modelName, query, 2, doc2Id, origScoreFeatureName, doc2Score, + nonScoringFeatureName, nonScoringFeatureValue, debugQuery); + implTestOriginalScoreResponseDocsCheck(modelName, query, 3, doc3Id, origScoreFeatureName, doc3Score, + nonScoringFeatureName, nonScoringFeatureValue, debugQuery); + } + + private static void implTestOriginalScoreResponseDocsCheck(String modelName, + SolrQuery query, int docIdx, int docId, + String origScoreFeatureName, String origScoreFeatureValue, + String nonScoringFeatureName, String nonScoringFeatureValue, + boolean debugQuery) throws Exception { + + final String fv; + if (nonScoringFeatureName == null) { + fv = FeatureLoggerTestUtils.toFeatureVector(origScoreFeatureName, origScoreFeatureValue); + } else { + fv = FeatureLoggerTestUtils.toFeatureVector(origScoreFeatureName, origScoreFeatureValue, nonScoringFeatureName, nonScoringFeatureValue); + } + + assertJQ("/query" + query.toQueryString(), "/response/docs/["+docIdx+"]/fv=='"+fv+"'"); + // TODO: use debugQuery } } From 186c5edd63fe292388ad435bc1cbb1a32a3a3824 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Mon, 3 Apr 2017 13:01:16 +0100 Subject: [PATCH 106/563] SOLR-10383: Fix debug related NullPointerException in solr/contrib/ltr OriginalScoreFeature class. (Vitezslav Zak, Christine Poerschke) --- solr/CHANGES.txt | 8 ++++++++ .../apache/solr/ltr/feature/OriginalScoreFeature.java | 2 +- .../solr/ltr/feature/TestOriginalScoreFeature.java | 11 +++++++++-- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index e30824f962e..cd4f7f5f1ab 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -188,6 +188,14 @@ Other Changes * SOLR-9601: Redone DataImportHandler 'tika' example, removing all unused and irrelevant definitions (Alexandre Rafalovitch) +================== 6.5.1 ================== + +Bug Fixes +---------------------- + +* SOLR-10383: Fix debug related NullPointerException in solr/contrib/ltr OriginalScoreFeature class. + (Vitezslav Zak, Christine Poerschke) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/OriginalScoreFeature.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/OriginalScoreFeature.java index 549880be06e..85fb8fdf5e4 100644 --- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/OriginalScoreFeature.java +++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/feature/OriginalScoreFeature.java @@ -104,7 +104,7 @@ public class OriginalScoreFeature extends Feature { // was already scored in step 1 // we shouldn't need to calc original score again. final DocInfo docInfo = getDocInfo(); - return (docInfo.hasOriginalDocScore() ? docInfo.getOriginalDocScore() : originalScorer.score()); + return (docInfo != null && docInfo.hasOriginalDocScore() ? docInfo.getOriginalDocScore() : originalScorer.score()); } @Override diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java index 4eb9bea280f..f4c0df1fbf5 100644 --- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java +++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/feature/TestOriginalScoreFeature.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Map; import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.ltr.FeatureLoggerTestUtils; import org.apache.solr.ltr.TestRerankBase; import org.apache.solr.ltr.model.LinearModel; @@ -106,7 +107,10 @@ public class TestOriginalScoreFeature extends TestRerankBase { final String doc3Score = ((Double) ((Map) ((ArrayList) ((Map) jsonParse .get("response")).get("docs")).get(3)).get("score")).toString(); - final boolean debugQuery = false; + final boolean debugQuery = random().nextBoolean(); + if (debugQuery) { + query.add(CommonParams.DEBUG_QUERY, "true"); + } query.remove("fl"); query.add("fl", "*, score, fv:[fv]"); @@ -142,7 +146,10 @@ public class TestOriginalScoreFeature extends TestRerankBase { } assertJQ("/query" + query.toQueryString(), "/response/docs/["+docIdx+"]/fv=='"+fv+"'"); - // TODO: use debugQuery + if (debugQuery) { + assertJQ("/query" + query.toQueryString(), + "/debug/explain/"+docId+"=='\n"+origScoreFeatureValue+" = LinearModel(name="+modelName+",featureWeights=["+origScoreFeatureName+"=1.0]) model applied to features, sum of:\n "+origScoreFeatureValue+" = prod of:\n 1.0 = weight on feature\n "+origScoreFeatureValue+" = OriginalScoreFeature [query:"+query.getQuery()+"]\n'"); + } } } From 2e545d78f5fe745905bcff19eb73a9a9faa4c032 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 3 Apr 2017 13:49:05 +0200 Subject: [PATCH 107/563] LUCENE-7749: Made LRUQueryCache delegate the scoreSupplier method. --- lucene/CHANGES.txt | 3 + .../apache/lucene/search/LRUQueryCache.java | 32 ++++++-- .../lucene/search/TestLRUQueryCache.java | 76 +++++++++++++++++++ 3 files changed, 105 insertions(+), 6 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 833fd3c9ef3..83113a8238e 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -106,6 +106,9 @@ Bug Fixes * LUCENE-7755: Fixed join queries to not reference IndexReaders, as it could cause leaks if they are cached. (Adrien Grand) +* LUCENE-7749: Made LRUQueryCache delegate the scoreSupplier method. + (Martin Amirault via Adrien Grand) + Other * LUCENE-7763: Remove outdated comment in IndexWriterConfig.setIndexSort javadocs. diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java index b1ba4e4ca49..451ce814ecd 100644 --- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java +++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java @@ -717,7 +717,7 @@ public class LRUQueryCache implements QueryCache, Accountable { } @Override - public Scorer scorer(LeafReaderContext context) throws IOException { + public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { if (used.compareAndSet(false, true)) { policy.onUse(getQuery()); } @@ -726,18 +726,18 @@ public class LRUQueryCache implements QueryCache, Accountable { final IndexReader.CacheHelper cacheHelper = context.reader().getCoreCacheHelper(); if (cacheHelper == null) { // this segment is not suitable for caching - return in.scorer(context); + return in.scorerSupplier(context); } // Short-circuit: Check whether this segment is eligible for caching // before we take a lock because of #get if (shouldCache(context) == false) { - return in.scorer(context); + return in.scorerSupplier(context); } // If the lock is already busy, prefer using the uncached version than waiting if (lock.tryLock() == false) { - return in.scorer(context); + return in.scorerSupplier(context); } DocIdSet docIdSet; @@ -752,7 +752,7 @@ public class LRUQueryCache implements QueryCache, Accountable { docIdSet = cache(context); putIfAbsent(in.getQuery(), context, docIdSet, cacheHelper); } else { - return in.scorer(context); + return in.scorerSupplier(context); } } @@ -765,7 +765,27 @@ public class LRUQueryCache implements QueryCache, Accountable { return null; } - return new ConstantScoreScorer(this, 0f, disi); + return new ScorerSupplier() { + @Override + public Scorer get(boolean randomAccess) throws IOException { + return new ConstantScoreScorer(CachingWrapperWeight.this, 0f, disi); + } + + @Override + public long cost() { + return disi.cost(); + } + }; + + } + + @Override + public Scorer scorer(LeafReaderContext context) throws IOException { + ScorerSupplier scorerSupplier = scorerSupplier(context); + if (scorerSupplier == null) { + return null; + } + return scorerSupplier.get(false); } @Override diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java index 91c1887248b..ee3f4ad9bb2 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java @@ -1273,4 +1273,80 @@ public class TestLRUQueryCache extends LuceneTestCase { w.close(); dir.close(); } + + private static class DummyQuery2 extends Query { + + private final AtomicBoolean scorerCreated; + + DummyQuery2(AtomicBoolean scorerCreated) { + this.scorerCreated = scorerCreated; + } + + @Override + public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + return new ConstantScoreWeight(this, boost) { + @Override + public Scorer scorer(LeafReaderContext context) throws IOException { + return scorerSupplier(context).get(false); + } + @Override + public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { + final Weight weight = this; + return new ScorerSupplier() { + @Override + public Scorer get(boolean randomAccess) throws IOException { + scorerCreated.set(true); + return new ConstantScoreScorer(weight, boost, DocIdSetIterator.all(1)); + } + + @Override + public long cost() { + return 1; + } + }; + } + }; + } + + @Override + public boolean equals(Object other) { + return sameClassAs(other); + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public String toString(String field) { + return "DummyQuery2"; + } + + } + + public void testPropagatesScorerSupplier() throws IOException { + Directory dir = newDirectory(); + IndexWriterConfig iwc = newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE); + RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc); + w.addDocument(new Document()); + DirectoryReader reader = w.getReader(); + IndexSearcher searcher = newSearcher(reader); + searcher.setQueryCachingPolicy(NEVER_CACHE); + + LRUQueryCache cache = new LRUQueryCache(1, 1000); + searcher.setQueryCache(cache); + + AtomicBoolean scorerCreated = new AtomicBoolean(false); + Query query = new DummyQuery2(scorerCreated); + Weight weight = searcher.createNormalizedWeight(query, false); + ScorerSupplier supplier = weight.scorerSupplier(searcher.getIndexReader().leaves().get(0)); + assertFalse(scorerCreated.get()); + supplier.get(random().nextBoolean()); + assertTrue(scorerCreated.get()); + + reader.close(); + w.close(); + dir.close(); + } } From 52632cfc0c0c945cff2e769e6c2dc4dc9a5da400 Mon Sep 17 00:00:00 2001 From: Erick Erickson Date: Mon, 3 Apr 2017 13:27:12 -0700 Subject: [PATCH 108/563] SOLR-8906: Make transient core cache pluggable --- solr/CHANGES.txt | 2 + .../org/apache/solr/core/CoreContainer.java | 25 ++- .../java/org/apache/solr/core/NodeConfig.java | 25 ++- .../java/org/apache/solr/core/SolrCores.java | 166 +++++++++------ .../org/apache/solr/core/SolrXmlConfig.java | 5 + .../solr/core/TransientSolrCoreCache.java | 127 +++++++++++ .../core/TransientSolrCoreCacheDefault.java | 198 ++++++++++++++++++ .../core/TransientSolrCoreCacheFactory.java | 85 ++++++++ .../TransientSolrCoreCacheFactoryDefault.java | 31 +++ solr/core/src/test-files/solr/solr.xml | 5 + .../apache/solr/cloud/ZkControllerTest.java | 9 +- .../apache/solr/core/TestCoreDiscovery.java | 7 +- .../org/apache/solr/core/TestLazyCores.java | 53 ++++- .../java/org/apache/solr/SolrTestCaseJ4.java | 13 +- 14 files changed, 662 insertions(+), 89 deletions(-) create mode 100644 solr/core/src/java/org/apache/solr/core/TransientSolrCoreCache.java create mode 100644 solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheDefault.java create mode 100644 solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheFactory.java create mode 100644 solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheFactoryDefault.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index cd4f7f5f1ab..6fe4cc0bf3e 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -188,6 +188,8 @@ Other Changes * SOLR-9601: Redone DataImportHandler 'tika' example, removing all unused and irrelevant definitions (Alexandre Rafalovitch) +* SOLR-8906: Make transient core cache pluggable (Erick Erickson) + ================== 6.5.1 ================== 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 5ec34ba8ee1..1ef036aa232 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -130,6 +130,7 @@ public class CoreContainer { protected CoreAdminHandler coreAdminHandler = null; protected CollectionsHandler collectionsHandler = null; + protected TransientSolrCoreCache transientSolrCoreCache = null; private InfoHandler infoHandler; protected ConfigSetsHandler configSetsHandler = null; @@ -144,6 +145,8 @@ public class CoreContainer { private UpdateShardHandler updateShardHandler; + private TransientSolrCoreCacheFactory transientCoreCache; + private ExecutorService coreContainerWorkExecutor = ExecutorUtil.newMDCAwareCachedThreadPool( new DefaultSolrThreadFactory("coreContainerWorkExecutor") ); @@ -492,7 +495,7 @@ public class CoreContainer { updateShardHandler = new UpdateShardHandler(cfg.getUpdateShardHandlerConfig()); updateShardHandler.initializeMetrics(metricManager, SolrInfoMBean.Group.node.toString(), "updateShardHandler"); - solrCores.allocateLazyCores(cfg.getTransientCacheSize(), loader); + transientCoreCache = TransientSolrCoreCacheFactory.newInstance(loader, this); logging = LogWatcher.newRegisteredLogWatcher(cfg.getLogWatcherConfig(), loader); @@ -535,9 +538,9 @@ public class CoreContainer { 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(), + metricManager.registerGauge(registryName, () -> solrCores.getLoadedCoreNames().size() - solrCores.getCores().size(), true, "lazy",SolrInfoMBean.Category.CONTAINER.toString(), "cores"); - metricManager.registerGauge(registryName, () -> solrCores.getAllCoreNames().size() - solrCores.getCoreNames().size(), + metricManager.registerGauge(registryName, () -> solrCores.getAllCoreNames().size() - solrCores.getLoadedCoreNames().size(), true, "unloaded",SolrInfoMBean.Category.CONTAINER.toString(), "cores"); metricManager.registerGauge(registryName, () -> cfg.getCoreRootDirectory().toFile().getTotalSpace(), true, "totalSpace", SolrInfoMBean.Category.CONTAINER.toString(), "fs"); @@ -629,6 +632,16 @@ public class CoreContainer { } } + public TransientSolrCoreCache getTransientCacheHandler() { + + if (transientCoreCache == null) { + log.error("No transient handler has been defined. Check solr.xml to see if an attempt to provide a custom " + + "TransientSolrCoreCacheFactory was done incorrectly since the default should have been used otherwise."); + return null; + } + return transientCoreCache.getTransientSolrCoreCache(); + } + public void securityNodeChanged() { log.info("Security node changed, reloading security.json"); reloadSecurityProperties(); @@ -1076,10 +1089,10 @@ public class CoreContainer { } /** - * @return a Collection of the names that cores are mapped to + * @return a Collection of the names that loaded cores are mapped to */ public Collection getCoreNames() { - return solrCores.getCoreNames(); + return solrCores.getLoadedCoreNames(); } /** This method is currently experimental. @@ -1092,6 +1105,8 @@ public class CoreContainer { /** * get a list of all the cores that are currently loaded * @return a list of al lthe available core names in either permanent or transient core lists. + * + * Note: this implies that the core is loaded */ public Collection getAllCoreNames() { return solrCores.getAllCoreNames(); diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java index 258fd140e01..de2dcead05f 100644 --- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java +++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java @@ -52,6 +52,8 @@ public class NodeConfig { private final Integer coreLoadThreads; + @Deprecated + // This should be part of the transientCacheConfig, remove in 7.0 private final int transientCacheSize; private final boolean useSchemaCache; @@ -62,6 +64,8 @@ public class NodeConfig { private final PluginInfo[] metricReporterPlugins; + private final PluginInfo transientCacheConfig; + private NodeConfig(String nodeName, Path coreRootDirectory, Path configSetBaseDirectory, String sharedLibDirectory, PluginInfo shardHandlerFactoryConfig, UpdateShardHandlerConfig updateShardHandlerConfig, String coreAdminHandlerClass, String collectionsAdminHandlerClass, @@ -69,7 +73,7 @@ public class NodeConfig { LogWatcherConfig logWatcherConfig, CloudConfig cloudConfig, Integer coreLoadThreads, int transientCacheSize, boolean useSchemaCache, String managementPath, SolrResourceLoader loader, Properties solrProperties, PluginInfo[] backupRepositoryPlugins, - PluginInfo[] metricReporterPlugins) { + PluginInfo[] metricReporterPlugins, PluginInfo transientCacheConfig) { this.nodeName = nodeName; this.coreRootDirectory = coreRootDirectory; this.configSetBaseDirectory = configSetBaseDirectory; @@ -90,6 +94,7 @@ public class NodeConfig { this.solrProperties = solrProperties; this.backupRepositoryPlugins = backupRepositoryPlugins; this.metricReporterPlugins = metricReporterPlugins; + this.transientCacheConfig = transientCacheConfig; if (this.cloudConfig != null && this.getCoreLoadThreadCount(false) < 2) { throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, @@ -182,6 +187,8 @@ public class NodeConfig { return metricReporterPlugins; } + public PluginInfo getTransientCachePluginInfo() { return transientCacheConfig; } + public static class NodeConfigBuilder { private Path coreRootDirectory; @@ -195,13 +202,16 @@ public class NodeConfig { private String configSetsHandlerClass = DEFAULT_CONFIGSETSHANDLERCLASS; private LogWatcherConfig logWatcherConfig = new LogWatcherConfig(true, null, null, 50); private CloudConfig cloudConfig; - private Integer coreLoadThreads; + private int coreLoadThreads = DEFAULT_CORE_LOAD_THREADS; + @Deprecated + //Remove in 7.0 and put it all in the transientCache element in solrconfig.xml private int transientCacheSize = DEFAULT_TRANSIENT_CACHE_SIZE; private boolean useSchemaCache = false; private String managementPath; private Properties solrProperties = new Properties(); private PluginInfo[] backupRepositoryPlugins; private PluginInfo[] metricReporterPlugins; + private PluginInfo transientCacheConfig; private final SolrResourceLoader loader; private final String nodeName; @@ -210,7 +220,7 @@ public class NodeConfig { //No:of core load threads in cloud mode is set to a default of 8 public static final int DEFAULT_CORE_LOAD_THREADS_IN_CLOUD = 8; - private static final int DEFAULT_TRANSIENT_CACHE_SIZE = Integer.MAX_VALUE; + public static final int DEFAULT_TRANSIENT_CACHE_SIZE = Integer.MAX_VALUE; private static final String DEFAULT_ADMINHANDLERCLASS = "org.apache.solr.handler.admin.CoreAdminHandler"; private static final String DEFAULT_INFOHANDLERCLASS = "org.apache.solr.handler.admin.InfoHandler"; @@ -284,6 +294,8 @@ public class NodeConfig { return this; } + // Remove in Solr 7.0 + @Deprecated public NodeConfigBuilder setTransientCacheSize(int transientCacheSize) { this.transientCacheSize = transientCacheSize; return this; @@ -313,12 +325,17 @@ public class NodeConfig { this.metricReporterPlugins = metricReporterPlugins; return this; } + + public NodeConfigBuilder setSolrCoreCacheFactoryConfig(PluginInfo transientCacheConfig) { + this.transientCacheConfig = transientCacheConfig; + return this; + } public NodeConfig build() { return new NodeConfig(nodeName, coreRootDirectory, configSetBaseDirectory, sharedLibDirectory, shardHandlerFactoryConfig, updateShardHandlerConfig, coreAdminHandlerClass, collectionsAdminHandlerClass, infoHandlerClass, configSetsHandlerClass, logWatcherConfig, cloudConfig, coreLoadThreads, transientCacheSize, useSchemaCache, managementPath, loader, solrProperties, - backupRepositoryPlugins, metricReporterPlugins); + backupRepositoryPlugins, metricReporterPlugins, transientCacheConfig); } } } diff --git a/solr/core/src/java/org/apache/solr/core/SolrCores.java b/solr/core/src/java/org/apache/solr/core/SolrCores.java index b25e9bb3972..40d511558f6 100644 --- a/solr/core/src/java/org/apache/solr/core/SolrCores.java +++ b/solr/core/src/java/org/apache/solr/core/SolrCores.java @@ -17,6 +17,7 @@ package org.apache.solr.core; import com.google.common.collect.Lists; +import org.apache.http.annotation.Experimental; import org.apache.solr.common.SolrException; import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.logging.MDCLoggingContext; @@ -32,6 +33,8 @@ import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Observable; +import java.util.Observer; import java.util.Set; import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; @@ -39,15 +42,12 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; -class SolrCores { +class SolrCores implements Observer { private static Object modifyLock = new Object(); // for locking around manipulating any of the core maps. private final Map cores = new LinkedHashMap<>(); // For "permanent" cores - //WARNING! The _only_ place you put anything into the list of transient cores is with the putTransientCore method! - private Map transientCores = new LinkedHashMap<>(); // For "lazily loaded" cores - - private final Map dynamicDescriptors = new LinkedHashMap<>(); + private final Map lazyDescriptors = new LinkedHashMap<>(); private final CoreContainer container; @@ -66,33 +66,19 @@ class SolrCores { SolrCores(CoreContainer container) { this.container = container; } - - // Trivial helper method for load, note it implements LRU on transient cores. Also note, if - // there is no setting for max size, nothing is done and all cores go in the regular "cores" list - protected void allocateLazyCores(final int cacheSize, final SolrResourceLoader loader) { - if (cacheSize != Integer.MAX_VALUE) { - log.info("Allocating transient cache for {} transient cores", cacheSize); - transientCores = new LinkedHashMap(cacheSize, 0.75f, true) { - @Override - protected boolean removeEldestEntry(Map.Entry eldest) { - if (size() > cacheSize) { - synchronized (modifyLock) { - SolrCore coreToClose = eldest.getValue(); - log.info("Closing transient core [{}]", coreToClose.getName()); - pendingCloses.add(coreToClose); // Essentially just queue this core up for closing. - modifyLock.notifyAll(); // Wakes up closer thread too - } - return true; - } - return false; - } - }; - } - } - - protected void putDynamicDescriptor(String rawName, CoreDescriptor p) { + + protected void putDynamicDescriptor(String rawName, CoreDescriptor cd) { synchronized (modifyLock) { - dynamicDescriptors.put(rawName, p); + if (cd.isTransient()) { + if (container.getTransientCacheHandler() != null) { + container.getTransientCacheHandler().addTransientDescriptor(rawName, cd); + } else { + log.error("Tried to add transient core to transient handler, but no transient core handler has been found. " + + " Descriptor: " + cd.toString()); + } + } else { + lazyDescriptors.put(rawName, cd); + } } } @@ -101,19 +87,25 @@ class SolrCores { protected void close() { Collection coreList = new ArrayList<>(); + + TransientSolrCoreCache transientSolrCoreCache = container.getTransientCacheHandler(); + // Release observer + if (transientSolrCoreCache != null) { + transientSolrCoreCache.close(); + } + // It might be possible for one of the cores to move from one list to another while we're closing them. So // loop through the lists until they're all empty. In particular, the core could have moved from the transient // list to the pendingCloses list. - do { coreList.clear(); synchronized (modifyLock) { // make a copy of the cores then clear the map so the core isn't handed out to a request again coreList.addAll(cores.values()); cores.clear(); - - coreList.addAll(transientCores.values()); - transientCores.clear(); + if (transientSolrCoreCache != null) { + coreList.addAll(transientSolrCoreCache.prepareForShutdown()); + } coreList.addAll(pendingCloses); pendingCloses.clear(); @@ -147,10 +139,12 @@ class SolrCores { //WARNING! This should be the _only_ place you put anything into the list of transient cores! protected SolrCore putTransientCore(NodeConfig cfg, String name, SolrCore core, SolrResourceLoader loader) { - SolrCore retCore; + SolrCore retCore = null; log.info("Opening transient core {}", name); synchronized (modifyLock) { - retCore = transientCores.put(name, core); + if (container.getTransientCacheHandler() != null) { + retCore = container.getTransientCacheHandler().addCore(name, core); + } } return retCore; } @@ -161,6 +155,17 @@ class SolrCores { } } + /** + * + * @return A list of "permanent" cores, i.e. cores that may not be swapped out and are currently loaded. + * + * A core may be non-transient but still lazily loaded. If it is "permanent" and lazy-load _and_ + * not yet loaded it will _not_ be returned by this call. + * + * Note: This is one of the places where SolrCloud is incompatible with Transient Cores. This call is used in + * cancelRecoveries, transient cores don't participate. + */ + List getCores() { List lst = new ArrayList<>(); @@ -170,16 +175,34 @@ class SolrCores { } } - Set getCoreNames() { + /** + * Gets the cores that are currently loaded, i.e. cores that have + * 1> loadOnStartup=true and are either not-transient or, if transient, have been loaded and have not been swapped out + * 2> loadOnStartup=false and have been loaded but either non-transient or have not been swapped out. + * + * Put another way, this will not return any names of cores that are lazily loaded but have not been called for yet + * or are transient and either not loaded or have been swapped out. + * + * @return List of currently loaded cores. + */ + Set getLoadedCoreNames() { Set set = new TreeSet<>(); synchronized (modifyLock) { set.addAll(cores.keySet()); - set.addAll(transientCores.keySet()); + if (container.getTransientCacheHandler() != null) { + set.addAll(container.getTransientCacheHandler().getLoadedCoreNames()); + } } return set; } + /** This method is currently experimental. + * @return a Collection of the names that a specific core is mapped to. + * + * Note: this implies that the core is loaded + */ + @Experimental List getCoreNames(SolrCore core) { List lst = new ArrayList<>(); @@ -189,26 +212,26 @@ class SolrCores { lst.add(entry.getKey()); } } - for (Map.Entry entry : transientCores.entrySet()) { - if (core == entry.getValue()) { - lst.add(entry.getKey()); - } + if (container.getTransientCacheHandler() != null) { + lst.addAll(container.getTransientCacheHandler().getNamesForCore(core)); } } return lst; } /** - * Gets a list of all cores, loaded and unloaded (dynamic) + * Gets a list of all cores, loaded and unloaded * - * @return all cores names, whether loaded or unloaded. + * @return all cores names, whether loaded or unloaded, transient or permenent. */ public Collection getAllCoreNames() { Set set = new TreeSet<>(); synchronized (modifyLock) { set.addAll(cores.keySet()); - set.addAll(transientCores.keySet()); - set.addAll(dynamicDescriptors.keySet()); + if (container.getTransientCacheHandler() != null) { + set.addAll(container.getTransientCacheHandler().getAllCoreNames()); + } + set.addAll(lazyDescriptors.keySet()); } return set; } @@ -251,14 +274,15 @@ class SolrCores { protected SolrCore remove(String name) { synchronized (modifyLock) { - SolrCore tmp = cores.remove(name); - SolrCore ret = null; - ret = (ret == null) ? tmp : ret; + SolrCore ret = cores.remove(name); // It could have been a newly-created core. It could have been a transient core. The newly-created cores // in particular should be checked. It could have been a dynamic core. - tmp = transientCores.remove(name); - ret = (ret == null) ? tmp : ret; - dynamicDescriptors.remove(name); + TransientSolrCoreCache transientHandler = container.getTransientCacheHandler(); + if (ret == null && transientHandler != null) { + ret = transientHandler.removeCore(name); + transientHandler.removeTransientDescriptor(name); + } + lazyDescriptors.remove(name); return ret; } } @@ -268,8 +292,8 @@ class SolrCores { synchronized (modifyLock) { SolrCore core = cores.get(name); - if (core == null) { - core = transientCores.get(name); + if (core == null && container.getTransientCacheHandler() != null) { + core = container.getTransientCacheHandler().getCore(name); } if (core != null && incRefCount) { @@ -282,7 +306,9 @@ class SolrCores { protected CoreDescriptor getDynamicDescriptor(String name) { synchronized (modifyLock) { - return dynamicDescriptors.get(name); + CoreDescriptor cd = lazyDescriptors.get(name); + if (cd != null || container.getTransientCacheHandler() == null) return cd; + return container.getTransientCacheHandler().getTransientDescriptor(name); } } @@ -295,7 +321,7 @@ class SolrCores { if (cores.containsKey(name)) { return true; } - if (transientCores.containsKey(name)) { + if (container.getTransientCacheHandler() != null && container.getTransientCacheHandler().containsCore(name)) { // Check pending for (SolrCore core : pendingCloses) { if (core.getName().equals(name)) { @@ -314,7 +340,7 @@ class SolrCores { if (cores.containsKey(name)) { return true; } - if (transientCores.containsKey(name)) { + if (container.getTransientCacheHandler() != null && container.getTransientCacheHandler().containsCore(name)) { return true; } } @@ -324,13 +350,16 @@ class SolrCores { protected CoreDescriptor getUnloadedCoreDescriptor(String cname) { synchronized (modifyLock) { - CoreDescriptor desc = dynamicDescriptors.get(cname); + CoreDescriptor desc = lazyDescriptors.get(cname); if (desc == null) { - return null; + if (container.getTransientCacheHandler() == null) return null; + desc = container.getTransientCacheHandler().getTransientDescriptor(cname); + if (desc == null) { + return null; + } } return new CoreDescriptor(cname, desc); } - } // Wait here until any pending operations (load, unload or reload) are completed on this core. @@ -412,9 +441,9 @@ class SolrCores { synchronized (modifyLock) { if (cores.containsKey(coreName)) return cores.get(coreName).getCoreDescriptor(); - if (dynamicDescriptors.containsKey(coreName)) - return dynamicDescriptors.get(coreName); - return null; + if (lazyDescriptors.containsKey(coreName) || container.getTransientCacheHandler() == null) + return lazyDescriptors.get(coreName); + return container.getTransientCacheHandler().getTransientDescriptor(coreName); } } @@ -494,4 +523,13 @@ class SolrCores { } return false; } + + // Let transient cache implementation tell us when it ages out a corel + @Override + public void update(Observable o, Object arg) { + synchronized (modifyLock) { + pendingCloses.add((SolrCore) arg); // Essentially just queue this core up for closing. + modifyLock.notifyAll(); // Wakes up closer thread too + } + } } diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java index 951d8d54ca7..b37bd521681 100644 --- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java +++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java @@ -91,6 +91,7 @@ public class SolrXmlConfig { NodeConfig.NodeConfigBuilder configBuilder = new NodeConfig.NodeConfigBuilder(nodeName, config.getResourceLoader()); configBuilder.setUpdateShardHandlerConfig(updateConfig); configBuilder.setShardHandlerFactoryConfig(getShardHandlerFactoryPluginInfo(config)); + configBuilder.setSolrCoreCacheFactoryConfig(getTransientCoreCacheFactoryPluginInfo(config)); configBuilder.setLogWatcherConfig(loadLogWatcherConfig(config, "solr/logging/*[@name]", "solr/logging/watcher/*[@name]")); configBuilder.setSolrProperties(loadProperties(config)); if (cloudConfig != null) @@ -456,5 +457,9 @@ public class SolrXmlConfig { } return configs; } + private static PluginInfo getTransientCoreCacheFactoryPluginInfo(Config config) { + Node node = config.getNode("solr/transientCoreCacheFactory", false); + return (node == null) ? null : new PluginInfo(node, "transientCoreCacheFactory", false, true); + } } diff --git a/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCache.java b/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCache.java new file mode 100644 index 00000000000..63df02b329b --- /dev/null +++ b/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCache.java @@ -0,0 +1,127 @@ +/* + * 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.core; + + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Observable; +import java.util.Set; + +import org.apache.http.annotation.Experimental; + +/** + * The base class for custom transient core maintenance. Any custom plugin that want's to take control of transient + * caches (i.e. any core defined with transient=true) should override this class. + * + * Register your plugin in solr.xml similarly to: + * + * <transientCoreCacheFactory name="transientCoreCacheFactory" class="TransientSolrCoreCacheFactoryDefault"> + * <int name="transientCacheSize">4</int> + * </transientCoreCacheFactory> + * + * + * WARNING: There is quite a bit of higher-level locking done by the CoreContainer to avoid various race conditions + * etc. You should _only_ manipulate them within the method calls designed to change them. E.g. + * only add to the transient core descriptors in addTransientDescriptor etc. + * + * Trust the higher-level code (mainly SolrCores and CoreContainer) to call the appropriate operations when + * necessary and to coordinate shutting down cores, manipulating the internal structures and the like.. + * + * The only real action you should _initiate_ is to close a core for whatever reason, and do that by + * calling notifyObservers(coreToClose); The observer will call back to removeCore(name) at the appropriate + * time. There is no need to directly remove the core _at that time_ from the transientCores list, a call + * will come back to this class when CoreContainer is closing this core. + * + * CoreDescriptors are read-once. During "core discovery" all valid descriptors are enumerated and added to + * the appropriate list. Thereafter, they are NOT re-read from disk. In those situations where you want + * to re-define the coreDescriptor, maintain a "side list" of changed core descriptors. Then override + * getTransientDescriptor to return your new core descriptor. NOTE: assuming you've already closed the + * core, the _next_ time that core is required getTransientDescriptor will be called and if you return the + * new core descriptor your re-definition should be honored. You'll have to maintain this list for the + * duration of this Solr instance running. If you persist the coreDescriptor, then next time Solr starts + * up the new definition will be read. + * + * + * If you need to manipulate the return, for instance block a core from being loaded for some period of time, override + * say getTransientDescriptor and return null. + * + * In particular, DO NOT reach into the transientCores structure from a method called to manipulate core descriptors + * or vice-versa. + */ +public abstract class TransientSolrCoreCache extends Observable { + + // Gets the core container that encloses this cache. + public abstract CoreContainer getContainer(); + + // Add the newly-opened core to the list of open cores. + public abstract SolrCore addCore(String name, SolrCore core); + + // Return the names of all possible cores, whether they are currently loaded or not. + public abstract Set getAllCoreNames(); + + // Return the names of all currently loaded cores + public abstract Set getLoadedCoreNames(); + + // Remove a core from the internal structures, presumably it + // being closed. If the core is re-opened, it will be readded by CoreContainer. + public abstract SolrCore removeCore(String name); + + // Get the core associated with the name. Return null if you don't want this core to be used. + public abstract SolrCore getCore(String name); + + // reutrn true if the cache contains the named core. + public abstract boolean containsCore(String name); + + // This method will be called when the container is to be shut down. It should return all + // transient solr cores and clear any internal structures that hold them. + public abstract Collection prepareForShutdown(); + + // These methods allow the implementation to maintain control over the core descriptors. + + // This method will only be called during core discovery at startup. + public abstract void addTransientDescriptor(String rawName, CoreDescriptor cd); + + // This method is used when opening cores and the like. If you want to change a core's descriptor, override this + // method and return the current core descriptor. + public abstract CoreDescriptor getTransientDescriptor(String name); + + + // Remove the core descriptor from your list of transient descriptors. + public abstract CoreDescriptor removeTransientDescriptor(String name); + + // Find all the names a specific core is mapped to. Should not return null, return empty set instead. + @Experimental + public List getNamesForCore(SolrCore core) { + return Collections.emptyList(); + } + + /** + * Must be called in order to free resources! + */ + public abstract void close(); + + + // These two methods allow custom implementations to communicate arbitrary information as necessary. + public abstract int getStatus(String coreName); + public abstract void setStatus(String coreName, int status); +} + + + diff --git a/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheDefault.java b/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheDefault.java new file mode 100644 index 00000000000..e1fd748b21c --- /dev/null +++ b/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheDefault.java @@ -0,0 +1,198 @@ +/* + * 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.core; + +import java.lang.invoke.MethodHandles; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Observer; +import java.util.Set; + +import org.apache.solr.common.util.NamedList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TransientSolrCoreCacheDefault extends TransientSolrCoreCache { + + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private int cacheSize = NodeConfig.NodeConfigBuilder.DEFAULT_TRANSIENT_CACHE_SIZE; + + protected Observer observer; + protected CoreContainer coreContainer; + + protected final Map transientDescriptors = new LinkedHashMap<>(); + + //WARNING! The _only_ place you put anything into the list of transient cores is with the putTransientCore method! + protected Map transientCores = new LinkedHashMap<>(); // For "lazily loaded" cores + + /** + * @param container The enclosing CoreContainer. It allows us to access everything we need. + */ + public TransientSolrCoreCacheDefault(final CoreContainer container) { + this.coreContainer = container; + this.observer= coreContainer.solrCores; + + NodeConfig cfg = container.getNodeConfig(); + if (cfg.getTransientCachePluginInfo() == null) { + // Still handle just having transientCacheSize defined in the body of solr.xml not in a transient handler clause. + // deprecate this for 7.0? + this.cacheSize = cfg.getTransientCacheSize(); + } else { + NamedList args = cfg.getTransientCachePluginInfo().initArgs; + Object obj = args.get("transientCacheSize"); + if (obj != null) { + this.cacheSize = (int) obj; + } + } + doInit(); + } + // This just moves the + private void doInit() { + NodeConfig cfg = coreContainer.getNodeConfig(); + if (cfg.getTransientCachePluginInfo() == null) { + // Still handle just having transientCacheSize defined in the body of solr.xml not in a transient handler clause. + this.cacheSize = cfg.getTransientCacheSize(); + } else { + NamedList args = cfg.getTransientCachePluginInfo().initArgs; + Object obj = args.get("transientCacheSize"); + if (obj != null) { + this.cacheSize = (int) obj; + } + } + + log.info("Allocating transient cache for {} transient cores", cacheSize); + addObserver(this.observer); + // it's possible for cache + if (cacheSize < 0) { // Trap old flag + cacheSize = Integer.MAX_VALUE; + } + // Now don't allow ridiculous allocations here, if the size is > 1,000, we'll just deal with + // adding cores as they're opened. This blows up with the marker value of -1. + transientCores = new LinkedHashMap(Math.min(cacheSize, 1000), 0.75f, true) { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + if (size() > cacheSize) { + SolrCore coreToClose = eldest.getValue(); + setChanged(); + notifyObservers(coreToClose); + log.info("Closing transient core [{}]", coreToClose.getName()); + return true; + } + return false; + } + }; + } + + + @Override + public Collection prepareForShutdown() { + // Returna copy of the values + List ret = new ArrayList(transientCores.values()); + transientCores.clear(); + return ret; + } + + @Override + public CoreContainer getContainer() { return this.coreContainer; } + + @Override + public SolrCore addCore(String name, SolrCore core) { + return transientCores.put(name, core); + } + + @Override + public Set getAllCoreNames() { + return transientDescriptors.keySet(); + } + + @Override + public Set getLoadedCoreNames() { + return transientCores.keySet(); + } + + // Remove a core from the internal structures, presumably it + // being closed. If the core is re-opened, it will be readded by CoreContainer. + @Override + public SolrCore removeCore(String name) { + return transientCores.remove(name); + } + + // Get the core associated with the name. Return null if you don't want this core to be used. + @Override + public SolrCore getCore(String name) { + return transientCores.get(name); + } + + @Override + public boolean containsCore(String name) { + return transientCores.containsKey(name); + } + + // These methods allow the implementation to maintain control over the core descriptors. + + + // This method will only be called during core discovery at startup. + @Override + public void addTransientDescriptor(String rawName, CoreDescriptor cd) { + transientDescriptors.put(rawName, cd); + } + + // This method is used when opening cores and the like. If you want to change a core's descriptor, override this + // method and return the current core descriptor. + @Override + public CoreDescriptor getTransientDescriptor(String name) { + return transientDescriptors.get(name); + } + + @Override + public CoreDescriptor removeTransientDescriptor(String name) { + return transientDescriptors.remove(name); + } + + @Override + public List getNamesForCore(SolrCore core) { + List ret = new ArrayList<>(); + for (Map.Entry entry : transientCores.entrySet()) { + if (core == entry.getValue()) { + ret.add(entry.getKey()); + } + } + return ret; + } + + /** + * Must be called in order to free resources! + */ + @Override + public void close() { + deleteObserver(this.observer); + } + + + // For custom implementations to communicate arbitrary information as necessary. + @Override + public int getStatus(String coreName) { return 0; } //no_op for default handler. + + @Override + public void setStatus(String coreName, int status) {} //no_op for default handler. + +} diff --git a/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheFactory.java b/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheFactory.java new file mode 100644 index 00000000000..b3b8cf0bb22 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheFactory.java @@ -0,0 +1,85 @@ +/* + * 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.core; + +import java.lang.invoke.MethodHandles; +import java.util.Collections; +import java.util.Locale; + +import com.google.common.collect.ImmutableMap; +import org.apache.solr.util.plugin.PluginInfoInitialized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An interface that allows custom transient caches to be maintained with different implementations + */ +public abstract class TransientSolrCoreCacheFactory { + private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + + private CoreContainer coreContainer = null; + + public abstract TransientSolrCoreCache getTransientSolrCoreCache(); + /** + * Create a new TransientSolrCoreCacheFactory instance + * + * @param loader a SolrResourceLoader used to find the TransientSolrCacheFactory classes + * @param coreContainer CoreContainer that encloses all the Solr cores. + * @return a new, initialized TransientSolrCoreCache instance + */ + + public static TransientSolrCoreCacheFactory newInstance(SolrResourceLoader loader, CoreContainer coreContainer) { + PluginInfo info = coreContainer.getConfig().getTransientCachePluginInfo(); + if (info == null) { // definition not in our solr.xml file, use default + info = DEFAULT_TRANSIENT_SOLR_CACHE_INFO; + } + + try { + // According to the docs, this returns a TransientSolrCoreCacheFactory with the default c'tor + TransientSolrCoreCacheFactory tccf = loader.findClass(info.className, TransientSolrCoreCacheFactory.class).newInstance(); + + // OK, now we call it's init method. + if (PluginInfoInitialized.class.isAssignableFrom(tccf.getClass())) + PluginInfoInitialized.class.cast(tccf).init(info); + tccf.setCoreContainer(coreContainer); + return tccf; + } catch (Exception e) { + // Many things could cuse this, bad solrconfig, mis-typed class name, whatever. However, this should not + // keep the enclosing coreContainer from instantiating, so log an error and continue. + log.error(String.format(Locale.ROOT, "Error instantiating TransientSolrCoreCacheFactory class [%s]: %s", + info.className, e.getMessage())); + return null; + } + + } + public static final PluginInfo DEFAULT_TRANSIENT_SOLR_CACHE_INFO = + new PluginInfo("transientSolrCoreCacheFactory", + ImmutableMap.of("class", TransientSolrCoreCacheFactoryDefault.class.getName(), + "name", TransientSolrCoreCacheFactory.class.getName()), + null, Collections.emptyList()); + + + // Need this because the plugin framework doesn't require a PluginINfo in the init method, don't see a way to + // pass additional parameters and we need this when we create the transient core cache, it's _really_ important. + public void setCoreContainer(CoreContainer coreContainer) { + this.coreContainer = coreContainer; + } + + public CoreContainer getCoreContainer() { + return coreContainer; + } +} diff --git a/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheFactoryDefault.java b/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheFactoryDefault.java new file mode 100644 index 00000000000..722ab9c76f4 --- /dev/null +++ b/solr/core/src/java/org/apache/solr/core/TransientSolrCoreCacheFactoryDefault.java @@ -0,0 +1,31 @@ +/* + * 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.core; + +public class TransientSolrCoreCacheFactoryDefault extends TransientSolrCoreCacheFactory { + + TransientSolrCoreCache transientSolrCoreCache = null; + + @Override + public TransientSolrCoreCache getTransientSolrCoreCache() { + if (transientSolrCoreCache == null) { + transientSolrCoreCache = new TransientSolrCoreCacheDefault(getCoreContainer()); + } + + return transientSolrCoreCache; + } +} diff --git a/solr/core/src/test-files/solr/solr.xml b/solr/core/src/test-files/solr/solr.xml index f381475ab1a..526dffa7fe7 100644 --- a/solr/core/src/test-files/solr/solr.xml +++ b/solr/core/src/test-files/solr/solr.xml @@ -31,6 +31,11 @@ ${connTimeout:15000} + + 4 + + + 127.0.0.1 ${hostPort:8983} diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java index d9257747584..d05cec9a8f0 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java @@ -29,6 +29,8 @@ import org.apache.solr.common.util.Utils; import org.apache.solr.core.CloudConfig; import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreDescriptor; +import org.apache.solr.core.SolrXmlConfig; +import org.apache.solr.core.TransientSolrCoreCache; import org.apache.solr.handler.admin.CoreAdminHandler; import org.apache.solr.handler.component.HttpShardHandlerFactory; import org.apache.solr.update.UpdateShardHandler; @@ -327,7 +329,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 { private static class MockCoreContainer extends CoreContainer { UpdateShardHandler updateShardHandler = new UpdateShardHandler(UpdateShardHandlerConfig.DEFAULT); public MockCoreContainer() { - super((Object)null); + super(SolrXmlConfig.fromString(null, "")); this.shardHandlerFactory = new HttpShardHandlerFactory(); this.coreAdminHandler = new CoreAdminHandler(); } @@ -345,6 +347,11 @@ public class ZkControllerTest extends SolrTestCaseJ4 { updateShardHandler.close(); super.shutdown(); } + + @Override + public TransientSolrCoreCache getTransientCacheHandler() { + return transientSolrCoreCache; + } } } diff --git a/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java b/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java index 65d459ae934..22020baf352 100644 --- a/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java +++ b/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java @@ -60,12 +60,12 @@ public class TestCoreDiscovery extends SolrTestCaseJ4 { setMeUp(null); } - private Properties makeCorePropFile(String name, boolean isLazy, boolean loadOnStartup, String... extraProps) { + private Properties makeCorePropFile(String name, boolean isTransient, boolean loadOnStartup, String... extraProps) { Properties props = new Properties(); props.put(CoreDescriptor.CORE_NAME, name); props.put(CoreDescriptor.CORE_SCHEMA, "schema-tiny.xml"); props.put(CoreDescriptor.CORE_CONFIG, "solrconfig-minimal.xml"); - props.put(CoreDescriptor.CORE_TRANSIENT, Boolean.toString(isLazy)); + props.put(CoreDescriptor.CORE_TRANSIENT, Boolean.toString(isTransient)); props.put(CoreDescriptor.CORE_LOADONSTARTUP, Boolean.toString(loadOnStartup)); props.put(CoreDescriptor.CORE_DATADIR, "${core.dataDir:stuffandnonsense}"); @@ -140,7 +140,7 @@ public class TestCoreDiscovery extends SolrTestCaseJ4 { try { TestLazyCores.checkInCores(cc, "core1"); - TestLazyCores.checkNotInCores(cc, "lazy1", "core2", "collection1"); + TestLazyCores.checkNotInCores(cc, "lazy1", "core2"); // force loading of core2 and lazy1 by getting them from the CoreContainer try (SolrCore core1 = cc.getCore("core1"); @@ -463,4 +463,5 @@ public class TestCoreDiscovery extends SolrTestCaseJ4 { NodeConfig absConfig = SolrXmlConfig.fromString(loader, "/absolute"); assertThat(absConfig.getCoreRootDirectory().toString(), not(containsString(solrHomeDirectory.getAbsolutePath()))); } + } diff --git a/solr/core/src/test/org/apache/solr/core/TestLazyCores.java b/solr/core/src/test/org/apache/solr/core/TestLazyCores.java index 0c0845bb07a..8690e27bdec 100644 --- a/solr/core/src/test/org/apache/solr/core/TestLazyCores.java +++ b/solr/core/src/test/org/apache/solr/core/TestLazyCores.java @@ -38,6 +38,7 @@ import org.apache.solr.handler.admin.CoreAdminHandler; import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.response.SolrQueryResponse; +import org.apache.solr.servlet.SolrDispatchFilter; import org.apache.solr.update.AddUpdateCommand; import org.apache.solr.update.CommitUpdateCommand; import org.apache.solr.update.UpdateHandler; @@ -83,13 +84,13 @@ public class TestLazyCores extends SolrTestCaseJ4 { private CoreContainer init() throws Exception { solrHomeDirectory = createTempDir().toFile(); + copyXmlToHome(solrHomeDirectory.getAbsoluteFile(), "solr.xml"); for (int idx = 1; idx < 10; ++idx) { copyMinConf(new File(solrHomeDirectory, "collection" + idx)); } - SolrResourceLoader loader = new SolrResourceLoader(solrHomeDirectory.toPath()); - NodeConfig config = new NodeConfig.NodeConfigBuilder("testNode", loader).setTransientCacheSize(4).build(); - return createCoreContainer(config, testCores); + NodeConfig cfg = SolrDispatchFilter.loadNodeConfig(solrHomeDirectory.toPath(), null); + return createCoreContainer(cfg, testCores); } @Test @@ -188,7 +189,7 @@ public class TestLazyCores extends SolrTestCaseJ4 { , "//result[@numFound='0']" ); - checkInCores(cc, "collection4"); + checkInCores(cc, "collection1", "collection2", "collection4", "collection5"); core4.close(); collection1.close(); @@ -454,11 +455,14 @@ public class TestLazyCores extends SolrTestCaseJ4 { // 1> produce errors as appropriate when the config or schema files are foo'd // 2> "self heal". That is, if the problem is corrected can the core be reloaded and used? // 3> that OK cores can be searched even when some cores failed to load. + // 4> that having no solr.xml entry for transient chache handler correctly uses the default. @Test public void testBadConfigsGenerateErrors() throws Exception { final CoreContainer cc = initGoodAndBad(Arrays.asList("core1", "core2"), Arrays.asList("badSchema1", "badSchema2"), Arrays.asList("badConfig1", "badConfig2")); + + try { // first, did the two good cores load successfully? checkInCores(cc, "core1", "core2"); @@ -491,8 +495,9 @@ public class TestLazyCores extends SolrTestCaseJ4 { copyGoodConf("badSchema1", "schema-tiny.xml", "schema.xml"); copyGoodConf("badSchema2", "schema-tiny.xml", "schema.xml"); + // This should force a reload of the cores. - SolrCore bc1 = cc.getCore("badConfig1"); + SolrCore bc1 = cc.getCore("badConfig1");; SolrCore bc2 = cc.getCore("badConfig2"); SolrCore bs1 = cc.getCore("badSchema1"); SolrCore bs2 = cc.getCore("badSchema2"); @@ -635,16 +640,46 @@ public class TestLazyCores extends SolrTestCaseJ4 { } public static void checkNotInCores(CoreContainer cc, String... nameCheck) { - Collection names = cc.getCoreNames(); + Collection loadedNames = cc.getCoreNames(); for (String name : nameCheck) { - assertFalse("core " + name + " was found in the list of cores", names.contains(name)); + assertFalse("core " + name + " was found in the list of cores", loadedNames.contains(name)); + } + + // There was a problem at one point exacerbated by the poor naming conventions. So parallel to loaded cores, there + // should be the ability to get the core _names_ that are loaded as well as all the core names _possible_ + // + // the names above should only contain loaded core names. Every name in names should be in allNames, but none of + // the names in nameCheck should be loaded and thus should not be in names. + + Collection allNames = cc.getAllCoreNames(); + // Every core, loaded or not should be in the accumulated coredescriptors: + List descriptors = cc.getCoreDescriptors(); + + assertEquals("There should be as many coreDescriptors as coreNames", allNames.size(), descriptors.size()); + for (CoreDescriptor desc : descriptors) { + assertTrue("Name should have a corresponding descriptor", allNames.contains(desc.getName())); + } + + // First check that all loaded cores are in allNames. + for (String name : loadedNames) { + assertTrue("Loaded core " + name + " should have been found in the list of all possible core names", + allNames.contains(name)); + } + + for (String name : nameCheck) { + assertTrue("Not-currently-loaded core " + name + " should have been found in the list of all possible core names", + allNames.contains(name)); } } public static void checkInCores(CoreContainer cc, String... nameCheck) { - Collection names = cc.getCoreNames(); + Collection loadedNames = cc.getCoreNames(); + + assertEquals("There whould be exactly as many loaded cores as loaded names returned. ", + loadedNames.size(), nameCheck.length); + for (String name : nameCheck) { - assertTrue("core " + name + " was not found in the list of cores", names.contains(name)); + assertTrue("core " + name + " was not found in the list of cores", loadedNames.contains(name)); } } diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java index be8e96db41e..faf67074f46 100644 --- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java +++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java @@ -2002,8 +2002,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase { FileUtils.copyFile(new File(top, "solrconfig.snippet.randomindexconfig.xml"), new File(subHome, "solrconfig.snippet.randomindexconfig.xml")); } - // Creates minimal full setup, including the old solr.xml file that used to be hard coded in ConfigSolrXmlOld - // TODO: remove for 5.0 + // Creates minimal full setup, including solr.xml public static void copyMinFullSetup(File dstRoot) throws IOException { if (! dstRoot.exists()) { assertTrue("Failed to make subdirectory ", dstRoot.mkdirs()); @@ -2013,6 +2012,15 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase { copyMinConf(dstRoot); } + // Just copies the file indicated to the tmp home directory naming it "solr.xml" + public static void copyXmlToHome(File dstRoot, String fromFile) throws IOException { + if (! dstRoot.exists()) { + assertTrue("Failed to make subdirectory ", dstRoot.mkdirs()); + } + File xmlF = new File(SolrTestCaseJ4.TEST_HOME(), fromFile); + FileUtils.copyFile(xmlF, new File(dstRoot, "solr.xml")); + + } // Creates a consistent configuration, _including_ solr.xml at dstRoot. Creates collection1/conf and copies // the stock files in there. @@ -2020,7 +2028,6 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase { if (!dstRoot.exists()) { assertTrue("Failed to make subdirectory ", dstRoot.mkdirs()); } - FileUtils.copyFile(new File(SolrTestCaseJ4.TEST_HOME(), "solr.xml"), new File(dstRoot, "solr.xml")); File subHome = new File(dstRoot, collection + File.separator + "conf"); From 65b4530fb39842418eae8d2acb7c463182039083 Mon Sep 17 00:00:00 2001 From: Mikhail Khludnev Date: Mon, 3 Apr 2017 23:45:54 +0300 Subject: [PATCH 109/563] SOLR-9745: fix solr.cmd to print errors from invoked script --- solr/CHANGES.txt | 2 + .../java/org/apache/solr/util/SolrCLI.java | 18 ++++++--- .../solr/util/TestSolrCLIRunExample.java | 38 +++++++++++++++++++ 3 files changed, 53 insertions(+), 5 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 6fe4cc0bf3e..c1a750313da 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -190,6 +190,8 @@ Other Changes * SOLR-8906: Make transient core cache pluggable (Erick Erickson) +* SOLR-9745: print errors from solr.cmd (Gopikannan Venugopalsamy via Mikhail Khludnev) + ================== 6.5.1 ================== Bug Fixes 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 6a85422185d..da7e63ee506 100644 --- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java +++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java @@ -62,6 +62,7 @@ import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; import org.apache.commons.exec.DefaultExecuteResultHandler; import org.apache.commons.exec.DefaultExecutor; +import org.apache.commons.exec.ExecuteException; import org.apache.commons.exec.Executor; import org.apache.commons.exec.OS; import org.apache.commons.exec.environment.EnvironmentUtils; @@ -2928,18 +2929,25 @@ public class SolrCLI { } } } - executor.execute(org.apache.commons.exec.CommandLine.parse(startCmd), startEnv, new DefaultExecuteResultHandler()); + DefaultExecuteResultHandler handler = new DefaultExecuteResultHandler(); + executor.execute(org.apache.commons.exec.CommandLine.parse(startCmd), startEnv, handler); - // brief wait before proceeding on Windows + // wait for execution. try { - Thread.sleep(3000); + handler.waitFor(); } catch (InterruptedException ie) { // safe to ignore ... Thread.interrupted(); } - + if (handler.getExitValue() != 0) { + throw new Exception("Failed to start Solr using command: "+startCmd+" Exception : "+handler.getException()); + } } else { - code = executor.execute(org.apache.commons.exec.CommandLine.parse(startCmd)); + try { + code = executor.execute(org.apache.commons.exec.CommandLine.parse(startCmd)); + } catch(ExecuteException e){ + throw new Exception("Failed to start Solr using command: "+startCmd+" Exception : "+ e); + } } if (code != 0) throw new Exception("Failed to start Solr using command: "+startCmd); diff --git a/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java b/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java index 7980560f899..02d91b03503 100644 --- a/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java +++ b/solr/core/src/test/org/apache/solr/util/TestSolrCLIRunExample.java @@ -482,4 +482,42 @@ public class TestSolrCLIRunExample extends SolrTestCaseJ4 { // stop the test instance executor.execute(org.apache.commons.exec.CommandLine.parse("bin/solr stop -p "+bindPort)); } + + @Test + public void testFailExecuteScript() throws Exception { + File solrHomeDir = new File(ExternalPaths.SERVER_HOME); + if (!solrHomeDir.isDirectory()) + fail(solrHomeDir.getAbsolutePath()+" not found and is required to run this test!"); + + Path tmpDir = createTempDir(); + File solrExampleDir = tmpDir.toFile(); + File solrServerDir = solrHomeDir.getParentFile(); + + // need a port to start the example server on + int bindPort = -1; + try (ServerSocket socket = new ServerSocket(0)) { + bindPort = socket.getLocalPort(); + } + + File toExecute = new File(tmpDir.toString(), "failExecuteScript"); + assertTrue("Should have been able to create file '" + toExecute.getAbsolutePath() + "' ", toExecute.createNewFile()); + + String[] toolArgs = new String[] { + "-e", "techproducts", + "-serverDir", solrServerDir.getAbsolutePath(), + "-exampleDir", solrExampleDir.getAbsolutePath(), + "-p", String.valueOf(bindPort), + "-script", toExecute.getAbsolutePath().toString() + }; + + // capture tool output to stdout + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintStream stdoutSim = new PrintStream(baos, true, StandardCharsets.UTF_8.name()); + + DefaultExecutor executor = new DefaultExecutor(); + + SolrCLI.RunExampleTool tool = new SolrCLI.RunExampleTool(executor, System.in, stdoutSim); + int code = tool.runTool(SolrCLI.processCommandLineArgs(SolrCLI.joinCommonAndToolOptions(tool.getOptions()), toolArgs)); + assertTrue("Execution should have failed with return code 1", code == 1); + } } From e872dc7913036c81b9ef48cf35c3456321b758b7 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Mon, 3 Apr 2017 20:39:37 -0400 Subject: [PATCH 110/563] SOLR-10351: Add try-with-resources clause around TokenStream --- .../apache/solr/handler/AnalyzeEvaluator.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/handler/AnalyzeEvaluator.java b/solr/core/src/java/org/apache/solr/handler/AnalyzeEvaluator.java index 207f4046590..485f9c39906 100644 --- a/solr/core/src/java/org/apache/solr/handler/AnalyzeEvaluator.java +++ b/solr/core/src/java/org/apache/solr/handler/AnalyzeEvaluator.java @@ -81,17 +81,16 @@ public class AnalyzeEvaluator extends SimpleEvaluator { return null; } - TokenStream tokenStream = analyzer.tokenStream(analyzerField, value); - CharTermAttribute termAtt = tokenStream.getAttribute(CharTermAttribute.class); - tokenStream.reset(); List tokens = new ArrayList(); - while (tokenStream.incrementToken()) { - tokens.add(termAtt.toString()); + + try(TokenStream tokenStream = analyzer.tokenStream(analyzerField, value)) { + CharTermAttribute termAtt = tokenStream.getAttribute(CharTermAttribute.class); + tokenStream.reset(); + while (tokenStream.incrementToken()) { + tokens.add(termAtt.toString()); + } + tokenStream.end(); } - - tokenStream.end(); - tokenStream.close(); - return tokens; } From 0bf7a5ff335d2825a014862f5d04e70a7f5c016e Mon Sep 17 00:00:00 2001 From: Mark Miller Date: Mon, 3 Apr 2017 22:00:08 -0300 Subject: [PATCH 111/563] SOLR-10338: Revert configure SecureRandom non blocking for tests. (reverted from commit 0445f8200e0630e1bb8b7117f200529ed1259747) --- dev-tools/idea/.idea/workspace.xml | 2 +- dev-tools/maven/pom.xml.template | 1 - lucene/common-build.xml | 3 -- solr/CHANGES.txt | 2 - .../test/SecureRandomAlgorithmTesterApp.java | 41 ------------------- .../java/org/apache/solr/SolrTestCaseJ4.java | 9 ---- 6 files changed, 1 insertion(+), 57 deletions(-) delete mode 100644 solr/core/src/test/SecureRandomAlgorithmTesterApp.java diff --git a/dev-tools/idea/.idea/workspace.xml b/dev-tools/idea/.idea/workspace.xml index e22108f04b4..0ca7f0c2b70 100644 --- a/dev-tools/idea/.idea/workspace.xml +++ b/dev-tools/idea/.idea/workspace.xml @@ -2,7 +2,7 @@ - diff --git a/dev-tools/maven/pom.xml.template b/dev-tools/maven/pom.xml.template index 6b7f915e22d..cd8d6b8db77 100644 --- a/dev-tools/maven/pom.xml.template +++ b/dev-tools/maven/pom.xml.template @@ -277,7 +277,6 @@ ${tests.postingsformat} ${tests.timezone} ${tests.verbose} - file:/dev/./urandom diff --git a/lucene/common-build.xml b/lucene/common-build.xml index aee7899d1f6..327a01d8486 100644 --- a/lucene/common-build.xml +++ b/lucene/common-build.xml @@ -1062,9 +1062,6 @@ - - - diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index c1a750313da..8a5c3e8d206 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -180,8 +180,6 @@ Other Changes * SOLR-10147: Admin UI -> Cloud -> Graph: Impossible to see shard state (Amrit Sarkar, janhoy) -* SOLR-10338: Configure SecureRandom non blocking for tests. (Mihaly Toth, hossman, Ishan Chattopadhyaya, via Mark Miller) - * SOLR-10399: Generalize some internal facet logic to simplify points/non-points field handling (Adrien Grand, hossman) * SOLR-7383: New DataImportHandler 'atom' example, replacing broken 'rss' example (Alexandre Rafalovitch) diff --git a/solr/core/src/test/SecureRandomAlgorithmTesterApp.java b/solr/core/src/test/SecureRandomAlgorithmTesterApp.java deleted file mode 100644 index 44f79e903c8..00000000000 --- a/solr/core/src/test/SecureRandomAlgorithmTesterApp.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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. - */ - -import java.security.NoSuchAlgorithmException; -import java.security.SecureRandom; - -public class SecureRandomAlgorithmTesterApp { - public static void main(String[] args) throws NoSuchAlgorithmException { - String algorithm = args[0]; - String method = args[1]; - int amount = Integer.valueOf(args[2]); - SecureRandom secureRandom; - if(algorithm.equals("default")) - secureRandom = new SecureRandom(); - else - secureRandom = SecureRandom.getInstance(algorithm); - System.out.println("Algorithm:" + secureRandom.getAlgorithm()); - switch(method) { - case "seed": secureRandom.generateSeed(amount); break; - case "bytes": secureRandom.nextBytes(new byte[amount]); break; - case "long": secureRandom.nextLong(); break; - case "int": secureRandom.nextInt(); break; - default: throw new IllegalArgumentException("Not supported random function: " + method); - } - System.out.println("SecureRandom function invoked"); - } -} diff --git a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java index faf67074f46..0d4cedd4422 100644 --- a/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java +++ b/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java @@ -39,7 +39,6 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; -import java.security.SecureRandom; import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; @@ -2439,14 +2438,6 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase { } } - @BeforeClass - public static void assertNonBlockingRandomGeneratorAvailable() { - if(Boolean.parseBoolean(System.getProperty("test.solr.allow.any.securerandom","false"))) - return; - // Use -Djava.security.egd=file:/dev/./urandom VM option if you hit this - assertEquals("SHA1PRNG", new SecureRandom().getAlgorithm()); - } - @AfterClass public static void unchooseMPForMP() { System.clearProperty(SYSTEM_PROPERTY_SOLR_TESTS_USEMERGEPOLICYFACTORY); From 3f172a019b21c0dafaa7a18e0ccd0e99b1c5f3bd Mon Sep 17 00:00:00 2001 From: Mikhail Khludnev Date: Tue, 4 Apr 2017 08:42:31 +0300 Subject: [PATCH 112/563] SOLR-9745: bring back timeout value to fix tests --- solr/core/src/java/org/apache/solr/util/SolrCLI.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 da7e63ee506..3572ea43ca6 100644 --- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java +++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java @@ -2934,7 +2934,7 @@ public class SolrCLI { // wait for execution. try { - handler.waitFor(); + handler.waitFor(3000); } catch (InterruptedException ie) { // safe to ignore ... Thread.interrupted(); From 23b002a0fdf2f6025f1eb026c0afca247fb21ed0 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 30 Mar 2017 09:12:45 +0200 Subject: [PATCH 113/563] LUCENE-7756: Only record the major Lucene version that created the index, and record the minimum Lucene version that contributed to segments. --- lucene/CHANGES.txt | 7 +- .../lucene50/Lucene50SegmentInfoFormat.java | 2 +- .../lucene/codecs/lucene62/Lucene62Codec.java | 2 +- .../apache/lucene/index/FixBrokenOffsets.java | 3 + .../lucene50/Lucene50RWSegmentInfoFormat.java | 2 +- .../TestLucene50SegmentInfoFormat.java | 10 + .../lucene53/TestLucene53NormsFormat.java | 6 + .../codecs/lucene62/Lucene62RWCodec.java | 12 + .../lucene62/Lucene62RWSegmentInfoFormat.java | 193 ++++++++ .../TestLucene62SegmentInfoFormat.java | 15 +- .../index/TestBackwardsCompatibility.java | 69 ++- .../lucene/index/TestFixBrokenOffsets.java | 10 +- .../index/TestIndexWriterOnOldIndex.java | 6 +- .../SimpleTextSegmentInfoFormat.java | 29 +- .../lucene62/Lucene62SegmentInfoFormat.java | 152 +----- .../lucene/codecs/lucene70/Lucene70Codec.java | 3 +- .../lucene70/Lucene70SegmentInfoFormat.java | 439 ++++++++++++++++++ .../org/apache/lucene/index/CheckIndex.java | 2 +- .../index/DocumentsWriterPerThread.java | 2 +- .../lucene/index/FilterCodecReader.java | 5 +- .../apache/lucene/index/FilterLeafReader.java | 5 +- .../org/apache/lucene/index/IndexWriter.java | 54 ++- .../org/apache/lucene/index/LeafMetaData.java | 74 +++ .../org/apache/lucene/index/LeafReader.java | 7 +- .../lucene/index/MergeReaderWrapper.java | 5 +- .../org/apache/lucene/index/MergeState.java | 2 +- .../lucene/index/ParallelLeafReader.java | 36 +- .../lucene/index/ReadersAndUpdates.java | 4 +- .../org/apache/lucene/index/SegmentInfo.java | 19 +- .../org/apache/lucene/index/SegmentInfos.java | 77 ++- .../apache/lucene/index/SegmentMerger.java | 14 + .../apache/lucene/index/SegmentReader.java | 10 +- .../lucene/index/SlowCodecReaderWrapper.java | 5 +- .../lucene/index/StandardDirectoryReader.java | 4 +- .../EarlyTerminatingSortingCollector.java | 2 +- .../TestLucene70SegmentInfoFormat.java | 35 ++ .../org/apache/lucene/index/TestCodecs.java | 4 +- .../index/TestDemoParallelLeafReader.java | 3 +- .../test/org/apache/lucene/index/TestDoc.java | 8 +- .../lucene/index/TestDocumentWriter.java | 9 +- .../apache/lucene/index/TestIndexSorting.java | 2 +- .../apache/lucene/index/TestIndexWriter.java | 2 +- .../TestIndexWriterThreadsToSegments.java | 3 +- .../TestOneMergeWrappingMergePolicy.java | 1 + .../apache/lucene/index/TestSegmentInfos.java | 22 +- .../lucene/index/TestSegmentMerger.java | 7 +- .../lucene/index/TestSegmentReader.java | 3 +- .../lucene/index/TestSegmentTermDocs.java | 7 +- .../highlight/TermVectorLeafReader.java | 7 +- .../lucene/index/memory/MemoryIndex.java | 6 +- .../apache/lucene/index/IndexSplitter.java | 4 +- .../lucene/replicator/nrt/ReplicaNode.java | 2 +- .../index/BaseCompoundFormatTestCase.java | 3 +- .../index/BaseFieldInfoFormatTestCase.java | 3 +- .../index/BaseIndexFileFormatTestCase.java | 28 +- .../lucene/index/BaseNormsFormatTestCase.java | 6 +- .../index/BaseSegmentInfoFormatTestCase.java | 54 ++- .../lucene/index/RandomPostingsTester.java | 2 +- .../org/apache/lucene/search/QueryUtils.java | 6 +- .../index/SlowCompositeReaderWrapper.java | 19 +- .../org/apache/solr/search/TestDocSet.java | 7 +- 61 files changed, 1172 insertions(+), 368 deletions(-) create mode 100644 lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWSegmentInfoFormat.java rename lucene/{core => backward-codecs}/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java (82%) create mode 100644 lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java create mode 100644 lucene/core/src/java/org/apache/lucene/index/LeafMetaData.java create mode 100644 lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70SegmentInfoFormat.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 83113a8238e..1f3f30c79e4 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -7,9 +7,12 @@ http://s.apache.org/luceneversions New Features -* LUCENE-7703: SegmentInfos now record the Lucene version at index creation - time. (Adrien Grand) +* LUCENE-7703: SegmentInfos now record the major Lucene version at index + creation time. (Adrien Grand) +* LUCENE-7756: LeafReader.getMetaData now exposes the index created version as + well as the oldest Lucene version that contributed to the segment. + (Adrien Grand) API Changes diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java index 69cda34494f..d2a384e975f 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java @@ -65,7 +65,7 @@ public class Lucene50SegmentInfoFormat extends SegmentInfoFormat { final Set files = input.readSetOfStrings(); final Map attributes = input.readMapOfStrings(); - si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null); + si = new SegmentInfo(dir, version, null, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null); si.setFiles(files); } catch (Throwable exception) { priorE = exception; diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java index 58b07ebe73e..3dd7daa3aa0 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/codecs/lucene62/Lucene62Codec.java @@ -114,7 +114,7 @@ public class Lucene62Codec extends Codec { } @Override - public final SegmentInfoFormat segmentInfoFormat() { + public SegmentInfoFormat segmentInfoFormat() { return segmentInfosFormat; } diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java b/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java index e775a282dea..9b3615eb37a 100644 --- a/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java +++ b/lucene/backward-codecs/src/java/org/apache/lucene/index/FixBrokenOffsets.java @@ -128,6 +128,9 @@ public class FixBrokenOffsets { } Directory destDir = FSDirectory.open(destPath); + // We need to maintain the same major version + int createdMajor = SegmentInfos.readLatestCommit(srcDir).getIndexCreatedVersionMajor(); + new SegmentInfos(createdMajor).commit(destDir); IndexWriter writer = new IndexWriter(destDir, new IndexWriterConfig()); writer.addIndexes(filtered); IOUtils.close(writer, reader, srcDir, destDir); diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java index 965ee964626..4bed31106dc 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/Lucene50RWSegmentInfoFormat.java @@ -65,7 +65,7 @@ public class Lucene50RWSegmentInfoFormat extends Lucene50SegmentInfoFormat { final Set files = input.readSetOfStrings(); final Map attributes = input.readMapOfStrings(); - si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null); + si = new SegmentInfo(dir, version, null, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, null); si.setFiles(files); } catch (Throwable exception) { priorE = exception; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java index 688afed23c4..0a9bf799dba 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene50/TestLucene50SegmentInfoFormat.java @@ -28,6 +28,11 @@ public class TestLucene50SegmentInfoFormat extends BaseSegmentInfoFormatTestCase return new Lucene60RWCodec(); } + @Override + protected int getCreatedVersionMajor() { + return Version.LUCENE_6_0_0.major; + } + @Override protected Version[] getVersions() { return new Version[] { Version.LUCENE_6_0_0 }; @@ -37,4 +42,9 @@ public class TestLucene50SegmentInfoFormat extends BaseSegmentInfoFormatTestCase protected boolean supportsIndexSort() { return false; } + + @Override + protected boolean supportsMinVersion() { + return false; + } } diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java index 80a8eee6269..7d37b45e56d 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene53/TestLucene53NormsFormat.java @@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene53; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.lucene62.Lucene62RWCodec; import org.apache.lucene.index.BaseNormsFormatTestCase; +import org.apache.lucene.util.Version; /** * Tests Lucene53NormsFormat @@ -26,6 +27,11 @@ import org.apache.lucene.index.BaseNormsFormatTestCase; public class TestLucene53NormsFormat extends BaseNormsFormatTestCase { private final Codec codec = new Lucene62RWCodec(); + @Override + protected int getCreatedVersionMajor() { + return Version.LUCENE_6_2_0.major; + } + @Override protected Codec getCodec() { return codec; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java index fcb414def03..34d3a7f0eae 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWCodec.java @@ -17,13 +17,25 @@ package org.apache.lucene.codecs.lucene62; import org.apache.lucene.codecs.NormsFormat; +import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.lucene53.Lucene53RWNormsFormat; import org.apache.lucene.codecs.lucene62.Lucene62Codec; +/** + * Read-write version of 6.2 codec for testing + * @deprecated for test purposes only + */ +@Deprecated public class Lucene62RWCodec extends Lucene62Codec { + private final SegmentInfoFormat segmentInfoFormat = new Lucene62RWSegmentInfoFormat(); private final NormsFormat normsFormat = new Lucene53RWNormsFormat(); + @Override + public SegmentInfoFormat segmentInfoFormat() { + return segmentInfoFormat; + } + @Override public NormsFormat normsFormat() { return normsFormat; diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWSegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWSegmentInfoFormat.java new file mode 100644 index 00000000000..f2fbe9d70b9 --- /dev/null +++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/Lucene62RWSegmentInfoFormat.java @@ -0,0 +1,193 @@ +/* + * 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.lucene.codecs.lucene62; + +import java.io.IOException; +import java.util.Set; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.SortedNumericSelector; +import org.apache.lucene.search.SortedNumericSortField; +import org.apache.lucene.search.SortedSetSelector; +import org.apache.lucene.search.SortedSetSortField; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.Version; + +/** + * Read-write version of 6.2 SegmentInfoFormat for testing + * @deprecated for test purposes only + */ +@Deprecated +public class Lucene62RWSegmentInfoFormat extends Lucene62SegmentInfoFormat { + + @Override + public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException { + final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene62SegmentInfoFormat.SI_EXTENSION); + + try (IndexOutput output = dir.createOutput(fileName, ioContext)) { + // Only add the file once we've successfully created it, else IFD assert can trip: + si.addFile(fileName); + CodecUtil.writeIndexHeader(output, + Lucene62SegmentInfoFormat.CODEC_NAME, + Lucene62SegmentInfoFormat.VERSION_CURRENT, + si.getId(), + ""); + Version version = si.getVersion(); + if (version.major < 5) { + throw new IllegalArgumentException("invalid major version: should be >= 5 but got: " + version.major + " segment=" + si); + } + // Write the Lucene version that created this segment, since 3.1 + output.writeInt(version.major); + output.writeInt(version.minor); + output.writeInt(version.bugfix); + assert version.prerelease == 0; + output.writeInt(si.maxDoc()); + + output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); + output.writeMapOfStrings(si.getDiagnostics()); + Set files = si.files(); + for (String file : files) { + if (!IndexFileNames.parseSegmentName(file).equals(si.name)) { + throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files); + } + } + output.writeSetOfStrings(files); + output.writeMapOfStrings(si.getAttributes()); + + Sort indexSort = si.getIndexSort(); + int numSortFields = indexSort == null ? 0 : indexSort.getSort().length; + output.writeVInt(numSortFields); + for (int i = 0; i < numSortFields; ++i) { + SortField sortField = indexSort.getSort()[i]; + SortField.Type sortType = sortField.getType(); + output.writeString(sortField.getField()); + int sortTypeID; + switch (sortField.getType()) { + case STRING: + sortTypeID = 0; + break; + case LONG: + sortTypeID = 1; + break; + case INT: + sortTypeID = 2; + break; + case DOUBLE: + sortTypeID = 3; + break; + case FLOAT: + sortTypeID = 4; + break; + case CUSTOM: + if (sortField instanceof SortedSetSortField) { + sortTypeID = 5; + sortType = SortField.Type.STRING; + } else if (sortField instanceof SortedNumericSortField) { + sortTypeID = 6; + sortType = ((SortedNumericSortField) sortField).getNumericType(); + } else { + throw new IllegalStateException("Unexpected SortedNumericSortField " + sortField); + } + break; + default: + throw new IllegalStateException("Unexpected sort type: " + sortField.getType()); + } + output.writeVInt(sortTypeID); + if (sortTypeID == 5) { + SortedSetSortField ssf = (SortedSetSortField) sortField; + if (ssf.getSelector() == SortedSetSelector.Type.MIN) { + output.writeByte((byte) 0); + } else if (ssf.getSelector() == SortedSetSelector.Type.MAX) { + output.writeByte((byte) 1); + } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MIN) { + output.writeByte((byte) 2); + } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MAX) { + output.writeByte((byte) 3); + } else { + throw new IllegalStateException("Unexpected SortedSetSelector type: " + ssf.getSelector()); + } + } else if (sortTypeID == 6) { + SortedNumericSortField snsf = (SortedNumericSortField) sortField; + if (snsf.getNumericType() == SortField.Type.LONG) { + output.writeByte((byte) 0); + } else if (snsf.getNumericType() == SortField.Type.INT) { + output.writeByte((byte) 1); + } else if (snsf.getNumericType() == SortField.Type.DOUBLE) { + output.writeByte((byte) 2); + } else if (snsf.getNumericType() == SortField.Type.FLOAT) { + output.writeByte((byte) 3); + } else { + throw new IllegalStateException("Unexpected SortedNumericSelector type: " + snsf.getNumericType()); + } + if (snsf.getSelector() == SortedNumericSelector.Type.MIN) { + output.writeByte((byte) 0); + } else if (snsf.getSelector() == SortedNumericSelector.Type.MAX) { + output.writeByte((byte) 1); + } else { + throw new IllegalStateException("Unexpected sorted numeric selector type: " + snsf.getSelector()); + } + } + output.writeByte((byte) (sortField.getReverse() ? 0 : 1)); + + // write missing value + Object missingValue = sortField.getMissingValue(); + if (missingValue == null) { + output.writeByte((byte) 0); + } else { + switch(sortType) { + case STRING: + if (missingValue == SortField.STRING_LAST) { + output.writeByte((byte) 1); + } else if (missingValue == SortField.STRING_FIRST) { + output.writeByte((byte) 2); + } else { + throw new AssertionError("unrecognized missing value for STRING field \"" + sortField.getField() + "\": " + missingValue); + } + break; + case LONG: + output.writeByte((byte) 1); + output.writeLong(((Long) missingValue).longValue()); + break; + case INT: + output.writeByte((byte) 1); + output.writeInt(((Integer) missingValue).intValue()); + break; + case DOUBLE: + output.writeByte((byte) 1); + output.writeLong(Double.doubleToLongBits(((Double) missingValue).doubleValue())); + break; + case FLOAT: + output.writeByte((byte) 1); + output.writeInt(Float.floatToIntBits(((Float) missingValue).floatValue())); + break; + default: + throw new IllegalStateException("Unexpected sort type: " + sortField.getType()); + } + } + } + + CodecUtil.writeFooter(output); + } + } + +} diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java similarity index 82% rename from lucene/core/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java rename to lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java index 54110f72a9d..e0efa9521a3 100644 --- a/lucene/core/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/codecs/lucene62/TestLucene62SegmentInfoFormat.java @@ -19,7 +19,6 @@ package org.apache.lucene.codecs.lucene62; import org.apache.lucene.codecs.Codec; import org.apache.lucene.index.BaseSegmentInfoFormatTestCase; -import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.Version; /** @@ -27,13 +26,23 @@ import org.apache.lucene.util.Version; */ public class TestLucene62SegmentInfoFormat extends BaseSegmentInfoFormatTestCase { + @Override + protected int getCreatedVersionMajor() { + return Version.LUCENE_6_2_0.major; + } + @Override protected Version[] getVersions() { - return new Version[] { Version.LATEST }; + return new Version[] { Version.LUCENE_6_2_0 }; } @Override protected Codec getCodec() { - return TestUtil.getDefaultCodec(); + return new Lucene62RWCodec(); + } + + @Override + protected boolean supportsMinVersion() { + return false; } } diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java index 8e87dcccda5..f180b47a3ed 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java @@ -693,10 +693,18 @@ public class TestBackwardsCompatibility extends LuceneTestCase { System.out.println("\nTEST: index=" + name); } Directory dir = newDirectory(oldIndexDirs.get(name)); + + final SegmentInfos oldSegInfos = SegmentInfos.readLatestCommit(dir); + IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random()))); w.forceMerge(1); w.close(); - + + final SegmentInfos segInfos = SegmentInfos.readLatestCommit(dir); + assertEquals(oldSegInfos.getIndexCreatedVersionMajor(), segInfos.getIndexCreatedVersionMajor()); + assertEquals(Version.LATEST, segInfos.asList().get(0).info.getVersion()); + assertEquals(oldSegInfos.asList().get(0).info.getMinVersion(), segInfos.asList().get(0).info.getMinVersion()); + dir.close(); } } @@ -707,26 +715,30 @@ public class TestBackwardsCompatibility extends LuceneTestCase { System.out.println("\nTEST: old index " + name); } Directory oldDir = oldIndexDirs.get(name); - Version indexCreatedVersion = SegmentInfos.readLatestCommit(oldDir).getIndexCreatedVersion(); + SegmentInfos infos = SegmentInfos.readLatestCommit(oldDir); Directory targetDir = newDirectory(); - // Simulate writing into an index that was created on the same version - new SegmentInfos(indexCreatedVersion).commit(targetDir); + if (infos.getCommitLuceneVersion().major != Version.LATEST.major) { + // both indexes are not compatible + Directory targetDir2 = newDirectory(); + IndexWriter w = new IndexWriter(targetDir2, newIndexWriterConfig(new MockAnalyzer(random()))); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> w.addIndexes(oldDir)); + assertTrue(e.getMessage(), e.getMessage().startsWith("Cannot use addIndexes(Directory) with indexes that have been created by a different Lucene version.")); + w.close(); + targetDir2.close(); + + // for the next test, we simulate writing to an index that was created on the same major version + new SegmentInfos(infos.getIndexCreatedVersionMajor()).commit(targetDir); + } + IndexWriter w = new IndexWriter(targetDir, newIndexWriterConfig(new MockAnalyzer(random()))); w.addIndexes(oldDir); w.close(); targetDir.close(); - // Now check that we forbid calling addIndexes with a different version - targetDir = newDirectory(); - IndexWriter oldWriter = new IndexWriter(targetDir, newIndexWriterConfig(new MockAnalyzer(random()))); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> oldWriter.addIndexes(oldDir)); - assertTrue(e.getMessage(), e.getMessage().startsWith("Cannot use addIndexes(Directory) with indexes that have been created by a different Lucene version.")); - if (VERBOSE) { System.out.println("\nTEST: done adding indices; now close"); } - oldWriter.close(); targetDir.close(); } @@ -734,9 +746,22 @@ public class TestBackwardsCompatibility extends LuceneTestCase { public void testAddOldIndexesReader() throws IOException { for (String name : oldNames) { - DirectoryReader reader = DirectoryReader.open(oldIndexDirs.get(name)); + Directory oldDir = oldIndexDirs.get(name); + SegmentInfos infos = SegmentInfos.readLatestCommit(oldDir); + DirectoryReader reader = DirectoryReader.open(oldDir); Directory targetDir = newDirectory(); + if (infos.getCommitLuceneVersion().major != Version.LATEST.major) { + Directory targetDir2 = newDirectory(); + IndexWriter w = new IndexWriter(targetDir2, newIndexWriterConfig(new MockAnalyzer(random()))); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> TestUtil.addIndexesSlowly(w, reader)); + assertEquals(e.getMessage(), "Cannot merge a segment that has been created with major version 6 into this index which has been created by major version 7"); + w.close(); + targetDir2.close(); + + // for the next test, we simulate writing to an index that was created on the same major version + new SegmentInfos(infos.getIndexCreatedVersionMajor()).commit(targetDir); + } IndexWriter w = new IndexWriter(targetDir, newIndexWriterConfig(new MockAnalyzer(random()))); TestUtil.addIndexesSlowly(w, reader); w.close(); @@ -1245,11 +1270,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase { SegmentInfos infos = SegmentInfos.readLatestCommit(dir); // those indexes are created by a single version so we can // compare the commit version with the created version - if (infos.getCommitLuceneVersion().onOrAfter(Version.LUCENE_7_0_0)) { - assertEquals(infos.getCommitLuceneVersion(), infos.getIndexCreatedVersion()); - } else { - assertNull(infos.getIndexCreatedVersion()); - } + assertEquals(infos.getCommitLuceneVersion().major, infos.getIndexCreatedVersionMajor()); } } @@ -1316,7 +1337,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase { } } - private int checkAllSegmentsUpgraded(Directory dir, Version indexCreatedVersion) throws IOException { + private int checkAllSegmentsUpgraded(Directory dir, int indexCreatedVersion) throws IOException { final SegmentInfos infos = SegmentInfos.readLatestCommit(dir); if (VERBOSE) { System.out.println("checkAllSegmentsUpgraded: " + infos); @@ -1325,7 +1346,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase { assertEquals(Version.LATEST, si.info.getVersion()); } assertEquals(Version.LATEST, infos.getCommitLuceneVersion()); - assertEquals(indexCreatedVersion, infos.getIndexCreatedVersion()); + assertEquals(indexCreatedVersion, infos.getIndexCreatedVersionMajor()); return infos.size(); } @@ -1343,7 +1364,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase { System.out.println("testUpgradeOldIndex: index=" +name); } Directory dir = newDirectory(oldIndexDirs.get(name)); - Version indexCreatedVersion = SegmentInfos.readLatestCommit(dir).getIndexCreatedVersion(); + int indexCreatedVersion = SegmentInfos.readLatestCommit(dir).getIndexCreatedVersionMajor(); newIndexUpgrader(dir).upgrade(); @@ -1360,7 +1381,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase { try { for (Map.Entry entry : oldIndexDirs.entrySet()) { String name = entry.getKey(); - Version indexCreatedVersion = SegmentInfos.readLatestCommit(entry.getValue()).getIndexCreatedVersion(); + int indexCreatedVersion = SegmentInfos.readLatestCommit(entry.getValue()).getIndexCreatedVersionMajor(); Path dir = createTempDir(name); TestUtil.unzip(getDataInputStream("index." + name + ".zip"), dir); @@ -1413,7 +1434,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase { } Directory dir = newDirectory(oldIndexDirs.get(name)); assertEquals("Original index must be single segment", 1, getNumberOfSegments(dir)); - Version indexCreatedVersion = SegmentInfos.readLatestCommit(dir).getIndexCreatedVersion(); + int indexCreatedVersion = SegmentInfos.readLatestCommit(dir).getIndexCreatedVersionMajor(); // create a bunch of dummy segments int id = 40; @@ -1472,7 +1493,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase { newIndexUpgrader(dir).upgrade(); - checkAllSegmentsUpgraded(dir, null); + checkAllSegmentsUpgraded(dir, 6); dir.close(); } @@ -1598,7 +1619,7 @@ public class TestBackwardsCompatibility extends LuceneTestCase { DirectoryReader reader = DirectoryReader.open(dir); assertEquals(1, reader.leaves().size()); - Sort sort = reader.leaves().get(0).reader().getIndexSort(); + Sort sort = reader.leaves().get(0).reader().getMetaData().getSort(); assertNotNull(sort); assertEquals("!", sort.toString()); reader.close(); diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java index 917785ecbfd..46b30d3a96b 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestFixBrokenOffsets.java @@ -16,7 +16,6 @@ */ package org.apache.lucene.index; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.nio.file.Path; @@ -94,14 +93,11 @@ public class TestFixBrokenOffsets extends LuceneTestCase { for(int i=0;i finalW2.addIndexes(codecReaders)); + assertEquals("Cannot merge a segment that has been created with major version 6 into this index which has been created by major version 7", e.getMessage()); reader.close(); w.close(); - - // NOT OK: broken offsets were copied into a 7.0 segment: - ByteArrayOutputStream output = new ByteArrayOutputStream(1024); - RuntimeException re = expectThrows(RuntimeException.class, () -> {TestUtil.checkIndex(tmpDir2, false, true, output);}); - assertEquals("term [66 6f 6f]: doc 0: pos 1: startOffset 7 < lastStartOffset 10; consider using the FixBrokenOffsets tool in Lucene's backward-codecs module to correct your index", re.getMessage()); tmpDir2.close(); // Now run the tool and confirm the broken offsets are fixed: diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java index 73d933a7b8b..c77b9260599 100644 --- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java +++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestIndexWriterOnOldIndex.java @@ -36,16 +36,16 @@ public class TestIndexWriterOnOldIndex extends LuceneTestCase { Directory dir = newFSDirectory(path); for (OpenMode openMode : OpenMode.values()) { Directory tmpDir = newDirectory(dir); - assertEquals(null /** 6.3.0 */, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersion()); + assertEquals(6 /** 6.3.0 */, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersionMajor()); IndexWriter w = new IndexWriter(tmpDir, newIndexWriterConfig().setOpenMode(openMode)); w.commit(); w.close(); switch (openMode) { case CREATE: - assertEquals(Version.LATEST, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersion()); + assertEquals(Version.LATEST.major, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersionMajor()); break; default: - assertEquals(null /** 6.3.0 */, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersion()); + assertEquals(6 /** 6.3.0 */, SegmentInfos.readLatestCommit(tmpDir).getIndexCreatedVersionMajor()); } tmpDir.close(); } diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java index 3d38d72385f..8a71c6df7dc 100644 --- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java +++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoFormat.java @@ -55,6 +55,7 @@ import org.apache.lucene.util.Version; */ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat { final static BytesRef SI_VERSION = new BytesRef(" version "); + final static BytesRef SI_MIN_VERSION = new BytesRef(" min version "); final static BytesRef SI_DOCCOUNT = new BytesRef(" number of documents "); final static BytesRef SI_USECOMPOUND = new BytesRef(" uses compound file "); final static BytesRef SI_NUM_DIAG = new BytesRef(" diagnostics "); @@ -88,7 +89,21 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat { } catch (ParseException pe) { throw new CorruptIndexException("unable to parse version string: " + pe.getMessage(), input, pe); } - + + SimpleTextUtil.readLine(input, scratch); + assert StringHelper.startsWith(scratch.get(), SI_MIN_VERSION); + Version minVersion; + try { + String versionString = readString(SI_MIN_VERSION.length, scratch); + if (versionString.equals("null")) { + minVersion = null; + } else { + minVersion = Version.parse(versionString); + } + } catch (ParseException pe) { + throw new CorruptIndexException("unable to parse version string: " + pe.getMessage(), input, pe); + } + SimpleTextUtil.readLine(input, scratch); assert StringHelper.startsWith(scratch.get(), SI_DOCCOUNT); final int docCount = Integer.parseInt(readString(SI_DOCCOUNT.length, scratch)); @@ -288,7 +303,7 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat { SimpleTextUtil.checkFooter(input); - SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount, + SegmentInfo info = new SegmentInfo(directory, version, minVersion, segmentName, docCount, isCompoundFile, null, Collections.unmodifiableMap(diagnostics), id, Collections.unmodifiableMap(attributes), indexSort); info.setFiles(files); @@ -345,7 +360,15 @@ public class SimpleTextSegmentInfoFormat extends SegmentInfoFormat { SimpleTextUtil.write(output, SI_VERSION); SimpleTextUtil.write(output, si.getVersion().toString(), scratch); SimpleTextUtil.writeNewline(output); - + + SimpleTextUtil.write(output, SI_MIN_VERSION); + if (si.getMinVersion() == null) { + SimpleTextUtil.write(output, "null", scratch); + } else { + SimpleTextUtil.write(output, si.getMinVersion().toString(), scratch); + } + SimpleTextUtil.writeNewline(output); + SimpleTextUtil.write(output, SI_DOCCOUNT); SimpleTextUtil.write(output, Integer.toString(si.maxDoc()), scratch); SimpleTextUtil.writeNewline(output); diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java index da6e395e27a..e91da3b5824 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene62/Lucene62SegmentInfoFormat.java @@ -37,7 +37,6 @@ import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.DataOutput; // javadocs import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.Version; /** @@ -244,7 +243,7 @@ public class Lucene62SegmentInfoFormat extends SegmentInfoFormat { indexSort = null; } - si = new SegmentInfo(dir, version, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, indexSort); + si = new SegmentInfo(dir, version, null, segment, docCount, isCompoundFile, null, diagnostics, segmentID, attributes, indexSort); si.setFiles(files); } catch (Throwable exception) { priorE = exception; @@ -256,153 +255,8 @@ public class Lucene62SegmentInfoFormat extends SegmentInfoFormat { } @Override - public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException { - final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene62SegmentInfoFormat.SI_EXTENSION); - - try (IndexOutput output = dir.createOutput(fileName, ioContext)) { - // Only add the file once we've successfully created it, else IFD assert can trip: - si.addFile(fileName); - CodecUtil.writeIndexHeader(output, - Lucene62SegmentInfoFormat.CODEC_NAME, - Lucene62SegmentInfoFormat.VERSION_CURRENT, - si.getId(), - ""); - Version version = si.getVersion(); - if (version.major < 5) { - throw new IllegalArgumentException("invalid major version: should be >= 5 but got: " + version.major + " segment=" + si); - } - // Write the Lucene version that created this segment, since 3.1 - output.writeInt(version.major); - output.writeInt(version.minor); - output.writeInt(version.bugfix); - assert version.prerelease == 0; - output.writeInt(si.maxDoc()); - - output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); - output.writeMapOfStrings(si.getDiagnostics()); - Set files = si.files(); - for (String file : files) { - if (!IndexFileNames.parseSegmentName(file).equals(si.name)) { - throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files); - } - } - output.writeSetOfStrings(files); - output.writeMapOfStrings(si.getAttributes()); - - Sort indexSort = si.getIndexSort(); - int numSortFields = indexSort == null ? 0 : indexSort.getSort().length; - output.writeVInt(numSortFields); - for (int i = 0; i < numSortFields; ++i) { - SortField sortField = indexSort.getSort()[i]; - SortField.Type sortType = sortField.getType(); - output.writeString(sortField.getField()); - int sortTypeID; - switch (sortField.getType()) { - case STRING: - sortTypeID = 0; - break; - case LONG: - sortTypeID = 1; - break; - case INT: - sortTypeID = 2; - break; - case DOUBLE: - sortTypeID = 3; - break; - case FLOAT: - sortTypeID = 4; - break; - case CUSTOM: - if (sortField instanceof SortedSetSortField) { - sortTypeID = 5; - sortType = SortField.Type.STRING; - } else if (sortField instanceof SortedNumericSortField) { - sortTypeID = 6; - sortType = ((SortedNumericSortField) sortField).getNumericType(); - } else { - throw new IllegalStateException("Unexpected SortedNumericSortField " + sortField); - } - break; - default: - throw new IllegalStateException("Unexpected sort type: " + sortField.getType()); - } - output.writeVInt(sortTypeID); - if (sortTypeID == 5) { - SortedSetSortField ssf = (SortedSetSortField) sortField; - if (ssf.getSelector() == SortedSetSelector.Type.MIN) { - output.writeByte((byte) 0); - } else if (ssf.getSelector() == SortedSetSelector.Type.MAX) { - output.writeByte((byte) 1); - } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MIN) { - output.writeByte((byte) 2); - } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MAX) { - output.writeByte((byte) 3); - } else { - throw new IllegalStateException("Unexpected SortedSetSelector type: " + ssf.getSelector()); - } - } else if (sortTypeID == 6) { - SortedNumericSortField snsf = (SortedNumericSortField) sortField; - if (snsf.getNumericType() == SortField.Type.LONG) { - output.writeByte((byte) 0); - } else if (snsf.getNumericType() == SortField.Type.INT) { - output.writeByte((byte) 1); - } else if (snsf.getNumericType() == SortField.Type.DOUBLE) { - output.writeByte((byte) 2); - } else if (snsf.getNumericType() == SortField.Type.FLOAT) { - output.writeByte((byte) 3); - } else { - throw new IllegalStateException("Unexpected SortedNumericSelector type: " + snsf.getNumericType()); - } - if (snsf.getSelector() == SortedNumericSelector.Type.MIN) { - output.writeByte((byte) 0); - } else if (snsf.getSelector() == SortedNumericSelector.Type.MAX) { - output.writeByte((byte) 1); - } else { - throw new IllegalStateException("Unexpected sorted numeric selector type: " + snsf.getSelector()); - } - } - output.writeByte((byte) (sortField.getReverse() ? 0 : 1)); - - // write missing value - Object missingValue = sortField.getMissingValue(); - if (missingValue == null) { - output.writeByte((byte) 0); - } else { - switch(sortType) { - case STRING: - if (missingValue == SortField.STRING_LAST) { - output.writeByte((byte) 1); - } else if (missingValue == SortField.STRING_FIRST) { - output.writeByte((byte) 2); - } else { - throw new AssertionError("unrecognized missing value for STRING field \"" + sortField.getField() + "\": " + missingValue); - } - break; - case LONG: - output.writeByte((byte) 1); - output.writeLong(((Long) missingValue).longValue()); - break; - case INT: - output.writeByte((byte) 1); - output.writeInt(((Integer) missingValue).intValue()); - break; - case DOUBLE: - output.writeByte((byte) 1); - output.writeLong(Double.doubleToLongBits(((Double) missingValue).doubleValue())); - break; - case FLOAT: - output.writeByte((byte) 1); - output.writeInt(Float.floatToIntBits(((Float) missingValue).floatValue())); - break; - default: - throw new IllegalStateException("Unexpected sort type: " + sortField.getType()); - } - } - } - - CodecUtil.writeFooter(output); - } + public void write(Directory dir, SegmentInfo info, IOContext ioContext) throws IOException { + throw new UnsupportedOperationException("This format can only be used for reading"); } /** File extension used to store {@link SegmentInfo}. */ diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java index 7f9aed0ed96..d04d5549c60 100644 --- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70Codec.java @@ -37,7 +37,6 @@ import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode; import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat; import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat; import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat; -import org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat; import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat; import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; @@ -55,7 +54,7 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat; public class Lucene70Codec extends Codec { private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat(); private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat(); - private final SegmentInfoFormat segmentInfosFormat = new Lucene62SegmentInfoFormat(); + private final SegmentInfoFormat segmentInfosFormat = new Lucene70SegmentInfoFormat(); private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat(); private final CompoundFormat compoundFormat = new Lucene50CompoundFormat(); diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java new file mode 100644 index 00000000000..bd2bf06e324 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70SegmentInfoFormat.java @@ -0,0 +1,439 @@ +/* + * 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.lucene.codecs.lucene70; + +import java.io.IOException; +import java.util.Map; +import java.util.Set; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.codecs.SegmentInfoFormat; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.index.IndexWriter; // javadocs +import org.apache.lucene.index.SegmentInfo; // javadocs +import org.apache.lucene.index.SegmentInfos; // javadocs +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.SortedNumericSelector; +import org.apache.lucene.search.SortedNumericSortField; +import org.apache.lucene.search.SortedSetSelector; +import org.apache.lucene.search.SortedSetSortField; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.DataOutput; // javadocs +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.Version; + +/** + * Lucene 7.0 Segment info format. + *

    + * Files: + *

      + *
    • .si: Header, SegVersion, SegSize, IsCompoundFile, Diagnostics, Files, Attributes, IndexSort, Footer + *
    + * Data types: + *
      + *
    • Header --> {@link CodecUtil#writeIndexHeader IndexHeader}
    • + *
    • SegSize --> {@link DataOutput#writeInt Int32}
    • + *
    • SegVersion --> {@link DataOutput#writeString String}
    • + *
    • SegMinVersion --> {@link DataOutput#writeString String}
    • + *
    • Files --> {@link DataOutput#writeSetOfStrings Set<String>}
    • + *
    • Diagnostics,Attributes --> {@link DataOutput#writeMapOfStrings Map<String,String>}
    • + *
    • IsCompoundFile --> {@link DataOutput#writeByte Int8}
    • + *
    • IndexSort --> {@link DataOutput#writeVInt Int32} count, followed by {@code count} SortField
    • + *
    • SortField --> {@link DataOutput#writeString String} field name, followed by {@link DataOutput#writeVInt Int32} sort type ID, + * followed by {@link DataOutput#writeByte Int8} indicatating reversed sort, followed by a type-specific encoding of the optional missing value + *
    • Footer --> {@link CodecUtil#writeFooter CodecFooter}
    • + *
    + * Field Descriptions: + *
      + *
    • SegVersion is the code version that created the segment.
    • + *
    • SegMinVersion is the minimum code version that contributed documents to the segment.
    • + *
    • SegSize is the number of documents contained in the segment index.
    • + *
    • IsCompoundFile records whether the segment is written as a compound file or + * not. If this is -1, the segment is not a compound file. If it is 1, the segment + * is a compound file.
    • + *
    • The Diagnostics Map is privately written by {@link IndexWriter}, as a debugging aid, + * for each segment it creates. It includes metadata like the current Lucene + * version, OS, Java version, why the segment was created (merge, flush, + * addIndexes), etc.
    • + *
    • Files is a list of files referred to by this segment.
    • + *
    + * + * @see SegmentInfos + * @lucene.experimental + */ +public class Lucene70SegmentInfoFormat extends SegmentInfoFormat { + + /** Sole constructor. */ + public Lucene70SegmentInfoFormat() { + } + + @Override + public SegmentInfo read(Directory dir, String segment, byte[] segmentID, IOContext context) throws IOException { + final String fileName = IndexFileNames.segmentFileName(segment, "", Lucene70SegmentInfoFormat.SI_EXTENSION); + try (ChecksumIndexInput input = dir.openChecksumInput(fileName, context)) { + Throwable priorE = null; + SegmentInfo si = null; + try { + int format = CodecUtil.checkIndexHeader(input, Lucene70SegmentInfoFormat.CODEC_NAME, + Lucene70SegmentInfoFormat.VERSION_START, + Lucene70SegmentInfoFormat.VERSION_CURRENT, + segmentID, ""); + final Version version = Version.fromBits(input.readInt(), input.readInt(), input.readInt()); + byte hasMinVersion = input.readByte(); + final Version minVersion; + switch (hasMinVersion) { + case 0: + minVersion = null; + break; + case 1: + minVersion = Version.fromBits(input.readInt(), input.readInt(), input.readInt()); + break; + default: + throw new CorruptIndexException("Illegal boolean value " + hasMinVersion, input); + } + + final int docCount = input.readInt(); + if (docCount < 0) { + throw new CorruptIndexException("invalid docCount: " + docCount, input); + } + final boolean isCompoundFile = input.readByte() == SegmentInfo.YES; + + final Map diagnostics = input.readMapOfStrings(); + final Set files = input.readSetOfStrings(); + final Map attributes = input.readMapOfStrings(); + + int numSortFields = input.readVInt(); + Sort indexSort; + if (numSortFields > 0) { + SortField[] sortFields = new SortField[numSortFields]; + for(int i=0;i= 7 but got: " + version.major + " segment=" + si); + } + // Write the Lucene version that created this segment, since 3.1 + output.writeInt(version.major); + output.writeInt(version.minor); + output.writeInt(version.bugfix); + + // Write the min Lucene version that contributed docs to the segment, since 7.0 + if (si.getMinVersion() != null) { + output.writeByte((byte) 1); + Version minVersion = si.getMinVersion(); + output.writeInt(minVersion.major); + output.writeInt(minVersion.minor); + output.writeInt(minVersion.bugfix); + } else { + output.writeByte((byte) 0); + } + + assert version.prerelease == 0; + output.writeInt(si.maxDoc()); + + output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO)); + output.writeMapOfStrings(si.getDiagnostics()); + Set files = si.files(); + for (String file : files) { + if (!IndexFileNames.parseSegmentName(file).equals(si.name)) { + throw new IllegalArgumentException("invalid files: expected segment=" + si.name + ", got=" + files); + } + } + output.writeSetOfStrings(files); + output.writeMapOfStrings(si.getAttributes()); + + Sort indexSort = si.getIndexSort(); + int numSortFields = indexSort == null ? 0 : indexSort.getSort().length; + output.writeVInt(numSortFields); + for (int i = 0; i < numSortFields; ++i) { + SortField sortField = indexSort.getSort()[i]; + SortField.Type sortType = sortField.getType(); + output.writeString(sortField.getField()); + int sortTypeID; + switch (sortField.getType()) { + case STRING: + sortTypeID = 0; + break; + case LONG: + sortTypeID = 1; + break; + case INT: + sortTypeID = 2; + break; + case DOUBLE: + sortTypeID = 3; + break; + case FLOAT: + sortTypeID = 4; + break; + case CUSTOM: + if (sortField instanceof SortedSetSortField) { + sortTypeID = 5; + sortType = SortField.Type.STRING; + } else if (sortField instanceof SortedNumericSortField) { + sortTypeID = 6; + sortType = ((SortedNumericSortField) sortField).getNumericType(); + } else { + throw new IllegalStateException("Unexpected SortedNumericSortField " + sortField); + } + break; + default: + throw new IllegalStateException("Unexpected sort type: " + sortField.getType()); + } + output.writeVInt(sortTypeID); + if (sortTypeID == 5) { + SortedSetSortField ssf = (SortedSetSortField) sortField; + if (ssf.getSelector() == SortedSetSelector.Type.MIN) { + output.writeByte((byte) 0); + } else if (ssf.getSelector() == SortedSetSelector.Type.MAX) { + output.writeByte((byte) 1); + } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MIN) { + output.writeByte((byte) 2); + } else if (ssf.getSelector() == SortedSetSelector.Type.MIDDLE_MAX) { + output.writeByte((byte) 3); + } else { + throw new IllegalStateException("Unexpected SortedSetSelector type: " + ssf.getSelector()); + } + } else if (sortTypeID == 6) { + SortedNumericSortField snsf = (SortedNumericSortField) sortField; + if (snsf.getNumericType() == SortField.Type.LONG) { + output.writeByte((byte) 0); + } else if (snsf.getNumericType() == SortField.Type.INT) { + output.writeByte((byte) 1); + } else if (snsf.getNumericType() == SortField.Type.DOUBLE) { + output.writeByte((byte) 2); + } else if (snsf.getNumericType() == SortField.Type.FLOAT) { + output.writeByte((byte) 3); + } else { + throw new IllegalStateException("Unexpected SortedNumericSelector type: " + snsf.getNumericType()); + } + if (snsf.getSelector() == SortedNumericSelector.Type.MIN) { + output.writeByte((byte) 0); + } else if (snsf.getSelector() == SortedNumericSelector.Type.MAX) { + output.writeByte((byte) 1); + } else { + throw new IllegalStateException("Unexpected sorted numeric selector type: " + snsf.getSelector()); + } + } + output.writeByte((byte) (sortField.getReverse() ? 0 : 1)); + + // write missing value + Object missingValue = sortField.getMissingValue(); + if (missingValue == null) { + output.writeByte((byte) 0); + } else { + switch(sortType) { + case STRING: + if (missingValue == SortField.STRING_LAST) { + output.writeByte((byte) 1); + } else if (missingValue == SortField.STRING_FIRST) { + output.writeByte((byte) 2); + } else { + throw new AssertionError("unrecognized missing value for STRING field \"" + sortField.getField() + "\": " + missingValue); + } + break; + case LONG: + output.writeByte((byte) 1); + output.writeLong(((Long) missingValue).longValue()); + break; + case INT: + output.writeByte((byte) 1); + output.writeInt(((Integer) missingValue).intValue()); + break; + case DOUBLE: + output.writeByte((byte) 1); + output.writeLong(Double.doubleToLongBits(((Double) missingValue).doubleValue())); + break; + case FLOAT: + output.writeByte((byte) 1); + output.writeInt(Float.floatToIntBits(((Float) missingValue).floatValue())); + break; + default: + throw new IllegalStateException("Unexpected sort type: " + sortField.getType()); + } + } + } + + CodecUtil.writeFooter(output); + } + } + + /** File extension used to store {@link SegmentInfo}. */ + public final static String SI_EXTENSION = "si"; + static final String CODEC_NAME = "Lucene70SegmentInfo"; + static final int VERSION_START = 0; + static final int VERSION_CURRENT = VERSION_START; +} diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java index f3bdfb0b24f..c7ad0f41bce 100644 --- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java +++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java @@ -697,7 +697,7 @@ public final class CheckIndex implements Closeable { long startOpenReaderNS = System.nanoTime(); if (infoStream != null) infoStream.print(" test: open reader........."); - reader = new SegmentReader(info, IOContext.DEFAULT); + reader = new SegmentReader(info, sis.getIndexCreatedVersionMajor(), IOContext.DEFAULT); msg(infoStream, String.format(Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime()-startOpenReaderNS))); segInfoStat.openReaderPassed = true; diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java index 48901e50229..ed50650f86e 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java +++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java @@ -178,7 +178,7 @@ class DocumentsWriterPerThread { assert numDocsInRAM == 0 : "num docs " + numDocsInRAM; deleteSlice = deleteQueue.newSlice(); - segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), indexWriterConfig.getIndexSort()); + segmentInfo = new SegmentInfo(directoryOrig, Version.LATEST, Version.LATEST, segmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), indexWriterConfig.getIndexSort()); assert numDocsInRAM == 0; if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) { infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segmentName + " delQueue=" + deleteQueue); diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java b/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java index 5949fca3cd1..fd36ecb6d6f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/FilterCodecReader.java @@ -27,7 +27,6 @@ import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.PointsReader; import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.TermVectorsReader; -import org.apache.lucene.search.Sort; import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Bits; @@ -104,8 +103,8 @@ public abstract class FilterCodecReader extends CodecReader { } @Override - public Sort getIndexSort() { - return in.getIndexSort(); + public LeafMetaData getMetaData() { + return in.getMetaData(); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java index 0a3ec7fa52d..f3d8112ce33 100644 --- a/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java @@ -20,7 +20,6 @@ package org.apache.lucene.index; import java.io.IOException; import java.util.Iterator; -import org.apache.lucene.search.Sort; import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; @@ -398,9 +397,9 @@ public abstract class FilterLeafReader extends LeafReader { } @Override - public Sort getIndexSort() { + public LeafMetaData getMetaData() { ensureOpen(); - return in.getIndexSort(); + return in.getMetaData(); } @Override diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index 899643a0679..9a2915087ef 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -30,7 +30,6 @@ import java.util.LinkedList; import java.util.List; import java.util.Locale; import java.util.Map.Entry; -import java.util.Objects; import java.util.Map; import java.util.Queue; import java.util.Set; @@ -855,7 +854,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // against an index that's currently open for // searching. In this case we write the next // segments_N file with no segments: - final SegmentInfos sis = new SegmentInfos(Version.LATEST); + final SegmentInfos sis = new SegmentInfos(Version.LATEST.major); try { final SegmentInfos previous = SegmentInfos.readLatestCommit(directory); sis.updateGenerationVersionAndCounter(previous); @@ -2654,12 +2653,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { infoStream.message("IW", "addIndexes: process directory " + dir); } SegmentInfos sis = SegmentInfos.readLatestCommit(dir); // read infos from dir - if (Objects.equals(segmentInfos.getIndexCreatedVersion(), sis.getIndexCreatedVersion()) == false) { + if (segmentInfos.getIndexCreatedVersionMajor() != sis.getIndexCreatedVersionMajor()) { throw new IllegalArgumentException("Cannot use addIndexes(Directory) with indexes that have been created " - + "by a different Lucene version. The current index was generated by " - + segmentInfos.getIndexCreatedVersion() - + " while one of the directories contains an index that was generated with " - + sis.getIndexCreatedVersion()); + + "by a different Lucene version. The current index was generated by Lucene " + + segmentInfos.getIndexCreatedVersionMajor() + + " while one of the directories contains an index that was generated with Lucene " + + sis.getIndexCreatedVersionMajor()); } totalMaxDoc += sis.totalMaxDoc(); commits.add(sis); @@ -2747,7 +2746,26 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { return seqNo; } - + + private void validateMergeReader(CodecReader leaf) { + LeafMetaData segmentMeta = leaf.getMetaData(); + if (segmentInfos.getIndexCreatedVersionMajor() != segmentMeta.getCreatedVersionMajor()) { + throw new IllegalArgumentException("Cannot merge a segment that has been created with major version " + + segmentMeta.getCreatedVersionMajor() + " into this index which has been created by major version " + + segmentInfos.getIndexCreatedVersionMajor()); + } + + if (segmentInfos.getIndexCreatedVersionMajor() >= 7 && segmentMeta.getMinVersion() == null) { + throw new IllegalStateException("Indexes created on or after Lucene 7 must record the created version major, but " + leaf + " hides it"); + } + + Sort leafIndexSort = segmentMeta.getSort(); + if (config.getIndexSort() != null && leafIndexSort != null + && config.getIndexSort().equals(leafIndexSort) == false) { + throw new IllegalArgumentException("cannot change index sort from " + leafIndexSort + " to " + config.getIndexSort()); + } + } + /** * Merges the provided indexes into this index. * @@ -2801,12 +2819,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { flush(false, true); String mergedName = newSegmentName(); + for (CodecReader leaf : readers) { numDocs += leaf.numDocs(); - Sort leafIndexSort = leaf.getIndexSort(); - if (indexSort != null && leafIndexSort != null && indexSort.equals(leafIndexSort) == false) { - throw new IllegalArgumentException("cannot change index sort from " + leafIndexSort + " to " + indexSort); - } + validateMergeReader(leaf); } // Best-effort up front check: @@ -2818,7 +2834,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // abortable so that IW.close(false) is able to stop it TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(directory); - SegmentInfo info = new SegmentInfo(directoryOrig, Version.LATEST, mergedName, -1, + // We set the min version to null for now, it will be set later by SegmentMerger + SegmentInfo info = new SegmentInfo(directoryOrig, Version.LATEST, null, mergedName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort()); SegmentMerger merger = new SegmentMerger(Arrays.asList(readers), info, infoStream, trackingDir, @@ -2907,7 +2924,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { //System.out.println("copy seg=" + info.info.name + " version=" + info.info.getVersion()); // Same SI as before but we change directory and name - SegmentInfo newInfo = new SegmentInfo(directoryOrig, info.info.getVersion(), segName, info.info.maxDoc(), + SegmentInfo newInfo = new SegmentInfo(directoryOrig, info.info.getVersion(), info.info.getMinVersion(), segName, info.info.maxDoc(), info.info.getUseCompoundFile(), info.info.getCodec(), info.info.getDiagnostics(), info.info.getId(), info.info.getAttributes(), info.info.getIndexSort()); SegmentCommitInfo newInfoPerCommit = new SegmentCommitInfo(newInfo, info.getDelCount(), info.getDelGen(), @@ -4117,7 +4134,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // ConcurrentMergePolicy we keep deterministic segment // names. final String mergeSegmentName = newSegmentName(); - SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort()); + // We set the min version to null for now, it will be set later by SegmentMerger + SegmentInfo si = new SegmentInfo(directoryOrig, Version.LATEST, null, mergeSegmentName, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), config.getIndexSort()); Map details = new HashMap<>(); details.put("mergeMaxNumSegments", "" + merge.maxNumSegments); details.put("mergeFactor", Integer.toString(merge.segments.size())); @@ -4322,7 +4340,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // Let the merge wrap readers List mergeReaders = new ArrayList<>(); for (SegmentReader reader : merge.readers) { - mergeReaders.add(merge.wrapForMerge(reader)); + CodecReader wrappedReader = merge.wrapForMerge(reader); + validateMergeReader(wrappedReader); + mergeReaders.add(wrappedReader); } final SegmentMerger merger = new SegmentMerger(mergeReaders, merge.info.info, infoStream, dirWrapper, @@ -4608,7 +4628,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { // For infoStream output synchronized SegmentInfos toLiveInfos(SegmentInfos sis) { - final SegmentInfos newSIS = new SegmentInfos(sis.getIndexCreatedVersion()); + final SegmentInfos newSIS = new SegmentInfos(sis.getIndexCreatedVersionMajor()); final Map liveSIS = new HashMap<>(); for(SegmentCommitInfo info : segmentInfos) { liveSIS.put(info, info); diff --git a/lucene/core/src/java/org/apache/lucene/index/LeafMetaData.java b/lucene/core/src/java/org/apache/lucene/index/LeafMetaData.java new file mode 100644 index 00000000000..567d43edf97 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/index/LeafMetaData.java @@ -0,0 +1,74 @@ +/* + * 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.lucene.index; + +import org.apache.lucene.search.Sort; +import org.apache.lucene.util.Version; + +/** + * Provides read-only metadata about a leaf. + * @lucene.experimental + */ +public final class LeafMetaData { + + private final int createdVersionMajor; + private final Version minVersion; + private final Sort sort; + + /** Expert: Sole constructor. Public for use by custom {@link LeafReader} impls. */ + public LeafMetaData(int createdVersionMajor, Version minVersion, Sort sort) { + this.createdVersionMajor = createdVersionMajor; + if (createdVersionMajor > Version.LATEST.major) { + throw new IllegalArgumentException("createdVersionMajor is in the future: " + createdVersionMajor); + } + if (createdVersionMajor < 6) { + throw new IllegalArgumentException("createdVersionMajor must be >= 6, got: " + createdVersionMajor); + } + if (minVersion != null && minVersion.onOrAfter(Version.LUCENE_7_0_0) == false) { + throw new IllegalArgumentException("minVersion must be >= 7.0.0: " + minVersion); + } + if (createdVersionMajor >= 7 && minVersion == null) { + throw new IllegalArgumentException("minVersion must be set when createdVersionMajor is >= 7"); + } + this.minVersion = minVersion; + this.sort = sort; + } + + /** Get the Lucene version that created this index. This can be used to implement + * backward compatibility on top of the codec API. A return value of {@code 6} + * indicates that the created version is unknown. */ + public int getCreatedVersionMajor() { + return createdVersionMajor; + } + + /** + * Return the minimum Lucene version that contributed documents to this index, + * or {@code null} if this information is not available. + */ + public Version getMinVersion() { + return minVersion; + } + + /** + * Return the order in which documents from this index are sorted, or + * {@code null} if documents are in no particular order. + */ + public Sort getSort() { + return sort; + } + +} diff --git a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java index 13c8646d86c..c738bc58683 100644 --- a/lucene/core/src/java/org/apache/lucene/index/LeafReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/LeafReader.java @@ -19,7 +19,6 @@ package org.apache.lucene.index; import java.io.IOException; import org.apache.lucene.index.IndexReader.CacheHelper; -import org.apache.lucene.search.Sort; import org.apache.lucene.util.Bits; /** {@code LeafReader} is an abstract class, providing an interface for accessing an @@ -246,6 +245,8 @@ public abstract class LeafReader extends IndexReader { */ public abstract void checkIntegrity() throws IOException; - /** Returns null if this leaf is unsorted, or the {@link Sort} that it was sorted by */ - public abstract Sort getIndexSort(); + /** + * Return metadata about this leaf. + * @lucene.experimental */ + public abstract LeafMetaData getMetaData(); } diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeReaderWrapper.java b/lucene/core/src/java/org/apache/lucene/index/MergeReaderWrapper.java index fffb693ead1..3a3573adb5b 100644 --- a/lucene/core/src/java/org/apache/lucene/index/MergeReaderWrapper.java +++ b/lucene/core/src/java/org/apache/lucene/index/MergeReaderWrapper.java @@ -24,7 +24,6 @@ import org.apache.lucene.codecs.FieldsProducer; import org.apache.lucene.codecs.NormsProducer; import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.TermVectorsReader; -import org.apache.lucene.search.Sort; import org.apache.lucene.util.Bits; /** This is a hack to make index sorting fast, with a {@link LeafReader} that always returns merge instances when you ask for the codec readers. */ @@ -235,7 +234,7 @@ class MergeReaderWrapper extends LeafReader { } @Override - public Sort getIndexSort() { - return in.getIndexSort(); + public LeafMetaData getMetaData() { + return in.getMetaData(); } } diff --git a/lucene/core/src/java/org/apache/lucene/index/MergeState.java b/lucene/core/src/java/org/apache/lucene/index/MergeState.java index a7c83071e71..9ad69f6b02f 100644 --- a/lucene/core/src/java/org/apache/lucene/index/MergeState.java +++ b/lucene/core/src/java/org/apache/lucene/index/MergeState.java @@ -231,7 +231,7 @@ public class MergeState { List readers = new ArrayList<>(originalReaders.size()); for (CodecReader leaf : originalReaders) { - Sort segmentSort = leaf.getIndexSort(); + Sort segmentSort = leaf.getMetaData().getSort(); if (segmentSort == null) { // This segment was written by flush, so documents are not yet sorted, so we sort them now: diff --git a/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java index eee0c16a463..5fb28b0fb15 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/ParallelLeafReader.java @@ -28,6 +28,7 @@ import java.util.TreeMap; import org.apache.lucene.search.Sort; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.Version; /** An {@link LeafReader} which reads multiple, parallel indexes. Each index * added must have the same number of documents, but typically each contains @@ -56,7 +57,7 @@ public class ParallelLeafReader extends LeafReader { private final boolean closeSubReaders; private final int maxDoc, numDocs; private final boolean hasDeletions; - private final Sort indexSort; + private final LeafMetaData metaData; private final SortedMap fieldToReader = new TreeMap<>(); private final SortedMap tvFieldToReader = new TreeMap<>(); @@ -104,16 +105,25 @@ public class ParallelLeafReader extends LeafReader { FieldInfos.Builder builder = new FieldInfos.Builder(); Sort indexSort = null; + int createdVersionMajor = -1; // build FieldInfos and fieldToReader map: for (final LeafReader reader : this.parallelReaders) { - Sort leafIndexSort = reader.getIndexSort(); + LeafMetaData leafMetaData = reader.getMetaData(); + + Sort leafIndexSort = leafMetaData.getSort(); if (indexSort == null) { indexSort = leafIndexSort; } else if (leafIndexSort != null && indexSort.equals(leafIndexSort) == false) { throw new IllegalArgumentException("cannot combine LeafReaders that have different index sorts: saw both sort=" + indexSort + " and " + leafIndexSort); } + if (createdVersionMajor == -1) { + createdVersionMajor = leafMetaData.getCreatedVersionMajor(); + } else if (createdVersionMajor != leafMetaData.getCreatedVersionMajor()) { + throw new IllegalArgumentException("cannot combine LeafReaders that have different creation versions: saw both version=" + createdVersionMajor + " and " + leafMetaData.getCreatedVersionMajor()); + } + final FieldInfos readerFieldInfos = reader.getFieldInfos(); for (FieldInfo fieldInfo : readerFieldInfos) { // NOTE: first reader having a given field "wins": @@ -126,8 +136,24 @@ public class ParallelLeafReader extends LeafReader { } } } + if (createdVersionMajor == -1) { + // empty reader + createdVersionMajor = Version.LATEST.major; + } + + Version minVersion = Version.LATEST; + for (final LeafReader reader : this.parallelReaders) { + Version leafVersion = reader.getMetaData().getMinVersion(); + if (leafVersion == null) { + minVersion = null; + break; + } else if (minVersion.onOrAfter(leafVersion)) { + minVersion = leafVersion; + } + } + fieldInfos = builder.finish(); - this.indexSort = indexSort; + this.metaData = new LeafMetaData(createdVersionMajor, minVersion, indexSort); // build Fields instance for (final LeafReader reader : this.parallelReaders) { @@ -358,8 +384,8 @@ public class ParallelLeafReader extends LeafReader { } @Override - public Sort getIndexSort() { - return indexSort; + public LeafMetaData getMetaData() { + return metaData; } } diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java index 3cd465c9d41..d4dd4a44a64 100644 --- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java +++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java @@ -140,7 +140,7 @@ class ReadersAndUpdates { public SegmentReader getReader(IOContext context) throws IOException { if (reader == null) { // We steal returned ref: - reader = new SegmentReader(info, context); + reader = new SegmentReader(info, writer.segmentInfos.getIndexCreatedVersionMajor(), context); if (liveDocs == null) { liveDocs = reader.getLiveDocs(); } @@ -552,7 +552,7 @@ class ReadersAndUpdates { // reader could be null e.g. for a just merged segment (from // IndexWriter.commitMergedDeletes). - final SegmentReader reader = this.reader == null ? new SegmentReader(info, IOContext.READONCE) : this.reader; + final SegmentReader reader = this.reader == null ? new SegmentReader(info, writer.segmentInfos.getIndexCreatedVersionMajor(), IOContext.READONCE) : this.reader; try { // clone FieldInfos so that we can update their dvGen separately from // the reader's infos and write them to a new fieldInfos_gen file diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java index ec12365e958..1c024417cb3 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java @@ -77,7 +77,13 @@ public final class SegmentInfo { // The format expected is "x.y" - "2.x" for pre-3.0 indexes (or null), and // specific versions afterwards ("3.0.0", "3.1.0" etc.). // see o.a.l.util.Version. - private Version version; + private final Version version; + + // Tracks the minimum version that contributed documents to a segment. For + // flush segments, that is the version that wrote it. For merged segments, + // this is the minimum minVersion of all the segments that have been merged + // into this segment + Version minVersion; void setDiagnostics(Map diagnostics) { this.diagnostics = Objects.requireNonNull(diagnostics); @@ -94,12 +100,13 @@ public final class SegmentInfo { *

    Note: this is public only to allow access from * the codecs package.

    */ - public SegmentInfo(Directory dir, Version version, String name, int maxDoc, + public SegmentInfo(Directory dir, Version version, Version minVersion, String name, int maxDoc, boolean isCompoundFile, Codec codec, Map diagnostics, byte[] id, Map attributes, Sort indexSort) { assert !(dir instanceof TrackingDirectoryWrapper); this.dir = Objects.requireNonNull(dir); this.version = Objects.requireNonNull(version); + this.minVersion = minVersion; this.name = Objects.requireNonNull(name); this.maxDoc = maxDoc; this.isCompoundFile = isCompoundFile; @@ -233,6 +240,14 @@ public final class SegmentInfo { return version; } + /** + * Return the minimum Lucene version that contributed documents to this + * segment, or {@code null} if it is unknown. + */ + public Version getMinVersion() { + return minVersion; + } + /** Return the id that uniquely identifies this segment. */ public byte[] getId() { return id.clone(); diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java index 12305d0f12c..e463259f9de 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java @@ -161,16 +161,19 @@ public final class SegmentInfos implements Cloneable, Iterable Version.LATEST.major) { + throw new IllegalArgumentException("indexCreatedVersionMajor is in the future: " + indexCreatedVersionMajor); } - this.indexCreatedVersion = indexCreatedVersion; + if (indexCreatedVersionMajor < 6) { + throw new IllegalArgumentException("indexCreatedVersionMajor must be >= 6, got: " + indexCreatedVersionMajor); + } + this.indexCreatedVersionMajor = indexCreatedVersionMajor; } /** Returns {@link SegmentCommitInfo} at the provided @@ -314,24 +317,9 @@ public final class SegmentInfos implements Cloneable, Iterable= VERSION_70) { - byte b = input.readByte(); - switch (b) { - case 0: - // version is not known: pre-7.0 index that has been modified since the 7.0 upgrade - indexCreatedVersion = null; - break; - case 1: - // version is known: index has been created on or after 7.0 - indexCreatedVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt()); - break; - default: - throw new CorruptIndexException("Illegal byte value for a boolean: " + b + ", expected 0 or 1", input); - } - } else { - // pre-7.0 index that has not been modified since the 7.0 upgrade - indexCreatedVersion = null; + indexCreatedVersion = input.readVInt(); } SegmentInfos infos = new SegmentInfos(indexCreatedVersion); @@ -399,6 +387,14 @@ public final class SegmentInfos implements Cloneable, Iterable= 7 && merge.info.info.minVersion == null) { + throw new IllegalArgumentException("All segments must record the minVersion for indices created on or after Lucene 7"); + } + final Set mergedAway = new HashSet<>(merge.segments); boolean inserted = false; int newSegIdx = 0; @@ -981,6 +975,10 @@ public final class SegmentInfos implements Cloneable, Iterable= 7 && si.info.minVersion == null) { + throw new IllegalArgumentException("All segments must record the minVersion for indices created on or after Lucene 7"); + } + segments.add(si); } @@ -1038,10 +1036,11 @@ public final class SegmentInfos implements Cloneable, Iterable(), null); + final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); this.write(si, fieldInfos, dir, fields); final FieldsProducer reader = codec.postingsFormat().fieldsProducer(new SegmentReadState(dir, si, fieldInfos, newIOContext(random()))); @@ -274,7 +274,7 @@ public class TestCodecs extends LuceneTestCase { } Codec codec = Codec.getDefault(); - final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); + final SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, SEGMENT, 10000, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); this.write(si, fieldInfos, dir, fields); if (VERBOSE) { diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java index 34bde51f268..7b7ec993a56 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java @@ -55,6 +55,7 @@ import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.TestUtil; +import org.apache.lucene.util.Version; // TODO: // - old parallel indices are only pruned on commit/close; can we do it on refresh? @@ -414,7 +415,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase { SegmentInfos infos = SegmentInfos.readLatestCommit(dir); assert infos.size() == 1; - final LeafReader parLeafReader = new SegmentReader(infos.info(0), IOContext.DEFAULT); + final LeafReader parLeafReader = new SegmentReader(infos.info(0), Version.LATEST.major, IOContext.DEFAULT); //checkParallelReader(leaf, parLeafReader, schemaGen); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java index 8b24b4d7bc5..a20dece4e22 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java @@ -213,12 +213,12 @@ public class TestDoc extends LuceneTestCase { private SegmentCommitInfo merge(Directory dir, SegmentCommitInfo si1, SegmentCommitInfo si2, String merged, boolean useCompoundFile) throws Exception { IOContext context = newIOContext(random(), new IOContext(new MergeInfo(-1, -1, false, -1))); - SegmentReader r1 = new SegmentReader(si1, context); - SegmentReader r2 = new SegmentReader(si2, context); + SegmentReader r1 = new SegmentReader(si1, Version.LATEST.major, context); + SegmentReader r2 = new SegmentReader(si2, Version.LATEST.major, context); final Codec codec = Codec.getDefault(); TrackingDirectoryWrapper trackingDir = new TrackingDirectoryWrapper(si1.info.dir); - final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); + final SegmentInfo si = new SegmentInfo(si1.info.dir, Version.LATEST, null, merged, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); SegmentMerger merger = new SegmentMerger(Arrays.asList(r1, r2), si, InfoStream.getDefault(), trackingDir, @@ -244,7 +244,7 @@ public class TestDoc extends LuceneTestCase { private void printSegment(PrintWriter out, SegmentCommitInfo si) throws Exception { - SegmentReader reader = new SegmentReader(si, newIOContext(random())); + SegmentReader reader = new SegmentReader(si, Version.LATEST.major, newIOContext(random())); for (int i = 0; i < reader.numDocs(); i++) out.println(reader.document(i)); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java index a814c4c351d..da96f985339 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java @@ -33,6 +33,7 @@ import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; +import org.apache.lucene.util.Version; public class TestDocumentWriter extends LuceneTestCase { private Directory dir; @@ -62,7 +63,7 @@ public class TestDocumentWriter extends LuceneTestCase { SegmentCommitInfo info = writer.newestSegment(); writer.close(); //After adding the document, we should be able to read it back in - SegmentReader reader = new SegmentReader(info, newIOContext(random())); + SegmentReader reader = new SegmentReader(info, Version.LATEST.major, newIOContext(random())); assertTrue(reader != null); Document doc = reader.document(0); assertTrue(doc != null); @@ -123,7 +124,7 @@ public class TestDocumentWriter extends LuceneTestCase { writer.commit(); SegmentCommitInfo info = writer.newestSegment(); writer.close(); - SegmentReader reader = new SegmentReader(info, newIOContext(random())); + SegmentReader reader = new SegmentReader(info, Version.LATEST.major, newIOContext(random())); PostingsEnum termPositions = MultiFields.getTermPositionsEnum(reader, "repeated", new BytesRef("repeated")); assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); @@ -194,7 +195,7 @@ public class TestDocumentWriter extends LuceneTestCase { writer.commit(); SegmentCommitInfo info = writer.newestSegment(); writer.close(); - SegmentReader reader = new SegmentReader(info, newIOContext(random())); + SegmentReader reader = new SegmentReader(info, Version.LATEST.major, newIOContext(random())); PostingsEnum termPositions = MultiFields.getTermPositionsEnum(reader, "f1", new BytesRef("a")); assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); @@ -236,7 +237,7 @@ public class TestDocumentWriter extends LuceneTestCase { writer.commit(); SegmentCommitInfo info = writer.newestSegment(); writer.close(); - SegmentReader reader = new SegmentReader(info, newIOContext(random())); + SegmentReader reader = new SegmentReader(info, Version.LATEST.major, newIOContext(random())); PostingsEnum termPositions = reader.postings(new Term("preanalyzed", "term1"), PostingsEnum.ALL); assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java index 4275056d7ae..4ef580bf071 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexSorting.java @@ -2094,7 +2094,7 @@ public class TestIndexSorting extends LuceneTestCase { if (VERBOSE) { System.out.println("TEST: now compare r1=" + r1 + " r2=" + r2); } - assertEquals(sort, getOnlyLeafReader(r2).getIndexSort()); + assertEquals(sort, getOnlyLeafReader(r2).getMetaData().getSort()); assertReaderEquals("left: sorted by hand; right: sorted by Lucene", r1, r2); IOUtils.close(w1, w2, r1, r2, dir1, dir2); } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java index 660280b4dd7..67add26e9c9 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java @@ -2805,7 +2805,7 @@ public class TestIndexWriter extends LuceneTestCase { IndexWriter w = new IndexWriter(dir, newIndexWriterConfig()); w.commit(); w.close(); - assertEquals(Version.LATEST, SegmentInfos.readLatestCommit(dir).getIndexCreatedVersion()); + assertEquals(Version.LATEST.major, SegmentInfos.readLatestCommit(dir).getIndexCreatedVersionMajor()); dir.close(); } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java index 4e884dfb888..359e7d02739 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterThreadsToSegments.java @@ -39,6 +39,7 @@ import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.TestUtil; +import org.apache.lucene.util.Version; public class TestIndexWriterThreadsToSegments extends LuceneTestCase { @@ -331,7 +332,7 @@ public class TestIndexWriterThreadsToSegments extends LuceneTestCase { SegmentInfo si = TestUtil.getDefaultCodec().segmentInfoFormat().read(dir, segName, id, IOContext.DEFAULT); si.setCodec(codec); SegmentCommitInfo sci = new SegmentCommitInfo(si, 0, -1, -1, -1); - SegmentReader sr = new SegmentReader(sci, IOContext.DEFAULT); + SegmentReader sr = new SegmentReader(sci, Version.LATEST.major, IOContext.DEFAULT); try { thread0Count += sr.docFreq(new Term("field", "threadID0")); thread1Count += sr.docFreq(new Term("field", "threadID1")); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestOneMergeWrappingMergePolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestOneMergeWrappingMergePolicy.java index f27437fcf6e..c85646cfc02 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestOneMergeWrappingMergePolicy.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestOneMergeWrappingMergePolicy.java @@ -124,6 +124,7 @@ public class TestOneMergeWrappingMergePolicy extends LuceneTestCase { final SegmentInfo si = new SegmentInfo( dir, // dir Version.LATEST, // version + Version.LATEST, // min version TestUtil.randomSimpleString(random()), // name random().nextInt(), // maxDoc random().nextBoolean(), // isCompoundFile diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java index 755278641ea..128601e4e11 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java @@ -30,13 +30,15 @@ import java.util.Collections; public class TestSegmentInfos extends LuceneTestCase { public void testIllegalCreatedVersion() { - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new SegmentInfos(Version.LUCENE_6_5_0)); - assertEquals("indexCreatedVersion may only be non-null if the index was created on or after 7.0, got 6.5.0", e.getMessage()); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new SegmentInfos(5)); + assertEquals("indexCreatedVersionMajor must be >= 6, got: 5", e.getMessage()); + e = expectThrows(IllegalArgumentException.class, () -> new SegmentInfos(8)); + assertEquals("indexCreatedVersionMajor is in the future: 8", e.getMessage()); } // LUCENE-5954 public void testVersionsNoSegments() throws IOException { - SegmentInfos sis = new SegmentInfos(Version.LATEST); + SegmentInfos sis = new SegmentInfos(Version.LATEST.major); BaseDirectoryWrapper dir = newDirectory(); dir.setCheckIndexOnClose(false); sis.commit(dir); @@ -53,8 +55,8 @@ public class TestSegmentInfos extends LuceneTestCase { byte id[] = StringHelper.randomId(); Codec codec = Codec.getDefault(); - SegmentInfos sis = new SegmentInfos(Version.LATEST); - SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_0", 1, false, Codec.getDefault(), + SegmentInfos sis = new SegmentInfos(Version.LATEST.major); + SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_7_0_0, Version.LUCENE_7_0_0, "_0", 1, false, Codec.getDefault(), Collections.emptyMap(), id, Collections.emptyMap(), null); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); @@ -63,7 +65,7 @@ public class TestSegmentInfos extends LuceneTestCase { sis.add(commitInfo); sis.commit(dir); sis = SegmentInfos.readLatestCommit(dir); - assertEquals(Version.LUCENE_6_0_0, sis.getMinSegmentLuceneVersion()); + assertEquals(Version.LUCENE_7_0_0, sis.getMinSegmentLuceneVersion()); assertEquals(Version.LATEST, sis.getCommitLuceneVersion()); dir.close(); } @@ -75,15 +77,15 @@ public class TestSegmentInfos extends LuceneTestCase { byte id[] = StringHelper.randomId(); Codec codec = Codec.getDefault(); - SegmentInfos sis = new SegmentInfos(Version.LATEST); - SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_0", 1, false, Codec.getDefault(), + SegmentInfos sis = new SegmentInfos(Version.LATEST.major); + SegmentInfo info = new SegmentInfo(dir, Version.LUCENE_7_0_0, Version.LUCENE_7_0_0, "_0", 1, false, Codec.getDefault(), Collections.emptyMap(), id, Collections.emptyMap(), null); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, -1, -1, -1); sis.add(commitInfo); - info = new SegmentInfo(dir, Version.LUCENE_6_0_0, "_1", 1, false, Codec.getDefault(), + info = new SegmentInfo(dir, Version.LUCENE_7_0_0, Version.LUCENE_7_0_0, "_1", 1, false, Codec.getDefault(), Collections.emptyMap(), id, Collections.emptyMap(), null); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); @@ -92,7 +94,7 @@ public class TestSegmentInfos extends LuceneTestCase { sis.commit(dir); sis = SegmentInfos.readLatestCommit(dir); - assertEquals(Version.LUCENE_6_0_0, sis.getMinSegmentLuceneVersion()); + assertEquals(Version.LUCENE_7_0_0, sis.getMinSegmentLuceneVersion()); assertEquals(Version.LATEST, sis.getCommitLuceneVersion()); dir.close(); } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java index 1ef37c0892f..6d0e04bbb2c 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentMerger.java @@ -60,8 +60,8 @@ public class TestSegmentMerger extends LuceneTestCase { SegmentCommitInfo info1 = DocHelper.writeDoc(random(), merge1Dir, doc1); DocHelper.setupDoc(doc2); SegmentCommitInfo info2 = DocHelper.writeDoc(random(), merge2Dir, doc2); - reader1 = new SegmentReader(info1, newIOContext(random())); - reader2 = new SegmentReader(info2, newIOContext(random())); + reader1 = new SegmentReader(info1, Version.LATEST.major, newIOContext(random())); + reader2 = new SegmentReader(info2, Version.LATEST.major, newIOContext(random())); } @Override @@ -84,7 +84,7 @@ public class TestSegmentMerger extends LuceneTestCase { public void testMerge() throws IOException { final Codec codec = Codec.getDefault(); - final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); + final SegmentInfo si = new SegmentInfo(mergedDir, Version.LATEST, null, mergedSegment, -1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); SegmentMerger merger = new SegmentMerger(Arrays.asList(reader1, reader2), si, InfoStream.getDefault(), mergedDir, @@ -97,6 +97,7 @@ public class TestSegmentMerger extends LuceneTestCase { SegmentReader mergedReader = new SegmentReader(new SegmentCommitInfo( mergeState.segmentInfo, 0, -1L, -1L, -1L), + Version.LATEST.major, newIOContext(random())); assertTrue(mergedReader != null); assertTrue(mergedReader.numDocs() == 2); diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java index f008afe7a79..543495623fd 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentReader.java @@ -29,6 +29,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; +import org.apache.lucene.util.Version; public class TestSegmentReader extends LuceneTestCase { private Directory dir; @@ -42,7 +43,7 @@ public class TestSegmentReader extends LuceneTestCase { dir = newDirectory(); DocHelper.setupDoc(testDoc); SegmentCommitInfo info = DocHelper.writeDoc(random(), dir, testDoc); - reader = new SegmentReader(info, IOContext.READ); + reader = new SegmentReader(info, Version.LATEST.major, IOContext.READ); } @Override diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java index 7acf3e44973..f1271b8aa55 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentTermDocs.java @@ -27,6 +27,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; +import org.apache.lucene.util.Version; public class TestSegmentTermDocs extends LuceneTestCase { private Document testDoc = new Document(); @@ -53,7 +54,7 @@ public class TestSegmentTermDocs extends LuceneTestCase { public void testTermDocs() throws IOException { //After adding the document, we should be able to read it back in - SegmentReader reader = new SegmentReader(info, newIOContext(random())); + SegmentReader reader = new SegmentReader(info, Version.LATEST.major, newIOContext(random())); assertTrue(reader != null); TermsEnum terms = reader.fields().terms(DocHelper.TEXT_FIELD_2_KEY).iterator(); @@ -71,7 +72,7 @@ public class TestSegmentTermDocs extends LuceneTestCase { public void testBadSeek() throws IOException { { //After adding the document, we should be able to read it back in - SegmentReader reader = new SegmentReader(info, newIOContext(random())); + SegmentReader reader = new SegmentReader(info, Version.LATEST.major, newIOContext(random())); assertTrue(reader != null); PostingsEnum termDocs = TestUtil.docs(random(), reader, "textField2", @@ -84,7 +85,7 @@ public class TestSegmentTermDocs extends LuceneTestCase { } { //After adding the document, we should be able to read it back in - SegmentReader reader = new SegmentReader(info, newIOContext(random())); + SegmentReader reader = new SegmentReader(info, Version.LATEST.major, newIOContext(random())); assertTrue(reader != null); PostingsEnum termDocs = TestUtil.docs(random(), reader, "junk", diff --git a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java index 2e3cdab4816..3a723cf8113 100644 --- a/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java +++ b/lucene/highlighter/src/java/org/apache/lucene/search/highlight/TermVectorLeafReader.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.Fields; +import org.apache.lucene.index.LeafMetaData; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.NumericDocValues; @@ -34,8 +35,8 @@ import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.StoredFieldVisitor; import org.apache.lucene.index.Terms; -import org.apache.lucene.search.Sort; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.Version; /** * Wraps a Terms with a {@link org.apache.lucene.index.LeafReader}, typically from term vectors. @@ -165,8 +166,8 @@ public class TermVectorLeafReader extends LeafReader { } @Override - public Sort getIndexSort() { - return null; + public LeafMetaData getMetaData() { + return new LeafMetaData(Version.LATEST.major, null, null); } @Override diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java index 0c8ea6d35e3..4bd72e993d6 100644 --- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java +++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java @@ -40,7 +40,6 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.SimpleCollector; -import org.apache.lucene.search.Sort; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.util.ArrayUtil; @@ -58,6 +57,7 @@ import org.apache.lucene.util.IntBlockPool.SliceWriter; import org.apache.lucene.util.RecyclingByteBlockAllocator; import org.apache.lucene.util.RecyclingIntBlockAllocator; import org.apache.lucene.util.StringHelper; +import org.apache.lucene.util.Version; /** * High-performance single-document main memory Apache Lucene fulltext search index. @@ -1625,8 +1625,8 @@ public class MemoryIndex { } @Override - public Sort getIndexSort() { - return null; + public LeafMetaData getMetaData() { + return new LeafMetaData(Version.LATEST.major, Version.LATEST, null); } @Override diff --git a/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java b/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java index a3d720d8b07..892564826f3 100644 --- a/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java +++ b/lucene/misc/src/java/org/apache/lucene/index/IndexSplitter.java @@ -133,13 +133,13 @@ public class IndexSplitter { public void split(Path destDir, String[] segs) throws IOException { Files.createDirectories(destDir); FSDirectory destFSDir = FSDirectory.open(destDir); - SegmentInfos destInfos = new SegmentInfos(infos.getIndexCreatedVersion()); + SegmentInfos destInfos = new SegmentInfos(infos.getIndexCreatedVersionMajor()); destInfos.counter = infos.counter; for (String n : segs) { SegmentCommitInfo infoPerCommit = getInfo(n); SegmentInfo info = infoPerCommit.info; // Same info just changing the dir: - SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.name, info.maxDoc(), + SegmentInfo newInfo = new SegmentInfo(destFSDir, info.getVersion(), info.getMinVersion(), info.name, info.maxDoc(), info.getUseCompoundFile(), info.getCodec(), info.getDiagnostics(), info.getId(), new HashMap<>(), null); destInfos.add(new SegmentCommitInfo(newInfo, infoPerCommit.getDelCount(), infoPerCommit.getDelGen(), infoPerCommit.getFieldInfosGen(), diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java index 53199566a18..ff82d18ad38 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java @@ -139,7 +139,7 @@ public abstract class ReplicaNode extends Node { SegmentInfos infos; if (segmentsFileName == null) { // No index here yet: - infos = new SegmentInfos(Version.LATEST); + infos = new SegmentInfos(Version.LATEST.major); message("top: init: no segments in index"); } else { message("top: init: read existing segments commit " + segmentsFileName); diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java index 30788e84065..07a3f068a32 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java @@ -638,7 +638,8 @@ public abstract class BaseCompoundFormatTestCase extends BaseIndexFileFormatTest /** Returns a new fake segment */ protected static SegmentInfo newSegmentInfo(Directory dir, String name) { - return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); + Version minVersion = random().nextBoolean() ? null : Version.LATEST; + return new SegmentInfo(dir, Version.LATEST, minVersion, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); } /** Creates a file of the specified size with random data. */ diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java index 528e92afc17..9363ce63fe3 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseFieldInfoFormatTestCase.java @@ -347,7 +347,8 @@ public abstract class BaseFieldInfoFormatTestCase extends BaseIndexFileFormatTes /** Returns a new fake segment */ protected static SegmentInfo newSegmentInfo(Directory dir, String name) { - return new SegmentInfo(dir, Version.LATEST, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); + Version minVersion = random().nextBoolean() ? null : Version.LATEST; + return new SegmentInfo(dir, Version.LATEST, minVersion, name, 10000, false, Codec.getDefault(), Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); } @Override diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java index 3097cdf2c89..e45011428ca 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java @@ -147,6 +147,22 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase { /** Returns the codec to run tests against */ protected abstract Codec getCodec(); + /** Returns the major version that this codec is compatible with. */ + protected int getCreatedVersionMajor() { + return Version.LATEST.major; + } + + /** Set the created version of the given {@link Directory} and return it. */ + protected final D applyCreatedVersionMajor(D d) throws IOException { + if (SegmentInfos.getLastCommitGeneration(d) != -1) { + throw new IllegalArgumentException("Cannot set the created version on a Directory that already has segments"); + } + if (getCreatedVersionMajor() != Version.LATEST.major || random().nextBoolean()) { + new SegmentInfos(getCreatedVersionMajor()).commit(d); + } + return d; + } + private Codec savedCodec; public void setUp() throws Exception { @@ -195,7 +211,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase { /** The purpose of this test is to make sure that bulk merge doesn't accumulate useless data over runs. */ public void testMergeStability() throws Exception { assumeTrue("merge is not stable", mergeIsStable()); - Directory dir = newDirectory(); + Directory dir = applyCreatedVersionMajor(newDirectory()); // do not use newMergePolicy that might return a MockMergePolicy that ignores the no-CFS ratio // do not use RIW which will change things up! @@ -214,7 +230,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase { w.close(); DirectoryReader reader = DirectoryReader.open(dir); - Directory dir2 = newDirectory(); + Directory dir2 = applyCreatedVersionMajor(newDirectory()); mp = newTieredMergePolicy(); mp.setNoCFSRatio(0); cfg = new IndexWriterConfig(new MockAnalyzer(random())).setUseCompoundFile(false).setMergePolicy(mp); @@ -245,7 +261,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase { avoidCodecs.add(new MockRandomPostingsFormat().getName()); Codec.setDefault(new RandomCodec(random(), avoidCodecs)); } - Directory dir = newDirectory(); + Directory dir = applyCreatedVersionMajor(newDirectory()); IndexWriterConfig cfg = newIndexWriterConfig(new MockAnalyzer(random())); IndexWriter w = new IndexWriter(dir, cfg); // we need to index enough documents so that constant overhead doesn't dominate @@ -286,7 +302,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase { /** Calls close multiple times on closeable codec apis */ public void testMultiClose() throws IOException { // first make a one doc index - Directory oneDocIndex = newDirectory(); + Directory oneDocIndex = applyCreatedVersionMajor(newDirectory()); IndexWriter iw = new IndexWriter(oneDocIndex, new IndexWriterConfig(new MockAnalyzer(random()))); Document oneDoc = new Document(); FieldType customType = new FieldType(TextField.TYPE_STORED); @@ -303,7 +319,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase { Directory dir = newFSDirectory(createTempDir("justSoYouGetSomeChannelErrors")); Codec codec = getCodec(); - SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", 1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); + SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "_0", 1, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); FieldInfo proto = oneDocReader.getFieldInfos().fieldInfo("field"); FieldInfo field = new FieldInfo(proto.name, proto.number, proto.hasVectors(), proto.omitsNorms(), proto.hasPayloads(), proto.getIndexOptions(), proto.getDocValuesType(), proto.getDocValuesGen(), new HashMap<>(), @@ -499,7 +515,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase { // codec does not corrupt the index or leak file handles. public void testRandomExceptions() throws Exception { // disable slow things: we don't rely upon sleeps here. - MockDirectoryWrapper dir = newMockDirectory(); + MockDirectoryWrapper dir = applyCreatedVersionMajor(newMockDirectory()); dir.setThrottling(MockDirectoryWrapper.Throttling.NEVER); dir.setUseSlowOpenClosers(false); dir.setRandomIOExceptionRate(0.001); // more rare diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java index cd62218056a..370d009172a 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java @@ -449,7 +449,7 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas norms[i] = longs.getAsLong(); } - Directory dir = newDirectory(); + Directory dir = applyCreatedVersionMajor(newDirectory()); Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false); IndexWriterConfig conf = newIndexWriterConfig(analyzer);conf.setMergePolicy(NoMergePolicy.INSTANCE); conf.setSimilarity(new CannedNormSimilarity(norms)); @@ -585,7 +585,7 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas * */ public void testUndeadNorms() throws Exception { - Directory dir = newDirectory(); + Directory dir = applyCreatedVersionMajor(newDirectory()); RandomIndexWriter w = new RandomIndexWriter(random(), dir); int numDocs = atLeast(500); List toDelete = new ArrayList<>(); @@ -646,7 +646,7 @@ public abstract class BaseNormsFormatTestCase extends BaseIndexFileFormatTestCas norms[i] = random().nextLong(); } - Directory dir = newDirectory(); + Directory dir = applyCreatedVersionMajor(newDirectory()); Analyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false); IndexWriterConfig conf = newIndexWriterConfig(analyzer);conf.setMergePolicy(NoMergePolicy.INSTANCE); conf.setSimilarity(new CannedNormSimilarity(norms)); diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java index ae5416fa479..29f31d6b550 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseSegmentInfoFormatTestCase.java @@ -48,13 +48,18 @@ import org.apache.lucene.util.Version; * if there is some bug in a given si Format that this * test fails to catch then this test needs to be improved! */ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatTestCase { - + + /** Whether this format records min versions. */ + protected boolean supportsMinVersion() { + return true; + } + /** Test files map */ public void testFiles() throws Exception { Directory dir = newDirectory(); Codec codec = getCodec(); byte id[] = StringHelper.randomId(); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, Collections.emptyMap(), id, new HashMap<>(), null); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); @@ -68,7 +73,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT Directory dir = newDirectory(); Codec codec = getCodec(); byte id[] = StringHelper.randomId(); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, Collections.emptyMap(), id, new HashMap<>(), null); Set originalFiles = Collections.singleton("_123.a"); info.setFiles(originalFiles); @@ -97,7 +102,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT Map diagnostics = new HashMap<>(); diagnostics.put("key1", "value1"); diagnostics.put("key2", "value2"); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, diagnostics, id, new HashMap<>(), null); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); @@ -120,7 +125,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT Map attributes = new HashMap<>(); attributes.put("key1", "value1"); attributes.put("key2", "value2"); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, Collections.emptyMap(), id, attributes, null); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); @@ -140,7 +145,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT Codec codec = getCodec(); Directory dir = newDirectory(); byte id[] = StringHelper.randomId(); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, Collections.emptyMap(), id, new HashMap<>(), null); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); @@ -153,15 +158,22 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT public void testVersions() throws Exception { Codec codec = getCodec(); for (Version v : getVersions()) { - Directory dir = newDirectory(); - byte id[] = StringHelper.randomId(); - SegmentInfo info = new SegmentInfo(dir, v, "_123", 1, false, codec, - Collections.emptyMap(), id, new HashMap<>(), null); - info.setFiles(Collections.emptySet()); - codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); - SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT); - assertEquals(info2.getVersion(), v); - dir.close(); + for (Version minV : new Version[] { v, null}) { + Directory dir = newDirectory(); + byte id[] = StringHelper.randomId(); + SegmentInfo info = new SegmentInfo(dir, v, minV, "_123", 1, false, codec, + Collections.emptyMap(), id, new HashMap<>(), null); + info.setFiles(Collections.emptySet()); + codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); + SegmentInfo info2 = codec.segmentInfoFormat().read(dir, "_123", id, IOContext.DEFAULT); + assertEquals(info2.getVersion(), v); + if (supportsMinVersion()) { + assertEquals(info2.getMinVersion(), minV); + } else { + assertEquals(info2.getMinVersion(), null); + } + dir.close(); + } } } @@ -262,7 +274,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT Directory dir = newDirectory(); Codec codec = getCodec(); byte id[] = StringHelper.randomId(); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, Collections.emptyMap(), id, new HashMap<>(), sort); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); @@ -292,7 +304,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT dir.failOn(fail); Codec codec = getCodec(); byte id[] = StringHelper.randomId(); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, Collections.emptyMap(), id, new HashMap<>(), null); info.setFiles(Collections.emptySet()); @@ -325,7 +337,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT dir.failOn(fail); Codec codec = getCodec(); byte id[] = StringHelper.randomId(); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, Collections.emptyMap(), id, new HashMap<>(), null); info.setFiles(Collections.emptySet()); @@ -358,7 +370,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT dir.failOn(fail); Codec codec = getCodec(); byte id[] = StringHelper.randomId(); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, Collections.emptyMap(), id, new HashMap<>(), null); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); @@ -392,7 +404,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT dir.failOn(fail); Codec codec = getCodec(); byte id[] = StringHelper.randomId(); - SegmentInfo info = new SegmentInfo(dir, getVersions()[0], "_123", 1, false, codec, + SegmentInfo info = new SegmentInfo(dir, getVersions()[0], getVersions()[0], "_123", 1, false, codec, Collections.emptyMap(), id, new HashMap<>(), null); info.setFiles(Collections.emptySet()); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); @@ -442,7 +454,7 @@ public abstract class BaseSegmentInfoFormatTestCase extends BaseIndexFileFormatT TestUtil.randomUnicodeString(random())); } - SegmentInfo info = new SegmentInfo(dir, version, name, docCount, isCompoundFile, codec, diagnostics, id, attributes, null); + SegmentInfo info = new SegmentInfo(dir, version, null, name, docCount, isCompoundFile, codec, diagnostics, id, attributes, null); info.setFiles(files); codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT); SegmentInfo info2 = codec.segmentInfoFormat().read(dir, name, id, IOContext.DEFAULT); diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java index d4159279311..d5eb105c9b4 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomPostingsTester.java @@ -611,7 +611,7 @@ public class RandomPostingsTester { // maxAllowed = the "highest" we can index, but we will still // randomly index at lower IndexOption public FieldsProducer buildIndex(Codec codec, Directory dir, IndexOptions maxAllowed, boolean allowPayloads, boolean alwaysTestMax) throws IOException { - SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, "_0", maxDoc, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); + SegmentInfo segmentInfo = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "_0", maxDoc, false, codec, Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null); int maxIndexOption = Arrays.asList(IndexOptions.values()).indexOf(maxAllowed); if (LuceneTestCase.VERBOSE) { diff --git a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java index c40f875fc5e..bd23fd210b3 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java +++ b/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java @@ -26,6 +26,7 @@ import org.apache.lucene.index.BinaryDocValues; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.Fields; +import org.apache.lucene.index.LeafMetaData; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; @@ -39,6 +40,7 @@ import org.apache.lucene.index.StoredFieldVisitor; import org.apache.lucene.index.Terms; import org.apache.lucene.util.Bits; import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.Version; import junit.framework.Assert; @@ -260,8 +262,8 @@ public class QueryUtils { protected void doClose() throws IOException {} @Override - public Sort getIndexSort() { - return null; + public LeafMetaData getMetaData() { + return new LeafMetaData(Version.LATEST.major, Version.LATEST, null); } @Override diff --git a/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java b/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java index c445cdf610f..ab7313fbf5e 100644 --- a/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java +++ b/solr/core/src/java/org/apache/solr/index/SlowCompositeReaderWrapper.java @@ -23,9 +23,8 @@ import java.util.Map; import org.apache.lucene.index.*; import org.apache.lucene.index.MultiDocValues.MultiSortedDocValues; import org.apache.lucene.index.MultiDocValues.OrdinalMap; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.search.Sort; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.Version; /** * This class forces a composite reader (eg a {@link @@ -47,6 +46,7 @@ public final class SlowCompositeReaderWrapper extends LeafReader { private final CompositeReader in; private final Fields fields; + private final LeafMetaData metaData; /** This method is sugar for getting an {@link LeafReader} from * an {@link IndexReader} of any kind. If the reader is already atomic, @@ -66,6 +66,17 @@ public final class SlowCompositeReaderWrapper extends LeafReader { in = reader; fields = MultiFields.getFields(in); in.registerParentReader(this); + if (reader.leaves().isEmpty()) { + metaData = new LeafMetaData(Version.LATEST.major, Version.LATEST, null); + } else { + Version minVersion = reader.leaves().stream() + .map(LeafReaderContext::reader) + .map(LeafReader::getMetaData) + .map(LeafMetaData::getMinVersion) + .reduce((v1, v2) -> v1 == null ? null : v2 == null ? null : v2.onOrAfter(v1) ? v1 : v2) + .get(); + metaData = new LeafMetaData(reader.leaves().get(0).reader().getMetaData().getCreatedVersionMajor(), minVersion, null); + } } @Override @@ -263,7 +274,7 @@ public final class SlowCompositeReaderWrapper extends LeafReader { } @Override - public Sort getIndexSort() { - return null; + public LeafMetaData getMetaData() { + return metaData; } } diff --git a/solr/core/src/test/org/apache/solr/search/TestDocSet.java b/solr/core/src/test/org/apache/solr/search/TestDocSet.java index db6523e9331..e5cc6ebde4c 100644 --- a/solr/core/src/test/org/apache/solr/search/TestDocSet.java +++ b/solr/core/src/test/org/apache/solr/search/TestDocSet.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.BinaryDocValues; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; import org.apache.lucene.index.Fields; +import org.apache.lucene.index.LeafMetaData; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.LeafReader; @@ -38,11 +39,11 @@ import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.StoredFieldVisitor; import org.apache.lucene.search.DocIdSet; import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.Sort; import org.apache.lucene.util.BitSetIterator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.FixedBitSet; import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.Version; /** * @@ -455,8 +456,8 @@ public class TestDocSet extends LuceneTestCase { } @Override - public Sort getIndexSort() { - return null; + public LeafMetaData getMetaData() { + return new LeafMetaData(Version.LATEST.major, Version.LATEST, null); } @Override From 4c737b8df9b130cf530d17271576730e21d5b4cc Mon Sep 17 00:00:00 2001 From: Mikhail Khludnev Date: Tue, 4 Apr 2017 11:01:19 +0300 Subject: [PATCH 114/563] SOLR-9745: check exit code only if process has finished --- solr/core/src/java/org/apache/solr/util/SolrCLI.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 3572ea43ca6..7be76af6ed1 100644 --- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java +++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java @@ -2939,7 +2939,7 @@ public class SolrCLI { // safe to ignore ... Thread.interrupted(); } - if (handler.getExitValue() != 0) { + if (handler.hasResult() && handler.getExitValue() != 0) { throw new Exception("Failed to start Solr using command: "+startCmd+" Exception : "+handler.getException()); } } else { From ee98cdc79014af0bd309ab4298fdbaeb38ee402b Mon Sep 17 00:00:00 2001 From: Shalin Shekhar Mangar Date: Tue, 4 Apr 2017 14:20:31 +0530 Subject: [PATCH 115/563] SOLR-10416: The JSON output of /admin/metrics is fixed to write the container as a map (SimpleOrderedMap) instead of an array (NamedList) --- solr/CHANGES.txt | 3 +++ .../java/org/apache/solr/handler/admin/MetricsHandler.java | 3 ++- .../org/apache/solr/handler/admin/MetricsHandlerTest.java | 7 ++++--- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 8a5c3e8d206..83f14407fe4 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -198,6 +198,9 @@ Bug Fixes * SOLR-10383: Fix debug related NullPointerException in solr/contrib/ltr OriginalScoreFeature class. (Vitezslav Zak, Christine Poerschke) +* SOLR-10416: The JSON output of /admin/metrics is fixed to write the container as a + map (SimpleOrderedMap) instead of an array (NamedList). (shalin) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. 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 25f317c00d7..4dc86d97855 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 @@ -33,6 +33,7 @@ import com.codahale.metrics.MetricRegistry; import com.codahale.metrics.Timer; import org.apache.solr.common.SolrException; import org.apache.solr.common.util.NamedList; +import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.common.util.StrUtils; import org.apache.solr.core.CoreContainer; import org.apache.solr.handler.RequestHandlerBase; @@ -79,7 +80,7 @@ public class MetricsHandler extends RequestHandlerBase implements PermissionName List metricFilters = metricTypes.stream().map(MetricType::asMetricFilter).collect(Collectors.toList()); Set requestedRegistries = parseRegistries(req); - NamedList response = new NamedList(); + NamedList response = new SimpleOrderedMap(); for (String registryName : requestedRegistries) { MetricRegistry registry = metricManager.registry(registryName); response.add(registryName, MetricUtils.toNamedList(registry, metricFilters, mustMatchFilter, false, 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 81e14d944a5..2f849977e9f 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 @@ -22,6 +22,7 @@ import java.util.Map; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.util.NamedList; +import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.response.SolrQueryResponse; import org.junit.BeforeClass; import org.junit.Test; @@ -147,9 +148,9 @@ public class MetricsHandlerTest extends SolrTestCaseJ4 { handler.handleRequestBody(req(CommonParams.QT, "/admin/metrics", CommonParams.WT, "json", "group", "node", "type", "timer", "prefix", "CONTAINER.cores"), resp); values = resp.getValues(); assertNotNull(values.get("metrics")); - values = (NamedList) values.get("metrics"); - assertEquals(1, values.size()); - assertEquals(0, ((NamedList)values.get("solr.node")).size()); + SimpleOrderedMap map = (SimpleOrderedMap) values.get("metrics"); + assertEquals(1, map.size()); + assertEquals(0, ((NamedList)map.get("solr.node")).size()); } @Test From 05749d0694ee6dce44764cae965c09857dc8975a Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Tue, 4 Apr 2017 12:52:09 +0100 Subject: [PATCH 116/563] SOLR-10394: Rename getSortWithinGroup to getWithinGroupSort in search.grouping.Command class. (Judith Silverman, Christine Poerschke) --- solr/CHANGES.txt | 3 +++ .../src/java/org/apache/solr/search/grouping/Command.java | 2 +- .../java/org/apache/solr/search/grouping/CommandHandler.java | 4 ++-- .../search/grouping/distributed/command/QueryCommand.java | 2 +- .../distributed/command/SearchGroupsFieldCommand.java | 2 +- .../grouping/distributed/command/TopGroupsFieldCommand.java | 2 +- 6 files changed, 9 insertions(+), 6 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 83f14407fe4..6b0658c7f3b 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -190,6 +190,9 @@ Other Changes * SOLR-9745: print errors from solr.cmd (Gopikannan Venugopalsamy via Mikhail Khludnev) +* SOLR-10394: Rename getSortWithinGroup to getWithinGroupSort in search.grouping.Command class. + (Judith Silverman, Christine Poerschke) + ================== 6.5.1 ================== Bug Fixes diff --git a/solr/core/src/java/org/apache/solr/search/grouping/Command.java b/solr/core/src/java/org/apache/solr/search/grouping/Command.java index 55e2d96a0ac..7391df6e081 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/Command.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/Command.java @@ -60,6 +60,6 @@ public interface Command { /** * @return The sort inside a group */ - Sort getSortWithinGroup(); + Sort getWithinGroupSort(); } diff --git a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java index 2dd2291def8..8ba0a6a45af 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/CommandHandler.java @@ -174,9 +174,9 @@ public class CommandHandler { final AllGroupHeadsCollector allGroupHeadsCollector; if (fieldType.getNumberType() != null) { ValueSource vs = fieldType.getValueSource(sf, null); - allGroupHeadsCollector = new FunctionAllGroupHeadsCollector(vs, new HashMap(), firstCommand.getSortWithinGroup()); + allGroupHeadsCollector = new FunctionAllGroupHeadsCollector(vs, new HashMap(), firstCommand.getWithinGroupSort()); } else { - allGroupHeadsCollector = TermAllGroupHeadsCollector.create(firstCommand.getKey(), firstCommand.getSortWithinGroup()); + allGroupHeadsCollector = TermAllGroupHeadsCollector.create(firstCommand.getKey(), firstCommand.getWithinGroupSort()); } if (collectors.isEmpty()) { searchWithTimeLimiter(query, filter, allGroupHeadsCollector); diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/QueryCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/QueryCommand.java index afb8ba78a9c..1615237da2d 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/QueryCommand.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/QueryCommand.java @@ -149,7 +149,7 @@ public class QueryCommand implements Command { } @Override - public Sort getSortWithinGroup() { + public Sort getWithinGroupSort() { return null; } } diff --git a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java index d5f9f9da3c6..7f923829e4a 100644 --- a/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java +++ b/solr/core/src/java/org/apache/solr/search/grouping/distributed/command/SearchGroupsFieldCommand.java @@ -138,7 +138,7 @@ public class SearchGroupsFieldCommand implements Command> { } @Override - public Sort getSortWithinGroup() { + public Sort getWithinGroupSort() { return sortWithinGroup; } } From f08889f390765c58a7f44f2ff1052484037ce336 Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Tue, 4 Apr 2017 13:11:02 -0700 Subject: [PATCH 117/563] SOLR-10347: Remove index level boost support from 'documents' section of the admin UI --- solr/CHANGES.txt | 3 +++ solr/webapp/web/js/angular/controllers/documents.js | 2 -- solr/webapp/web/js/scripts/documents.js | 8 -------- solr/webapp/web/partials/documents.html | 7 ------- solr/webapp/web/tpl/documents.html | 7 ------- 5 files changed, 3 insertions(+), 24 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 6b0658c7f3b..4fa03534a21 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -93,6 +93,9 @@ Optimizations Other Changes * SOLR-10236: Removed FieldType.getNumericType(). Use getNumberType() instead. (Tomás Fernández Löbbe) +* SOLR-10347: Removed index level boost support from "documents" section of the admin UI (Amrit Sarkar via + Tomás Fernández Löbbe) + ---------------------- ================== 6.6.0 ================== diff --git a/solr/webapp/web/js/angular/controllers/documents.js b/solr/webapp/web/js/angular/controllers/documents.js index be37c9fbde7..d38265a0548 100644 --- a/solr/webapp/web/js/angular/controllers/documents.js +++ b/solr/webapp/web/js/angular/controllers/documents.js @@ -38,7 +38,6 @@ solrAdminApp.controller('DocumentsController', $scope.type = "json"; $scope.commitWithin = 1000; $scope.overwrite = true; - $scope.boost = "1.0"; }; $scope.refresh(); @@ -78,7 +77,6 @@ solrAdminApp.controller('DocumentsController', } params.commitWithin = $scope.commitWithin; - params.boost = $scope.boost; params.overwrite = $scope.overwrite; params.core = $routeParams.core; params.wt = "json"; diff --git a/solr/webapp/web/js/scripts/documents.js b/solr/webapp/web/js/scripts/documents.js index 9d12e2371c4..45cfbed7d88 100644 --- a/solr/webapp/web/js/scripts/documents.js +++ b/solr/webapp/web/js/scripts/documents.js @@ -29,7 +29,6 @@ var content_generator = { //Utiltity function for turning on/off various elements function toggles(documents_form, show_json, show_file, show_doc, doc_text, show_wizard) { - var json_only = $('#json-only'); var the_document = $('#document', documents_form); if (show_doc) { //console.log("doc: " + doc_text); @@ -38,11 +37,6 @@ function toggles(documents_form, show_json, show_file, show_doc, doc_text, show_ } else { the_document.hide(); } - if (show_json) { - json_only.show(); - } else { - json_only.hide(); - } var file_upload = $('#file-upload', documents_form); var upload_only = $('#upload-only', documents_form); if (show_file) { @@ -233,7 +227,6 @@ sammy.get .trigger('change'); var the_document = $('#document', documents_form).val(); var commit_within = $('#commitWithin', documents_form).val(); - var boost = $('#boost', documents_form).val(); var overwrite = $('#overwrite', documents_form).val(); var the_command = ""; var content_type = ""; @@ -245,7 +238,6 @@ sammy.get //create a JSON command the_command = "{" + '"add":{ "doc":' + the_document + "," - + '"boost":' + boost + "," + '"overwrite":' + overwrite + "," + '"commitWithin":' + commit_within + "}}"; diff --git a/solr/webapp/web/partials/documents.html b/solr/webapp/web/partials/documents.html index 74d034f279e..2bf3f12982d 100644 --- a/solr/webapp/web/partials/documents.html +++ b/solr/webapp/web/partials/documents.html @@ -88,13 +88,6 @@ - -
    - - -
    diff --git a/solr/webapp/web/tpl/documents.html b/solr/webapp/web/tpl/documents.html index bd953a40b40..d2a2e0e7d2d 100644 --- a/solr/webapp/web/tpl/documents.html +++ b/solr/webapp/web/tpl/documents.html @@ -85,13 +85,6 @@ - -
    - - -
    From 60303028debf3927e0c3abfaaa4015f73b88e689 Mon Sep 17 00:00:00 2001 From: Shalin Shekhar Mangar Date: Wed, 5 Apr 2017 16:01:44 +0530 Subject: [PATCH 118/563] SOLR-10277: On 'downnode', lots of wasteful mutations are done to ZK --- solr/CHANGES.txt | 3 + .../java/org/apache/solr/cloud/Overseer.java | 2 +- .../solr/cloud/overseer/NodeMutator.java | 29 +- .../solr/cloud/overseer/ZkWriteCommand.java | 5 + .../solr/cloud/ClusterStateMockUtil.java | 233 ++++++++++++++++ .../apache/solr/cloud/NodeMutatorTest.java | 95 +++++++ .../SharedFSAutoReplicaFailoverUtilsTest.java | 263 ++---------------- 7 files changed, 378 insertions(+), 252 deletions(-) create mode 100644 solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java create mode 100644 solr/core/src/test/org/apache/solr/cloud/NodeMutatorTest.java diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 4fa03534a21..35403159c53 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -207,6 +207,9 @@ Bug Fixes * SOLR-10416: The JSON output of /admin/metrics is fixed to write the container as a map (SimpleOrderedMap) instead of an array (NamedList). (shalin) +* SOLR-10277: On 'downnode', lots of wasteful mutations are done to ZK. + (Joshua Humphries, Scott Blum, Varun Thacker, shalin) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java index 4d3cee7d737..f97fbaca833 100644 --- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java +++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java @@ -383,7 +383,7 @@ public class Overseer implements Closeable { } break; case DOWNNODE: - return new NodeMutator(getZkStateReader()).downNode(clusterState, message); + return new NodeMutator().downNode(clusterState, message); default: throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties()); } diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java index 0036fe1049c..55fd3efb618 100644 --- a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java +++ b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java @@ -19,7 +19,6 @@ package org.apache.solr.cloud.overseer; import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -38,44 +37,44 @@ public class NodeMutator { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - public NodeMutator(ZkStateReader zkStateReader) { - - } - public List downNode(ClusterState clusterState, ZkNodeProps message) { - List zkWriteCommands = new ArrayList(); + List zkWriteCommands = new ArrayList<>(); String nodeName = message.getStr(ZkStateReader.NODE_NAME_PROP); log.debug("DownNode state invoked for node: " + nodeName); Map collections = clusterState.getCollectionsMap(); for (Map.Entry entry : collections.entrySet()) { + String collection = entry.getKey(); DocCollection docCollection = entry.getValue(); + Map slicesCopy = new LinkedHashMap<>(docCollection.getSlicesMap()); - for (Entry sliceEntry : slicesCopy.entrySet()) { - Slice slice = docCollection.getSlice(sliceEntry.getKey()); - Map newReplicas = new HashMap(); + boolean needToUpdateCollection = false; + for (Entry sliceEntry : slicesCopy.entrySet()) { + Slice slice = sliceEntry.getValue(); + Map newReplicas = slice.getReplicasCopy(); Collection replicas = slice.getReplicas(); for (Replica replica : replicas) { - Map props = replica.shallowCopy(); String rNodeName = replica.getNodeName(); if (rNodeName.equals(nodeName)) { log.debug("Update replica state for " + replica + " to " + Replica.State.DOWN.toString()); + Map props = replica.shallowCopy(); props.put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString()); + Replica newReplica = new Replica(replica.getName(), props); + newReplicas.put(replica.getName(), newReplica); + needToUpdateCollection = true; } - - Replica newReplica = new Replica(replica.getName(), props); - newReplicas.put(replica.getName(), newReplica); } Slice newSlice = new Slice(slice.getName(), newReplicas, slice.shallowCopy()); slicesCopy.put(slice.getName(), newSlice); - } - zkWriteCommands.add(new ZkWriteCommand(entry.getKey(), docCollection.copyWithSlices(slicesCopy))); + if (needToUpdateCollection) { + zkWriteCommands.add(new ZkWriteCommand(collection, docCollection.copyWithSlices(slicesCopy))); + } } return zkWriteCommands; diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkWriteCommand.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkWriteCommand.java index 1697522e3c8..d464863df4b 100644 --- a/solr/core/src/java/org/apache/solr/cloud/overseer/ZkWriteCommand.java +++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ZkWriteCommand.java @@ -41,5 +41,10 @@ public class ZkWriteCommand { public static ZkWriteCommand noop() { return new ZkWriteCommand(); } + + @Override + public String toString() { + return getClass().getSimpleName() + ": " + (noop ? "no-op" : name + "=" + collection); + } } diff --git a/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java b/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java new file mode 100644 index 00000000000..e0cf3f733b0 --- /dev/null +++ b/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtil.java @@ -0,0 +1,233 @@ +/* + * 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.Closeable; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.solr.common.cloud.ClusterState; +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.ZkStateReader; +import org.apache.solr.common.util.Utils; + +public class ClusterStateMockUtil { + + private final static Pattern BLUEPRINT = Pattern.compile("([a-z])(\\d+)?(?:(['A','R','D','F']))?(\\*)?"); + + protected static class Result implements Closeable { + OverseerAutoReplicaFailoverThread.DownReplica badReplica; + ZkStateReader reader; + + @Override + public void close() throws IOException { + reader.close(); + } + } + + protected static ClusterStateMockUtil.Result buildClusterState(List results, String string, String ... liveNodes) { + return buildClusterState(results, string, 1, liveNodes); + } + + protected static ClusterStateMockUtil.Result buildClusterState(List results, String string, int replicationFactor, String ... liveNodes) { + return buildClusterState(results, string, replicationFactor, 10, liveNodes); + } + + /** + * This method lets you construct a complex ClusterState object by using simple strings of letters. + * + * c = collection, s = slice, r = replica, \d = node number (r2 means the replica is on node 2), + * state = [A,R,D,F], * = replica to replace, binds to the left. + * + * For example: + * csrr2rD*sr2csr + * + * Creates: + * + * 'csrr2rD*' + * A collection, a shard, a replica on node 1 (the default) that is active (the default), a replica on node 2, and a replica on node 1 + * that has a state of down and is the replica we will be looking to put somewhere else (the *). + * + * 'sr2' + * Then, another shard that has a replica on node 2. + * + * 'csr' + * Then, another collection that has a shard with a single active replica on node 1. + * + * Result: + * { + * "collection2":{ + * "maxShardsPerNode":"1", + * "replicationFactor":"1", + * "shards":{"slice1":{ + * "state":"active", + * "replicas":{"replica5":{ + * "state":"active", + * "node_name":"baseUrl1_", + * "base_url":"http://baseUrl1"}}}}}, + * "collection1":{ + * "maxShardsPerNode":"1", + * "replicationFactor":"1", + * "shards":{ + * "slice1":{ + * "state":"active", + * "replicas":{ + * "replica3 (bad)":{ + * "state":"down", + * "node_name":"baseUrl1_", + * "base_url":"http://baseUrl1"}, + * "replica2":{ + * "state":"active", + * "node_name":"baseUrl2_", + * "base_url":"http://baseUrl2"}, + * "replica1":{ + * "state":"active", + * "node_name":"baseUrl1_", + * "base_url":"http://baseUrl1"}}}, + * "slice2":{ + * "state":"active", + * "replicas":{"replica4":{ + * "state":"active", + * "node_name":"baseUrl2_", + * "base_url":"http://baseUrl2"}}}}}} + * + */ + @SuppressWarnings("resource") + protected static ClusterStateMockUtil.Result buildClusterState(List results, String clusterDescription, int replicationFactor, int maxShardsPerNode, String ... liveNodes) { + ClusterStateMockUtil.Result result = new ClusterStateMockUtil.Result(); + + Map slices = null; + Map replicas = null; + Map collectionProps = new HashMap<>(); + collectionProps.put(ZkStateReader.MAX_SHARDS_PER_NODE, Integer.toString(maxShardsPerNode)); + collectionProps.put(ZkStateReader.REPLICATION_FACTOR, Integer.toString(replicationFactor)); + Map collectionStates = new HashMap<>(); + DocCollection docCollection = null; + Slice slice = null; + int replicaCount = 1; + + Matcher m = BLUEPRINT.matcher(clusterDescription); + while (m.find()) { + Replica replica; + switch (m.group(1)) { + case "c": + slices = new HashMap<>(); + docCollection = new DocCollection("collection" + (collectionStates.size() + 1), slices, collectionProps, null); + collectionStates.put(docCollection.getName(), docCollection); + break; + case "s": + replicas = new HashMap<>(); + slice = new Slice("slice" + (slices.size() + 1), replicas, null); + slices.put(slice.getName(), slice); + break; + case "r": + Map replicaPropMap = new HashMap<>(); + String node; + + node = m.group(2); + + if (node == null || node.trim().length() == 0) { + node = "1"; + } + + Replica.State state = Replica.State.ACTIVE; + String stateCode = m.group(3); + + if (stateCode != null) { + switch (stateCode.charAt(0)) { + case 'S': + state = Replica.State.ACTIVE; + break; + case 'R': + state = Replica.State.RECOVERING; + break; + case 'D': + state = Replica.State.DOWN; + break; + case 'F': + state = Replica.State.RECOVERY_FAILED; + break; + default: + throw new IllegalArgumentException( + "Unexpected state for replica: " + stateCode); + } + } + + String nodeName = "baseUrl" + node + "_"; + String replicaName = "replica" + replicaCount++; + + if ("*".equals(m.group(4))) { + replicaName += " (bad)"; + } + + replicaPropMap.put(ZkStateReader.NODE_NAME_PROP, nodeName); + replicaPropMap.put(ZkStateReader.BASE_URL_PROP, "http://baseUrl" + node); + replicaPropMap.put(ZkStateReader.STATE_PROP, state.toString()); + + replica = new Replica(replicaName, replicaPropMap); + + if ("*".equals(m.group(4))) { + result.badReplica = new OverseerAutoReplicaFailoverThread.DownReplica(); + result.badReplica.replica = replica; + result.badReplica.slice = slice; + result.badReplica.collection = docCollection; + } + + replicas.put(replica.getName(), replica); + break; + default: + break; + } + } + + ClusterState clusterState = new ClusterState(1, new HashSet<>(Arrays.asList(liveNodes)), collectionStates); + MockZkStateReader reader = new MockZkStateReader(clusterState, collectionStates.keySet()); + + String json; + try { + json = new String(Utils.toJSON(clusterState), "UTF-8"); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("Unexpected"); + } + System.err.println(json); + + // todo remove the limitation of always having a bad replica + assert result.badReplica != null : "Is there no bad replica?"; + assert result.badReplica.slice != null : "Is there no bad replica?"; + + result.reader = reader; + + if (results != null) { + results.add(result); + } + + return result; + } + + +} diff --git a/solr/core/src/test/org/apache/solr/cloud/NodeMutatorTest.java b/solr/core/src/test/org/apache/solr/cloud/NodeMutatorTest.java new file mode 100644 index 00000000000..ffa6ba2f567 --- /dev/null +++ b/solr/core/src/test/org/apache/solr/cloud/NodeMutatorTest.java @@ -0,0 +1,95 @@ +/* + * 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.List; + +import org.apache.solr.SolrTestCaseJ4Test; +import org.apache.solr.cloud.overseer.NodeMutator; +import org.apache.solr.cloud.overseer.ZkWriteCommand; +import org.apache.solr.common.cloud.ClusterState; +import org.apache.solr.common.cloud.Replica; +import org.apache.solr.common.cloud.ZkNodeProps; +import org.apache.solr.common.cloud.ZkStateReader; +import org.junit.Test; + +public class NodeMutatorTest extends SolrTestCaseJ4Test { + + private static final String NODE3 = "baseUrl3_"; + private static final String NODE3_URL = "http://baseUrl3"; + + private static final String NODE2 = "baseUrl2_"; + private static final String NODE2_URL = "http://baseUrl2"; + + private static final String NODE1 = "baseUrl1_"; + private static final String NODE1_URL = "http://baseUrl1"; + + @Test + public void downNodeReportsAllImpactedCollectionsAndNothingElse() throws IOException { + NodeMutator nm = new NodeMutator(); + ZkNodeProps props = new ZkNodeProps(ZkStateReader.NODE_NAME_PROP, NODE1); + + //We use 2 nodes with maxShardsPerNode as 1 + //Collection1: 2 shards X 1 replica = replica1 on node1 and replica2 on node2 + //Collection2: 1 shard X 1 replica = replica1 on node2 + ClusterStateMockUtil.Result result = ClusterStateMockUtil.buildClusterState(null, "csrr2rD*csr2", 1, 1, NODE1, NODE2); + ClusterState clusterState = result.reader.getClusterState(); + assertEquals(clusterState.getCollection("collection1").getReplica("replica1").getBaseUrl(), NODE1_URL); + assertEquals(clusterState.getCollection("collection1").getReplica("replica2").getBaseUrl(), NODE2_URL); + assertEquals(clusterState.getCollection("collection2").getReplica("replica4").getBaseUrl(), NODE2_URL); + + props = new ZkNodeProps(ZkStateReader.NODE_NAME_PROP, NODE1); + List writes = nm.downNode(clusterState, props); + assertEquals(writes.size(), 1); + assertEquals(writes.get(0).name, "collection1"); + assertEquals(writes.get(0).collection.getReplica("replica1").getState(), Replica.State.DOWN); + assertEquals(writes.get(0).collection.getReplica("replica2").getState(), Replica.State.ACTIVE); + result.close(); + + //We use 3 nodes with maxShardsPerNode as 1 + //Collection1: 2 shards X 1 replica = replica1 on node1 and replica2 on node2 + //Collection2: 1 shard X 1 replica = replica1 on node2 + //Collection3: 1 shard X 3 replica = replica1 on node1 , replica2 on node2, replica3 on node3 + result = ClusterStateMockUtil.buildClusterState(null, "csrr2rD*csr2csr1r2r3", 1, 1, NODE1, NODE2, NODE3); + clusterState = result.reader.getClusterState(); + assertEquals(clusterState.getCollection("collection1").getReplica("replica1").getBaseUrl(), NODE1_URL); + assertEquals(clusterState.getCollection("collection1").getReplica("replica2").getBaseUrl(), NODE2_URL); + + assertEquals(clusterState.getCollection("collection2").getReplica("replica4").getBaseUrl(), NODE2_URL); + + assertEquals(clusterState.getCollection("collection3").getReplica("replica5").getBaseUrl(), NODE1_URL); + assertEquals(clusterState.getCollection("collection3").getReplica("replica6").getBaseUrl(), NODE2_URL); + assertEquals(clusterState.getCollection("collection3").getReplica("replica7").getBaseUrl(), NODE3_URL); + + writes = nm.downNode(clusterState, props); + assertEquals(writes.size(), 2); + for (ZkWriteCommand write : writes) { + if (write.name.equals("collection1")) { + assertEquals(write.collection.getReplica("replica1").getState(), Replica.State.DOWN); + assertEquals(write.collection.getReplica("replica2").getState(), Replica.State.ACTIVE); + } else if (write.name.equals("collection3")) { + assertEquals(write.collection.getReplica("replica5").getState(), Replica.State.DOWN); + assertEquals(write.collection.getReplica("replica6").getState(), Replica.State.ACTIVE); + assertEquals(write.collection.getReplica("replica7").getState(), Replica.State.ACTIVE); + } else { + fail("No other collection needs to be changed"); + } + } + result.close(); + } +} diff --git a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverUtilsTest.java b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverUtilsTest.java index f5fee2174e3..342342020ec 100644 --- a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverUtilsTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverUtilsTest.java @@ -16,30 +16,16 @@ */ package org.apache.solr.cloud; -import java.io.Closeable; -import java.io.IOException; -import java.io.UnsupportedEncodingException; import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; import org.apache.solr.SolrTestCaseJ4; -import org.apache.solr.cloud.OverseerAutoReplicaFailoverThread.DownReplica; -import org.apache.solr.common.cloud.ClusterState; -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.ZkStateReader; -import org.apache.solr.common.util.Utils; import org.junit.After; import org.junit.Before; import org.junit.Test; +import static org.apache.solr.cloud.ClusterStateMockUtil.buildClusterState; + public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 { private static final String NODE6 = "baseUrl6_"; private static final String NODE6_URL = "http://baseUrl6"; @@ -58,12 +44,8 @@ public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 { private static final String NODE1 = "baseUrl1_"; private static final String NODE1_URL = "http://baseUrl1"; - - private final static Pattern BLUEPRINT = Pattern.compile("([a-z])(\\d+)?(?:(['A','R','D','F']))?(\\*)?"); - - private int buildNumber = 1; - private List results; + private List results; @Before public void setUp() throws Exception { @@ -74,61 +56,50 @@ public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 { @After public void tearDown() throws Exception { super.tearDown(); - for (Result result : results) { + for (ClusterStateMockUtil.Result result : results) { result.close(); } } @Test public void testGetBestCreateUrlBasics() { - Result result = buildClusterState("csr1R*r2", NODE1); + ClusterStateMockUtil.Result result = buildClusterState(results, "csr1R*r2", NODE1); String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertNull("Should be no live node to failover to", createUrl); - result = buildClusterState("csr1R*r2", NODE1, NODE2); + result = buildClusterState(results, "csr1R*r2", NODE1, NODE2); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertNull("Only failover candidate node already has a replica", createUrl); - result = buildClusterState("csr1R*r2sr3", NODE1, NODE2, NODE3); + result = buildClusterState(results, "csr1R*r2sr3", NODE1, NODE2, NODE3); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals("Node3 does not have a replica from the bad slice and should be the best choice", NODE3_URL, createUrl); - result = buildClusterState("csr1R*r2Fsr3r4r5", NODE1, NODE2, NODE3); + result = buildClusterState(results, "csr1R*r2Fsr3r4r5", NODE1, NODE2, NODE3); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertTrue(createUrl.equals(NODE3_URL)); - result = buildClusterState("csr1*r2r3sr3r3sr4", NODE1, NODE2, NODE3, NODE4); + result = buildClusterState(results, "csr1*r2r3sr3r3sr4", NODE1, NODE2, NODE3, NODE4); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals(NODE4_URL, createUrl); - result = buildClusterState("csr1*r2sr3r3sr4sr4", NODE1, NODE2, NODE3, NODE4); + result = buildClusterState(results, "csr1*r2sr3r3sr4sr4", NODE1, NODE2, NODE3, NODE4); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertTrue(createUrl.equals(NODE3_URL) || createUrl.equals(NODE4_URL)); } - - - private static class Result implements Closeable { - DownReplica badReplica; - ZkStateReader reader; - - @Override - public void close() throws IOException { - reader.close(); - } - } @Test public void testGetBestCreateUrlMultipleCollections() throws Exception { - Result result = buildClusterState("csr*r2csr2", NODE1); + ClusterStateMockUtil.Result result = buildClusterState(results, "csr*r2csr2", NODE1); String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertNull(createUrl); - result = buildClusterState("csr*r2csr2", NODE1); + result = buildClusterState(results, "csr*r2csr2", NODE1); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertNull(createUrl); - result = buildClusterState("csr*r2csr2", NODE1, NODE2); + result = buildClusterState(results, "csr*r2csr2", NODE1, NODE2); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertNull(createUrl); } @@ -136,11 +107,11 @@ public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 { @Test public void testGetBestCreateUrlMultipleCollections2() { - Result result = buildClusterState("csr*r2sr3cr2", NODE1); + ClusterStateMockUtil.Result result = buildClusterState(results, "csr*r2sr3cr2", NODE1); String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertNull(createUrl); - result = buildClusterState("csr*r2sr3cr2", NODE1, NODE2, NODE3); + result = buildClusterState(results, "csr*r2sr3cr2", NODE1, NODE2, NODE3); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals(NODE3_URL, createUrl); } @@ -148,253 +119,73 @@ public class SharedFSAutoReplicaFailoverUtilsTest extends SolrTestCaseJ4 { @Test public void testGetBestCreateUrlMultipleCollections3() { - Result result = buildClusterState("csr5r1sr4r2sr3r6csr2*r6sr5r3sr4r3", NODE1, NODE4, NODE5, NODE6); + ClusterStateMockUtil.Result result = buildClusterState(results, "csr5r1sr4r2sr3r6csr2*r6sr5r3sr4r3", NODE1, NODE4, NODE5, NODE6); String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals(NODE1_URL, createUrl); } @Test public void testGetBestCreateUrlMultipleCollections4() { - Result result = buildClusterState("csr1r4sr3r5sr2r6csr5r6sr4r6sr5*r4", NODE6); + ClusterStateMockUtil.Result result = buildClusterState(results, "csr1r4sr3r5sr2r6csr5r6sr4r6sr5*r4", NODE6); String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals(NODE6_URL, createUrl); } @Test public void testFailOverToEmptySolrInstance() { - Result result = buildClusterState("csr1*r1sr1csr1", NODE2); + ClusterStateMockUtil.Result result = buildClusterState(results, "csr1*r1sr1csr1", NODE2); String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals(NODE2_URL, createUrl); } @Test public void testFavorForeignSlices() { - Result result = buildClusterState("csr*sr2csr3r3", NODE2, NODE3); + ClusterStateMockUtil.Result result = buildClusterState(results, "csr*sr2csr3r3", NODE2, NODE3); String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals(NODE3_URL, createUrl); - result = buildClusterState("csr*sr2csr3r3r3r3r3r3r3", NODE2, NODE3); + result = buildClusterState(results, "csr*sr2csr3r3r3r3r3r3r3", NODE2, NODE3); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals(NODE2_URL, createUrl); } @Test public void testCollectionMaxNodesPerShard() { - Result result = buildClusterState("csr*sr2", 1, 1, NODE2); + ClusterStateMockUtil.Result result = buildClusterState(results, "csr*sr2", 1, 1, NODE2); String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertNull(createUrl); - result = buildClusterState("csr*sr2", 1, 2, NODE2); + result = buildClusterState(results, "csr*sr2", 1, 2, NODE2); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals(NODE2_URL, createUrl); - result = buildClusterState("csr*csr2r2", 1, 1, NODE2); + result = buildClusterState(results, "csr*csr2r2", 1, 1, NODE2); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, null); assertEquals(NODE2_URL, createUrl); } @Test public void testMaxCoresPerNode() { - Result result = buildClusterState("csr*sr2", 1, 1, NODE2); + ClusterStateMockUtil.Result result = buildClusterState(results, "csr*sr2", 1, 1, NODE2); String createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 1); assertNull(createUrl); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2); assertNull(createUrl); - result = buildClusterState("csr*sr2", 1, 2, NODE2); + result = buildClusterState(results, "csr*sr2", 1, 2, NODE2); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2); assertEquals(NODE2_URL, createUrl); - result = buildClusterState("csr*sr2sr3sr4", 1, 1, NODE2, NODE3, NODE4); + result = buildClusterState(results, "csr*sr2sr3sr4", 1, 1, NODE2, NODE3, NODE4); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 1); assertNull(createUrl); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2); assertNull(createUrl); - result = buildClusterState("csr*sr2sr3sr4", 1, 2, NODE2, NODE3, NODE4); + result = buildClusterState(results, "csr*sr2sr3sr4", 1, 2, NODE2, NODE3, NODE4); createUrl = OverseerAutoReplicaFailoverThread.getBestCreateUrl(result.reader, result.badReplica, 2); assertTrue(createUrl.equals(NODE3_URL) || createUrl.equals(NODE4_URL)); } - - private Result buildClusterState(String string, String ... liveNodes) { - return buildClusterState(string, 1, liveNodes); - } - - private Result buildClusterState(String string, int replicationFactor, String ... liveNodes) { - return buildClusterState(string, replicationFactor, 10, liveNodes); - } - - /** - * This method lets you construct a complex ClusterState object by using simple strings of letters. - * - * c = collection, s = slice, r = replica, \d = node number (r2 means the replica is on node 2), - * state = [A,R,D,F], * = replica to replace, binds to the left. - * - * For example: - * csrr2rD*sr2csr - * - * Creates: - * - * 'csrr2rD*' - * A collection, a shard, a replica on node 1 (the default) that is active (the default), a replica on node 2, and a replica on node 1 - * that has a state of down and is the replica we will be looking to put somewhere else (the *). - * - * 'sr2' - * Then, another shard that has a replica on node 2. - * - * 'csr' - * Then, another collection that has a shard with a single active replica on node 1. - * - * Result: - * { - * "collection2":{ - * "maxShardsPerNode":"1", - * "replicationFactor":"1", - * "shards":{"slice1":{ - * "state":"active", - * "replicas":{"replica5":{ - * "state":"active", - * "node_name":"baseUrl1_", - * "base_url":"http://baseUrl1"}}}}}, - * "collection1":{ - * "maxShardsPerNode":"1", - * "replicationFactor":"1", - * "shards":{ - * "slice1":{ - * "state":"active", - * "replicas":{ - * "replica3 (bad)":{ - * "state":"down", - * "node_name":"baseUrl1_", - * "base_url":"http://baseUrl1"}, - * "replica2":{ - * "state":"active", - * "node_name":"baseUrl2_", - * "base_url":"http://baseUrl2"}, - * "replica1":{ - * "state":"active", - * "node_name":"baseUrl1_", - * "base_url":"http://baseUrl1"}}}, - * "slice2":{ - * "state":"active", - * "replicas":{"replica4":{ - * "state":"active", - * "node_name":"baseUrl2_", - * "base_url":"http://baseUrl2"}}}}}} - * - */ - @SuppressWarnings("resource") - private Result buildClusterState(String clusterDescription, int replicationFactor, int maxShardsPerNode, String ... liveNodes) { - Result result = new Result(); - - Map slices = null; - Map replicas = null; - Map collectionProps = new HashMap<>(); - collectionProps.put(ZkStateReader.MAX_SHARDS_PER_NODE, Integer.toString(maxShardsPerNode)); - collectionProps.put(ZkStateReader.REPLICATION_FACTOR, Integer.toString(replicationFactor)); - Map collectionStates = new HashMap<>(); - DocCollection docCollection = null; - Slice slice = null; - int replicaCount = 1; - - Matcher m = BLUEPRINT.matcher(clusterDescription); - while (m.find()) { - Replica replica; - switch (m.group(1)) { - case "c": - slices = new HashMap<>(); - docCollection = new DocCollection("collection" + (collectionStates.size() + 1), slices, collectionProps, null); - collectionStates.put(docCollection.getName(), docCollection); - break; - case "s": - replicas = new HashMap<>(); - slice = new Slice("slice" + (slices.size() + 1), replicas, null); - slices.put(slice.getName(), slice); - break; - case "r": - Map replicaPropMap = new HashMap<>(); - String node; - - node = m.group(2); - - if (node == null || node.trim().length() == 0) { - node = "1"; - } - - Replica.State state = Replica.State.ACTIVE; - String stateCode = m.group(3); - - if (stateCode != null) { - switch (stateCode.charAt(0)) { - case 'S': - state = Replica.State.ACTIVE; - break; - case 'R': - state = Replica.State.RECOVERING; - break; - case 'D': - state = Replica.State.DOWN; - break; - case 'F': - state = Replica.State.RECOVERY_FAILED; - break; - default: - throw new IllegalArgumentException( - "Unexpected state for replica: " + stateCode); - } - } - - String nodeName = "baseUrl" + node + "_"; - String replicaName = "replica" + replicaCount++; - - if ("*".equals(m.group(4))) { - replicaName += " (bad)"; - } - - replicaPropMap.put(ZkStateReader.NODE_NAME_PROP, nodeName); - replicaPropMap.put(ZkStateReader.BASE_URL_PROP, "http://baseUrl" + node); - replicaPropMap.put(ZkStateReader.STATE_PROP, state.toString()); - - replica = new Replica(replicaName, replicaPropMap); - - if ("*".equals(m.group(4))) { - result.badReplica = new DownReplica(); - result.badReplica.replica = replica; - result.badReplica.slice = slice; - result.badReplica.collection = docCollection; - } - - replicas.put(replica.getName(), replica); - break; - default: - break; - } - } - - // trunk briefly had clusterstate taking a zkreader :( this was required to work around that - leaving - // until that issue is resolved. - MockZkStateReader reader = new MockZkStateReader(null, collectionStates.keySet()); - ClusterState clusterState = new ClusterState(1, new HashSet<>(Arrays.asList(liveNodes)), collectionStates); - reader = new MockZkStateReader(clusterState, collectionStates.keySet()); - - String json; - try { - json = new String(Utils.toJSON(clusterState), "UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("Unexpected"); - } - System.err.println("build:" + buildNumber++); - System.err.println(json); - - assert result.badReplica != null : "Is there no bad replica?"; - assert result.badReplica.slice != null : "Is there no bad replica?"; - - result.reader = reader; - - results.add(result); - - return result; - } } From dfa342bc90f8f3012e26169286ee61d42e9b15cf Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Wed, 5 Apr 2017 11:50:13 +0100 Subject: [PATCH 119/563] SOLR-10421: Fix params persistence for solr/contrib/ltr (MinMax|Standard)Normalizer classes. (Jianxiong Dong, Christine Poerschke) --- solr/CHANGES.txt | 3 +++ .../src/java/org/apache/solr/ltr/norm/MinMaxNormalizer.java | 4 ++-- .../src/java/org/apache/solr/ltr/norm/StandardNormalizer.java | 4 ++-- .../test/org/apache/solr/ltr/norm/TestMinMaxNormalizer.java | 1 + .../test/org/apache/solr/ltr/norm/TestStandardNormalizer.java | 1 + 5 files changed, 9 insertions(+), 4 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 35403159c53..bb623dc0d5d 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -210,6 +210,9 @@ Bug Fixes * SOLR-10277: On 'downnode', lots of wasteful mutations are done to ZK. (Joshua Humphries, Scott Blum, Varun Thacker, shalin) +* SOLR-10421: Fix params persistence for solr/contrib/ltr (MinMax|Standard)Normalizer classes. + (Jianxiong Dong, Christine Poerschke) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/norm/MinMaxNormalizer.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/norm/MinMaxNormalizer.java index 92e233c95ca..ff31c0180f8 100644 --- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/norm/MinMaxNormalizer.java +++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/norm/MinMaxNormalizer.java @@ -90,8 +90,8 @@ public class MinMaxNormalizer extends Normalizer { @Override public LinkedHashMap paramsToMap() { final LinkedHashMap params = new LinkedHashMap<>(2, 1.0f); - params.put("min", min); - params.put("max", max); + params.put("min", '"'+Float.toString(min)+'"'); + params.put("max", '"'+Float.toString(max)+'"'); return params; } diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/norm/StandardNormalizer.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/norm/StandardNormalizer.java index 7ab525cb296..57df7b4eb0f 100644 --- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/norm/StandardNormalizer.java +++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/norm/StandardNormalizer.java @@ -82,8 +82,8 @@ public class StandardNormalizer extends Normalizer { @Override public LinkedHashMap paramsToMap() { final LinkedHashMap params = new LinkedHashMap<>(2, 1.0f); - params.put("avg", avg); - params.put("std", std); + params.put("avg", '"'+Float.toString(avg)+'"'); + params.put("std", '"'+Float.toString(std)+'"'); return params; } diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/norm/TestMinMaxNormalizer.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/norm/TestMinMaxNormalizer.java index 055b3bccacd..794e393a13d 100644 --- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/norm/TestMinMaxNormalizer.java +++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/norm/TestMinMaxNormalizer.java @@ -40,6 +40,7 @@ public class TestMinMaxNormalizer { final MinMaxNormalizer mmn = (MinMaxNormalizer)n; assertEquals(mmn.getMin(), expectedMin, 0.0); assertEquals(mmn.getMax(), expectedMax, 0.0); + assertEquals("{min=\""+expectedMin+"\", max=\""+expectedMax+"\"}", mmn.paramsToMap().toString()); return n; } diff --git a/solr/contrib/ltr/src/test/org/apache/solr/ltr/norm/TestStandardNormalizer.java b/solr/contrib/ltr/src/test/org/apache/solr/ltr/norm/TestStandardNormalizer.java index 10fa9720ccd..1794686b1bc 100644 --- a/solr/contrib/ltr/src/test/org/apache/solr/ltr/norm/TestStandardNormalizer.java +++ b/solr/contrib/ltr/src/test/org/apache/solr/ltr/norm/TestStandardNormalizer.java @@ -40,6 +40,7 @@ public class TestStandardNormalizer { final StandardNormalizer sn = (StandardNormalizer)n; assertEquals(sn.getAvg(), expectedAvg, 0.0); assertEquals(sn.getStd(), expectedStd, 0.0); + assertEquals("{avg=\""+expectedAvg+"\", std=\""+expectedStd+"\"}", sn.paramsToMap().toString()); return n; } From 50ed729459a55360c6179191babb0ee93b91b632 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Wed, 5 Apr 2017 11:52:34 +0100 Subject: [PATCH 120/563] Remove unused (private static final) loggers in LTRQParserPlugin and LTRFeatureLoggerTransformerFactory. --- .../transform/LTRFeatureLoggerTransformerFactory.java | 5 ----- .../java/org/apache/solr/ltr/search/LTRQParserPlugin.java | 5 ----- 2 files changed, 10 deletions(-) diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java index 9585a7f97ab..beb5ddf22ef 100644 --- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java +++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/response/transform/LTRFeatureLoggerTransformerFactory.java @@ -17,7 +17,6 @@ package org.apache.solr.ltr.response.transform; import java.io.IOException; -import java.lang.invoke.MethodHandles; import java.util.Collections; import java.util.List; import java.util.Locale; @@ -47,8 +46,6 @@ import org.apache.solr.response.transform.DocTransformer; import org.apache.solr.response.transform.TransformerFactory; import org.apache.solr.search.SolrIndexSearcher; import org.apache.solr.util.SolrPluginUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * This transformer will take care to generate and append in the response the @@ -65,8 +62,6 @@ import org.slf4j.LoggerFactory; public class LTRFeatureLoggerTransformerFactory extends TransformerFactory { - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - // used inside fl to specify the format (dense|sparse) of the extracted features private static final String FV_FORMAT = "format"; diff --git a/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java b/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java index 2b4d570bdd5..c5db963581a 100644 --- a/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java +++ b/solr/contrib/ltr/src/java/org/apache/solr/ltr/search/LTRQParserPlugin.java @@ -17,7 +17,6 @@ package org.apache.solr.ltr.search; import java.io.IOException; -import java.lang.invoke.MethodHandles; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -46,8 +45,6 @@ import org.apache.solr.search.QParserPlugin; import org.apache.solr.search.RankQuery; import org.apache.solr.search.SyntaxError; import org.apache.solr.util.SolrPluginUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Plug into solr a rerank model. @@ -60,8 +57,6 @@ public class LTRQParserPlugin extends QParserPlugin implements ResourceLoaderAwa public static final String NAME = "ltr"; private static Query defaultQuery = new MatchAllDocsQuery(); - private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - // params for setting custom external info that features can use, like query // intent static final String EXTERNAL_FEATURE_INFO = "efi."; From cb9f151db4b5ad5c5f581b6b8cf2e5916ddb0f35 Mon Sep 17 00:00:00 2001 From: David Smiley Date: Wed, 5 Apr 2017 08:56:50 -0400 Subject: [PATCH 121/563] SOLR-10404: fetch() streaming expression: escape values in generated query. --- solr/CHANGES.txt | 3 +++ .../client/solrj/io/stream/FetchStream.java | 20 +++++++------------ .../solrj/io/stream/StreamExpressionTest.java | 18 ++++++++++++++++- 3 files changed, 27 insertions(+), 14 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index bb623dc0d5d..ea825d9e6f2 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -213,6 +213,9 @@ Bug Fixes * SOLR-10421: Fix params persistence for solr/contrib/ltr (MinMax|Standard)Normalizer classes. (Jianxiong Dong, Christine Poerschke) +* SOLR-10404: The fetch() streaming expression wouldn't work if a value included query syntax chars (like :+-). + Fixed, and enhanced the generated query to not pollute the queryCache. (David Smiley) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java index 06e6fdcced7..2cd60ec55da 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FetchStream.java @@ -35,6 +35,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpression; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter; import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionValue; import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; +import org.apache.solr.client.solrj.util.ClientUtils; import org.apache.solr.common.params.ModifiableSolrParams; import static org.apache.solr.common.params.CommonParams.SORT; @@ -208,9 +209,8 @@ public class FetchStream extends TupleStream implements Expressible { } private void fetchBatch() throws IOException { - Tuple EOFTuple = null; - List batch = new ArrayList(); + List batch = new ArrayList<>(batchSize); for(int i=0; i 0) { - StringBuilder buf = new StringBuilder(); - buf.append(rightKey); - buf.append(":("); - for (int i = 0; i < batch.size(); i++) { - if (i > 0) { - buf.append(" "); - } - Tuple tuple = batch.get(i); + StringBuilder buf = new StringBuilder(batch.size() * 10 + 20); + buf.append("{! df=").append(rightKey).append(" q.op=OR cache=false }");//disable queryCache + for (Tuple tuple : batch) { String key = tuple.getString(leftKey); - buf.append(key); + buf.append(' ').append(ClientUtils.escapeQueryChars(key)); } - buf.append(")"); ModifiableSolrParams params = new ModifiableSolrParams(); params.add("q", buf.toString()); @@ -245,7 +239,7 @@ public class FetchStream extends TupleStream implements Expressible { StreamContext newContext = new StreamContext(); newContext.setSolrClientCache(streamContext.getSolrClientCache()); cloudSolrStream.setStreamContext(newContext); - Map fetched = new HashMap(); + Map fetched = new HashMap<>(); try { cloudSolrStream.open(); while (true) { diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java index f153a1be05d..581013fc9d9 100644 --- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java +++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java @@ -1044,7 +1044,7 @@ public class StreamExpressionTest extends SolrCloudTestCase { @Test public void testFetchStream() throws Exception { - SolrClientCache solrClientCache = new SolrClientCache(); + SolrClientCache solrClientCache = new SolrClientCache();//TODO share in @Before ; close in @After ? new UpdateRequest() .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "1", "subject", "blah blah blah 0") @@ -1123,6 +1123,22 @@ public class StreamExpressionTest extends SolrCloudTestCase { assertTrue("blah blah blah 8".equals(t.getString("subject"))); t = tuples.get(9); assertTrue("blah blah blah 9".equals(t.getString("subject"))); + + // SOLR-10404 test that "hello 99" as a value gets escaped + new UpdateRequest() + .add(id, "99", "a1_s", "hello 99", "a2_s", "hello 99", "subject", "blah blah blah 99") + .commit(cluster.getSolrClient(), COLLECTIONORALIAS); + + stream = factory.constructStream("fetch("+ COLLECTIONORALIAS +", search(" + COLLECTIONORALIAS + ", q=" + id + ":99, fl=\"id,a1_s\", sort=\"id asc\"), on=\"a1_s=a2_s\", fl=\"subject\")"); + context = new StreamContext(); + context.setSolrClientCache(solrClientCache); + stream.setStreamContext(context); + tuples = getTuples(stream); + + assertEquals(1, tuples.size()); + t = tuples.get(0); + assertTrue("blah blah blah 99".equals(t.getString("subject"))); + solrClientCache.close(); } From ef8126e5eab7aec9c8775c2e08bd6c2bb1ef690f Mon Sep 17 00:00:00 2001 From: Nicholas Knize Date: Wed, 5 Apr 2017 11:10:15 -0500 Subject: [PATCH 122/563] LUCENE-7738: Fix min/max verification bug in InetAddressRange to correctly compare IPv4 and IPv6. Update tests. --- .../lucene/document/InetAddressRange.java | 14 ++- .../search/TestInetAddressRangeQueries.java | 100 +++++++----------- 2 files changed, 50 insertions(+), 64 deletions(-) diff --git a/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java b/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java index 5fa1fb91bb7..84f0d6b3caf 100644 --- a/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java +++ b/lucene/misc/src/java/org/apache/lucene/document/InetAddressRange.java @@ -68,9 +68,6 @@ public class InetAddressRange extends Field { * @param max range max value; defined as an {@code InetAddress} */ public void setRangeValues(InetAddress min, InetAddress max) { - if (StringHelper.compare(BYTES, min.getAddress(), 0, max.getAddress(), 0) > 0) { - throw new IllegalArgumentException("min value cannot be greater than max value for range field (name=" + name + ")"); - } final byte[] bytes; if (fieldsData == null) { bytes = new byte[BYTES*2]; @@ -83,8 +80,15 @@ public class InetAddressRange extends Field { /** encode the min/max range into the provided byte array */ private static void encode(final InetAddress min, final InetAddress max, final byte[] bytes) { - System.arraycopy(InetAddressPoint.encode(min), 0, bytes, 0, BYTES); - System.arraycopy(InetAddressPoint.encode(max), 0, bytes, BYTES, BYTES); + // encode min and max value (consistent w/ InetAddressPoint encoding) + final byte[] minEncoded = InetAddressPoint.encode(min); + final byte[] maxEncoded = InetAddressPoint.encode(max); + // ensure min is lt max + if (StringHelper.compare(BYTES, minEncoded, 0, maxEncoded, 0) > 0) { + throw new IllegalArgumentException("min value cannot be greater than max value for InetAddressRange field"); + } + System.arraycopy(minEncoded, 0, bytes, 0, BYTES); + System.arraycopy(maxEncoded, 0, bytes, BYTES, BYTES); } /** encode the min/max range and return the byte array */ diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java b/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java index e22cf9b34ef..252162c4d97 100644 --- a/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java +++ b/lucene/misc/src/test/org/apache/lucene/search/TestInetAddressRangeQueries.java @@ -19,6 +19,7 @@ package org.apache.lucene.search; import java.net.InetAddress; import java.net.UnknownHostException; +import org.apache.lucene.document.InetAddressPoint; import org.apache.lucene.document.InetAddressRange; import org.apache.lucene.util.StringHelper; @@ -28,16 +29,12 @@ import org.apache.lucene.util.StringHelper; public class TestInetAddressRangeQueries extends BaseRangeFieldQueryTestCase { private static final String FIELD_NAME = "ipRangeField"; - private IPVersion ipVersion; - - private enum IPVersion {IPv4, IPv6} - @Override protected Range nextRange(int dimensions) throws Exception { InetAddress min = nextInetaddress(); - byte[] bMin = min.getAddress(); + byte[] bMin = InetAddressPoint.encode(min); InetAddress max = nextInetaddress(); - byte[] bMax = max.getAddress(); + byte[] bMax = InetAddressPoint.encode(max); if (StringHelper.compare(bMin.length, bMin, 0, bMax, 0) > 0) { return new IpRange(max, min); } @@ -46,89 +43,74 @@ public class TestInetAddressRangeQueries extends BaseRangeFieldQueryTestCase { /** return random IPv4 or IPv6 address */ private InetAddress nextInetaddress() throws UnknownHostException { - byte[] b; - switch (ipVersion) { - case IPv4: - b = new byte[4]; - break; - case IPv6: - b = new byte[16]; - break; - default: - throw new IllegalArgumentException("incorrect IP version: " + ipVersion); - } + byte[] b = random().nextBoolean() ? new byte[4] : new byte[16]; random().nextBytes(b); return InetAddress.getByAddress(b); } - /** randomly select version across tests */ - private IPVersion ipVersion() { - return random().nextBoolean() ? IPVersion.IPv4 : IPVersion.IPv6; - } - @Override public void testRandomTiny() throws Exception { - ipVersion = ipVersion(); super.testRandomTiny(); } @Override public void testMultiValued() throws Exception { - ipVersion = ipVersion(); super.testRandomMedium(); } @Override public void testRandomMedium() throws Exception { - ipVersion = ipVersion(); super.testMultiValued(); } @Nightly @Override public void testRandomBig() throws Exception { - ipVersion = ipVersion(); super.testRandomBig(); } /** return random range */ @Override protected InetAddressRange newRangeField(Range r) { - return new InetAddressRange(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max); + return new InetAddressRange(FIELD_NAME, ((IpRange)r).minAddress, ((IpRange)r).maxAddress); } /** return random intersects query */ @Override protected Query newIntersectsQuery(Range r) { - return InetAddressRange.newIntersectsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max); + return InetAddressRange.newIntersectsQuery(FIELD_NAME, ((IpRange)r).minAddress, ((IpRange)r).maxAddress); } /** return random contains query */ @Override protected Query newContainsQuery(Range r) { - return InetAddressRange.newContainsQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max); + return InetAddressRange.newContainsQuery(FIELD_NAME, ((IpRange)r).minAddress, ((IpRange)r).maxAddress); } /** return random within query */ @Override protected Query newWithinQuery(Range r) { - return InetAddressRange.newWithinQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max); + return InetAddressRange.newWithinQuery(FIELD_NAME, ((IpRange)r).minAddress, ((IpRange)r).maxAddress); } /** return random crosses query */ @Override protected Query newCrossesQuery(Range r) { - return InetAddressRange.newCrossesQuery(FIELD_NAME, ((IpRange)r).min, ((IpRange)r).max); + return InetAddressRange.newCrossesQuery(FIELD_NAME, ((IpRange)r).minAddress, ((IpRange)r).maxAddress); } /** encapsulated IpRange for test validation */ private class IpRange extends Range { - InetAddress min; - InetAddress max; + InetAddress minAddress; + InetAddress maxAddress; + byte[] min; + byte[] max; IpRange(InetAddress min, InetAddress max) { - this.min = min; - this.max = max; + this.minAddress = min; + this.maxAddress = max; + this.min = InetAddressPoint.encode(min); + this.max = InetAddressPoint.encode(max); } @Override @@ -138,33 +120,39 @@ public class TestInetAddressRangeQueries extends BaseRangeFieldQueryTestCase { @Override protected InetAddress getMin(int dim) { - return min; + return minAddress; } @Override protected void setMin(int dim, Object val) { - byte[] v = ((InetAddress)val).getAddress(); + InetAddress v = (InetAddress)val; + byte[] e = InetAddressPoint.encode(v); - if (StringHelper.compare(v.length, min.getAddress(), 0, v, 0) < 0) { - max = (InetAddress)val; + if (StringHelper.compare(e.length, min, 0, e, 0) < 0) { + max = e; + maxAddress = v; } else { - min = (InetAddress) val; + min = e; + minAddress = v; } } @Override protected InetAddress getMax(int dim) { - return max; + return maxAddress; } @Override protected void setMax(int dim, Object val) { - byte[] v = ((InetAddress)val).getAddress(); + InetAddress v = (InetAddress)val; + byte[] e = InetAddressPoint.encode(v); - if (StringHelper.compare(v.length, max.getAddress(), 0, v, 0) > 0) { - min = (InetAddress)val; + if (StringHelper.compare(e.length, max, 0, e, 0) > 0) { + min = e; + minAddress = v; } else { - max = (InetAddress) val; + max = e; + maxAddress = v; } } @@ -177,37 +165,31 @@ public class TestInetAddressRangeQueries extends BaseRangeFieldQueryTestCase { @Override protected boolean isDisjoint(Range o) { IpRange other = (IpRange)o; - byte[] bMin = min.getAddress(); - byte[] bMax = max.getAddress(); - return StringHelper.compare(bMin.length, bMin, 0, other.max.getAddress(), 0) > 0 || - StringHelper.compare(bMax.length, bMax, 0, other.min.getAddress(), 0) < 0; + return StringHelper.compare(min.length, min, 0, other.max, 0) > 0 || + StringHelper.compare(max.length, max, 0, other.min, 0) < 0; } @Override protected boolean isWithin(Range o) { IpRange other = (IpRange)o; - byte[] bMin = min.getAddress(); - byte[] bMax = max.getAddress(); - return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) >= 0 && - StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) <= 0; + return StringHelper.compare(min.length, min, 0, other.min, 0) >= 0 && + StringHelper.compare(max.length, max, 0, other.max, 0) <= 0; } @Override protected boolean contains(Range o) { IpRange other = (IpRange)o; - byte[] bMin = min.getAddress(); - byte[] bMax = max.getAddress(); - return StringHelper.compare(bMin.length, bMin, 0, other.min.getAddress(), 0) <= 0 && - StringHelper.compare(bMax.length, bMax, 0, other.max.getAddress(), 0) >= 0; + return StringHelper.compare(min.length, min, 0, other.min, 0) <= 0 && + StringHelper.compare(max.length, max, 0, other.max, 0) >= 0; } @Override public String toString() { StringBuilder b = new StringBuilder(); b.append("Box("); - b.append(min.getHostAddress()); + b.append(minAddress.getHostAddress()); b.append(" TO "); - b.append(max.getHostAddress()); + b.append(maxAddress.getHostAddress()); b.append(")"); return b.toString(); } From dbd22a6ada774eb30aee4b9312eb7913dee6890e Mon Sep 17 00:00:00 2001 From: Steve Rowe Date: Wed, 5 Apr 2017 16:23:26 -0400 Subject: [PATCH 123/563] SOLR-10423: Disable graph query production via schema configuration . This fixes broken queries for ShingleFilter-containing query-time analyzers when request param sow=false. --- solr/CHANGES.txt | 4 ++++ .../org/apache/solr/parser/QueryParser.java | 5 ++-- .../org/apache/solr/parser/QueryParser.jj | 5 ++-- .../solr/parser/SolrQueryParserBase.java | 23 +++++++++++++------ .../org/apache/solr/schema/FieldType.java | 4 +++- .../org/apache/solr/schema/TextField.java | 13 ++++++++++- .../solr/search/ExtendedDismaxQParser.java | 5 ++-- .../solr/collection1/conf/schema12.xml | 11 ++++++++- .../solr/search/TestExtendedDismaxParser.java | 18 +++++++++++++++ .../solr/search/TestSolrQueryParser.java | 18 +++++++++++++++ 10 files changed, 90 insertions(+), 16 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index ea825d9e6f2..214d6bb3d14 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -215,6 +215,10 @@ Bug Fixes * SOLR-10404: The fetch() streaming expression wouldn't work if a value included query syntax chars (like :+-). Fixed, and enhanced the generated query to not pollute the queryCache. (David Smiley) + +* SOLR-10423: Disable graph query production via schema configuration . + This fixes broken queries for ShingleFilter-containing query-time analyzers when request param sow=false. + (Steve Rowe) ================== 6.5.0 ================== diff --git a/solr/core/src/java/org/apache/solr/parser/QueryParser.java b/solr/core/src/java/org/apache/solr/parser/QueryParser.java index e846c6e631b..2b64b88b68d 100644 --- a/solr/core/src/java/org/apache/solr/parser/QueryParser.java +++ b/solr/core/src/java/org/apache/solr/parser/QueryParser.java @@ -52,12 +52,13 @@ public class QueryParser extends SolrQueryParserBase implements QueryParserConst @Override protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, - boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { + boolean quoted, boolean fieldAutoGenPhraseQueries, boolean fieldEnableGraphQueries) + throws SyntaxError { setAutoGenerateMultiTermSynonymsPhraseQuery(fieldAutoGenPhraseQueries || getAutoGeneratePhraseQueries()); // Don't auto-quote graph-aware field queries boolean treatAsQuoted = getSplitOnWhitespace() ? (quoted || fieldAutoGenPhraseQueries || getAutoGeneratePhraseQueries()) : quoted; - return super.newFieldQuery(analyzer, field, queryText, treatAsQuoted, false); + return super.newFieldQuery(analyzer, field, queryText, treatAsQuoted, false, fieldEnableGraphQueries); } // * Query ::= ( Clause )* diff --git a/solr/core/src/java/org/apache/solr/parser/QueryParser.jj b/solr/core/src/java/org/apache/solr/parser/QueryParser.jj index d4d65396eca..c07b28d477d 100644 --- a/solr/core/src/java/org/apache/solr/parser/QueryParser.jj +++ b/solr/core/src/java/org/apache/solr/parser/QueryParser.jj @@ -76,12 +76,13 @@ public class QueryParser extends SolrQueryParserBase { @Override protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, - boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { + boolean quoted, boolean fieldAutoGenPhraseQueries, boolean fieldEnableGraphQueries) + throws SyntaxError { setAutoGenerateMultiTermSynonymsPhraseQuery(fieldAutoGenPhraseQueries || getAutoGeneratePhraseQueries()); // Don't auto-quote graph-aware field queries boolean treatAsQuoted = getSplitOnWhitespace() ? (quoted || fieldAutoGenPhraseQueries || getAutoGeneratePhraseQueries()) : quoted; - return super.newFieldQuery(analyzer, field, queryText, treatAsQuoted, false); + return super.newFieldQuery(analyzer, field, queryText, treatAsQuoted, false, fieldEnableGraphQueries); } } 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 08ccdd11b58..21e0aa0630a 100644 --- a/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java +++ b/solr/core/src/java/org/apache/solr/parser/SolrQueryParserBase.java @@ -437,10 +437,14 @@ public abstract class SolrQueryParserBase extends QueryBuilder { } protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, - boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { + boolean quoted, boolean fieldAutoGenPhraseQueries, boolean fieldEnableGraphQueries) + throws SyntaxError { BooleanClause.Occur occur = operator == Operator.AND ? BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD; - return createFieldQuery(analyzer, occur, field, queryText, + setEnableGraphQueries(fieldEnableGraphQueries); + Query query = createFieldQuery(analyzer, occur, field, queryText, quoted || fieldAutoGenPhraseQueries || autoGeneratePhraseQueries, phraseSlop); + setEnableGraphQueries(true); // reset back to default + return query; } @@ -632,8 +636,10 @@ public abstract class SolrQueryParserBase extends QueryBuilder { Query subq; if (ft.isTokenized() && sfield.indexed()) { boolean fieldAutoGenPhraseQueries = ft instanceof TextField && ((TextField)ft).getAutoGeneratePhraseQueries(); + boolean fieldEnableGraphQueries = ft instanceof TextField && ((TextField)ft).getEnableGraphQueries(); + subq = newFieldQuery(getAnalyzer(), sfield.getName(), rawq.getJoinedExternalVal(), - false, fieldAutoGenPhraseQueries); + false, fieldAutoGenPhraseQueries, fieldEnableGraphQueries); booleanBuilder.add(subq, BooleanClause.Occur.SHOULD); } else { for (String externalVal : rawq.getExternalVals()) { @@ -949,7 +955,8 @@ public abstract class SolrQueryParserBase extends QueryBuilder { // delegate to type for everything except tokenized fields if (ft.isTokenized() && sf.indexed()) { boolean fieldAutoGenPhraseQueries = ft instanceof TextField && ((TextField)ft).getAutoGeneratePhraseQueries(); - return newFieldQuery(getAnalyzer(), field, queryText, quoted, fieldAutoGenPhraseQueries); + boolean fieldEnableGraphQueries = ft instanceof TextField && ((TextField)ft).getEnableGraphQueries(); + return newFieldQuery(getAnalyzer(), field, queryText, quoted, fieldAutoGenPhraseQueries, fieldEnableGraphQueries); } else { if (raw) { return new RawQuery(sf, queryText); @@ -960,7 +967,7 @@ public abstract class SolrQueryParserBase extends QueryBuilder { } // default to a normal field query - return newFieldQuery(getAnalyzer(), field, queryText, quoted, false); + return newFieldQuery(getAnalyzer(), field, queryText, quoted, false, true); } // Assumption: quoted is always false @@ -993,7 +1000,9 @@ public abstract class SolrQueryParserBase extends QueryBuilder { if (ft.isTokenized() && sf.indexed()) { String queryText = queryTerms.size() == 1 ? queryTerms.get(0) : String.join(" ", queryTerms); boolean fieldAutoGenPhraseQueries = ft instanceof TextField && ((TextField)ft).getAutoGeneratePhraseQueries(); - return newFieldQuery(getAnalyzer(), field, queryText, false, fieldAutoGenPhraseQueries); + boolean fieldEnableGraphQueries = ft instanceof TextField && ((TextField)ft).getEnableGraphQueries(); + return newFieldQuery + (getAnalyzer(), field, queryText, false, fieldAutoGenPhraseQueries, fieldEnableGraphQueries); } else { if (raw) { return new RawQuery(sf, queryTerms); @@ -1025,7 +1034,7 @@ public abstract class SolrQueryParserBase extends QueryBuilder { // default to a normal field query String queryText = queryTerms.size() == 1 ? queryTerms.get(0) : String.join(" ", queryTerms); - return newFieldQuery(getAnalyzer(), field, queryText, false, false); + return newFieldQuery(getAnalyzer(), field, queryText, false, false, true); } protected boolean isRangeShouldBeProtectedFromReverse(String field, String part1){ diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java index 016e1666a9c..07eb866c7c4 100644 --- a/solr/core/src/java/org/apache/solr/schema/FieldType.java +++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java @@ -833,7 +833,8 @@ public abstract class FieldType extends FieldProperties { private static final String POSTINGS_FORMAT = "postingsFormat"; private static final String DOC_VALUES_FORMAT = "docValuesFormat"; - private static final String AUTO_GENERATE_PHRASE_QUERIES = "autoGeneratePhraseQueries"; + protected static final String AUTO_GENERATE_PHRASE_QUERIES = "autoGeneratePhraseQueries"; + protected static final String ENABLE_GRAPH_QUERIES = "enableGraphQueries"; private static final String ARGS = "args"; private static final String POSITION_INCREMENT_GAP = "positionIncrementGap"; @@ -856,6 +857,7 @@ public abstract class FieldType extends FieldProperties { } if (this instanceof TextField) { namedPropertyValues.add(AUTO_GENERATE_PHRASE_QUERIES, ((TextField) this).getAutoGeneratePhraseQueries()); + namedPropertyValues.add(ENABLE_GRAPH_QUERIES, ((TextField) this).getEnableGraphQueries()); } namedPropertyValues.add(getPropertyName(INDEXED), hasProperty(INDEXED)); namedPropertyValues.add(getPropertyName(STORED), hasProperty(STORED)); diff --git a/solr/core/src/java/org/apache/solr/schema/TextField.java b/solr/core/src/java/org/apache/solr/schema/TextField.java index 3d34df3300b..d8bae24df6e 100644 --- a/solr/core/src/java/org/apache/solr/schema/TextField.java +++ b/solr/core/src/java/org/apache/solr/schema/TextField.java @@ -41,6 +41,7 @@ import org.apache.solr.uninverting.UninvertingReader.Type; */ public class TextField extends FieldType { protected boolean autoGeneratePhraseQueries; + protected boolean enableGraphQueries; /** * Analyzer set by schema for text types to use when searching fields @@ -69,9 +70,15 @@ public class TextField extends FieldType { } else { autoGeneratePhraseQueries = true; } - String autoGeneratePhraseQueriesStr = args.remove("autoGeneratePhraseQueries"); + String autoGeneratePhraseQueriesStr = args.remove(AUTO_GENERATE_PHRASE_QUERIES); if (autoGeneratePhraseQueriesStr != null) autoGeneratePhraseQueries = Boolean.parseBoolean(autoGeneratePhraseQueriesStr); + + enableGraphQueries = true; + String enableGraphQueriesStr = args.remove(ENABLE_GRAPH_QUERIES); + if (enableGraphQueriesStr != null) + enableGraphQueries = Boolean.parseBoolean(enableGraphQueriesStr); + super.init(schema, args); } @@ -93,6 +100,10 @@ public class TextField extends FieldType { public boolean getAutoGeneratePhraseQueries() { return autoGeneratePhraseQueries; } + + public boolean getEnableGraphQueries() { + return enableGraphQueries; + } @Override public SortField getSortField(SchemaField field, boolean reverse) { diff --git a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java index 9825f72b9f3..07d7e516394 100644 --- a/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java +++ b/solr/core/src/java/org/apache/solr/search/ExtendedDismaxQParser.java @@ -1080,7 +1080,8 @@ public class ExtendedDismaxQParser extends QParser { @Override protected Query newFieldQuery(Analyzer analyzer, String field, String queryText, - boolean quoted, boolean fieldAutoGenPhraseQueries) throws SyntaxError { + boolean quoted, boolean fieldAutoGenPhraseQueries, boolean enableGraphQueries) + throws SyntaxError { Analyzer actualAnalyzer; if (removeStopFilter) { if (nonStopFilterAnalyzerPerField == null) { @@ -1093,7 +1094,7 @@ public class ExtendedDismaxQParser extends QParser { } else { actualAnalyzer = parser.getReq().getSchema().getFieldType(field).getQueryAnalyzer(); } - return super.newFieldQuery(actualAnalyzer, field, queryText, quoted, fieldAutoGenPhraseQueries); + return super.newFieldQuery(actualAnalyzer, field, queryText, quoted, fieldAutoGenPhraseQueries, enableGraphQueries); } @Override diff --git a/solr/core/src/test-files/solr/collection1/conf/schema12.xml b/solr/core/src/test-files/solr/collection1/conf/schema12.xml index 7ea770be1fb..5a85c1f2044 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema12.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema12.xml @@ -467,7 +467,14 @@ - + + + + + + + @@ -568,6 +575,8 @@ + + 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 36e8c1031f7..de458445fe4 100644 --- a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java +++ b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java @@ -16,22 +16,48 @@ */ package org.apache.solr.schema; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; import java.util.Locale; import java.util.Set; import java.util.TreeSet; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.DoublePoint; +import org.apache.lucene.document.FloatPoint; +import org.apache.lucene.document.IntPoint; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.PointValues; +import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexOrDocValuesQuery; import org.apache.lucene.search.PointRangeQuery; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.schema.IndexSchema.DynamicField; +import org.apache.solr.search.SolrIndexSearcher; import org.apache.solr.util.DateMathParser; +import org.apache.solr.util.RefCounted; import org.junit.After; import org.junit.BeforeClass; import org.junit.Test; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.ibm.icu.text.SimpleDateFormat; /** * Tests for PointField functionality @@ -57,17 +83,24 @@ public class TestPointFields extends SolrTestCaseJ4 { public void testIntPointFieldExactQuery() throws Exception { doTestIntPointFieldExactQuery("number_p_i", false); doTestIntPointFieldExactQuery("number_p_i_mv", false); + doTestIntPointFieldExactQuery("number_p_i_dv", false); + doTestIntPointFieldExactQuery("number_p_i_mv_dv", false); doTestIntPointFieldExactQuery("number_p_i_ni_dv", false); doTestIntPointFieldExactQuery("number_p_i_ni_ns_dv", false); doTestIntPointFieldExactQuery("number_p_i_ni_mv_dv", false); } + @Test + public void testIntPointFieldNonSearchableExactQuery() throws Exception { + doTestIntPointFieldExactQuery("number_p_i_ni", false, false); + doTestIntPointFieldExactQuery("number_p_i_ni_ns", false, false); + } + @Test public void testIntPointFieldReturn() throws Exception { testPointFieldReturn("number_p_i", "int", new String[]{"0", "-1", "2", "3", "43", "52", "-60", "74", "80", "99"}); - clearIndex(); - assertU(commit()); testPointFieldReturn("number_p_i_dv_ns", "int", new String[]{"0", "-1", "2", "3", "43", "52", "-60", "74", "80", "99"}); + testPointFieldReturn("number_p_i_ni", "int", new String[]{"0", "-1", "2", "3", "43", "52", "-60", "74", "80", "99"}); } @Test @@ -77,6 +110,13 @@ public class TestPointFields extends SolrTestCaseJ4 { doTestIntPointFieldRangeQuery("number_p_i_dv", "int", false); } + @Test + public void testIntPointFieldNonSearchableRangeQuery() throws Exception { + doTestPointFieldNonSearchableRangeQuery("number_p_i_ni", "42"); + doTestPointFieldNonSearchableRangeQuery("number_p_i_ni_ns", "42"); + doTestPointFieldNonSearchableRangeQuery("number_p_i_ni_ns_mv", "42", "666"); + } + @Test public void testIntPointFieldSort() throws Exception { doTestPointFieldSort("number_p_i", "number_p_i_dv", new String[]{"0", "1", "2", "3", "4", "5", "6", "7", "8", "9"}); @@ -112,6 +152,12 @@ public class TestPointFields extends SolrTestCaseJ4 { testPointFieldMultiValuedExactQuery("number_p_i_mv", getSequentialStringArrayWithInts(20)); testPointFieldMultiValuedExactQuery("number_p_i_ni_mv_dv", getSequentialStringArrayWithInts(20)); } + + @Test + public void testIntPointFieldMultiValuedNonSearchableExactQuery() throws Exception { + testPointFieldMultiValuedExactQuery("number_p_i_ni_mv", getSequentialStringArrayWithInts(20), false); + testPointFieldMultiValuedExactQuery("number_p_i_ni_ns_mv", getSequentialStringArrayWithInts(20), false); + } @Test public void testIntPointFieldMultiValuedReturn() throws Exception { @@ -127,6 +173,12 @@ public class TestPointFields extends SolrTestCaseJ4 { testPointFieldMultiValuedRangeQuery("number_p_i_mv_dv", "int", getSequentialStringArrayWithInts(20)); } + @Test + public void testIntPointFieldNotIndexed() throws Exception { + doTestFieldNotIndexed("number_p_i_ni", getSequentialStringArrayWithInts(10)); + doTestFieldNotIndexed("number_p_i_ni_mv", getSequentialStringArrayWithInts(10)); + } + //TODO MV SORT? @Test public void testIntPointFieldMultiValuedFacetField() throws Exception { @@ -178,19 +230,24 @@ public class TestPointFields extends SolrTestCaseJ4 { doTestFloatPointFieldExactQuery("number_d"); doTestFloatPointFieldExactQuery("number_p_d"); doTestFloatPointFieldExactQuery("number_p_d_mv"); + doTestFloatPointFieldExactQuery("number_p_d_dv"); + doTestFloatPointFieldExactQuery("number_p_d_mv_dv"); doTestFloatPointFieldExactQuery("number_p_d_ni_dv"); doTestFloatPointFieldExactQuery("number_p_d_ni_ns_dv"); + doTestFloatPointFieldExactQuery("number_p_d_ni_dv_ns"); doTestFloatPointFieldExactQuery("number_p_d_ni_mv_dv"); } + @Test + public void testDoublePointFieldNonSearchableExactQuery() throws Exception { + doTestFloatPointFieldExactQuery("number_p_d_ni", false); + doTestFloatPointFieldExactQuery("number_p_d_ni_ns", false); + } + @Test public void testDoublePointFieldReturn() throws Exception { testPointFieldReturn("number_p_d", "double", new String[]{"0.0", "1.2", "2.5", "3.02", "0.43", "5.2", "6.01", "74.0", "80.0", "9.9"}); - clearIndex(); - assertU(commit()); testPointFieldReturn("number_p_d_dv_ns", "double", new String[]{"0.0", "1.2", "2.5", "3.02", "0.43", "5.2", "6.01", "74.0", "80.0", "9.9"}); - clearIndex(); - assertU(commit()); String[] arr = new String[atLeast(10)]; for (int i = 0; i < arr.length; i++) { double rand = random().nextDouble() * 10; @@ -206,6 +263,14 @@ public class TestPointFields extends SolrTestCaseJ4 { doTestFloatPointFieldRangeQuery("number_p_d_dv", "double", true); } + @Test + public void testDoubleFieldNonSearchableRangeQuery() throws Exception { + doTestPointFieldNonSearchableRangeQuery("number_p_d_ni", "42.3"); + doTestPointFieldNonSearchableRangeQuery("number_p_d_ni_ns", "42.3"); + doTestPointFieldNonSearchableRangeQuery("number_p_d_ni_ns_mv", "42.3", "-66.6"); + } + + @Test public void testDoublePointFieldSort() throws Exception { String[] arr = getRandomStringArrayWithDoubles(10, true); @@ -244,6 +309,12 @@ public class TestPointFields extends SolrTestCaseJ4 { testPointFieldMultiValuedExactQuery("number_p_d_ni_mv_dv", getRandomStringArrayWithDoubles(20, false)); } + @Test + public void testDoublePointFieldMultiValuedNonSearchableExactQuery() throws Exception { + testPointFieldMultiValuedExactQuery("number_p_d_ni_mv", getRandomStringArrayWithDoubles(20, false), false); + testPointFieldMultiValuedExactQuery("number_p_d_ni_ns_mv", getRandomStringArrayWithDoubles(20, false), false); + } + @Test public void testDoublePointFieldMultiValuedReturn() throws Exception { testPointFieldMultiValuedReturn("number_p_d_mv", "double", getSequentialStringArrayWithDoubles(20)); @@ -295,6 +366,12 @@ public class TestPointFields extends SolrTestCaseJ4 { testMultiValuedFloatPointFieldsAtomicUpdates("number_p_d_dv_ns_mv", "double"); } + @Test + public void testDoublePointFieldNotIndexed() throws Exception { + doTestFieldNotIndexed("number_p_d_ni", getSequentialStringArrayWithDoubles(10)); + doTestFieldNotIndexed("number_p_d_ni_mv", getSequentialStringArrayWithDoubles(10)); + } + private void doTestFloatPointFieldsAtomicUpdates(String field, String type) throws Exception { assertU(adoc(sdoc("id", "1", field, "1.1234"))); @@ -342,19 +419,24 @@ public class TestPointFields extends SolrTestCaseJ4 { public void testFloatPointFieldExactQuery() throws Exception { doTestFloatPointFieldExactQuery("number_p_f"); doTestFloatPointFieldExactQuery("number_p_f_mv"); + doTestFloatPointFieldExactQuery("number_p_f_dv"); + doTestFloatPointFieldExactQuery("number_p_f_mv_dv"); doTestFloatPointFieldExactQuery("number_p_f_ni_dv"); doTestFloatPointFieldExactQuery("number_p_f_ni_ns_dv"); + doTestFloatPointFieldExactQuery("number_p_f_ni_dv_ns"); doTestFloatPointFieldExactQuery("number_p_f_ni_mv_dv"); } + @Test + public void testFloatPointFieldNonSearchableExactQuery() throws Exception { + doTestFloatPointFieldExactQuery("number_p_f_ni", false); + doTestFloatPointFieldExactQuery("number_p_f_ni_ns", false); + } + @Test public void testFloatPointFieldReturn() throws Exception { testPointFieldReturn("number_p_f", "float", new String[]{"0.0", "-1.2", "2.5", "3.02", "0.43", "5.2", "6.01", "74.0", "80.0", "9.9"}); - clearIndex(); - assertU(commit()); testPointFieldReturn("number_p_f_dv_ns", "float", new String[]{"0.0", "-1.2", "2.5", "3.02", "0.43", "5.2", "6.01", "74.0", "80.0", "9.9"}); - clearIndex(); - assertU(commit()); String[] arr = new String[atLeast(10)]; for (int i = 0; i < arr.length; i++) { float rand = random().nextFloat() * 10; @@ -370,6 +452,13 @@ public class TestPointFields extends SolrTestCaseJ4 { doTestFloatPointFieldRangeQuery("number_p_f_dv", "float", false); } + @Test + public void testFloatPointFieldNonSearchableRangeQuery() throws Exception { + doTestPointFieldNonSearchableRangeQuery("number_p_f_ni", "42.3"); + doTestPointFieldNonSearchableRangeQuery("number_p_f_ni_ns", "42.3"); + doTestPointFieldNonSearchableRangeQuery("number_p_f_ni_ns_mv", "42.3", "-66.6"); + } + @Test public void testFloatPointFieldSort() throws Exception { String[] arr = getRandomStringArrayWithFloats(10, true); @@ -408,6 +497,12 @@ public class TestPointFields extends SolrTestCaseJ4 { testPointFieldMultiValuedExactQuery("number_p_f_ni_mv_dv", getRandomStringArrayWithFloats(20, false)); } + @Test + public void testFloatPointFieldMultiValuedNonSearchableExactQuery() throws Exception { + testPointFieldMultiValuedExactQuery("number_p_f_ni_mv", getRandomStringArrayWithFloats(20, false), false); + testPointFieldMultiValuedExactQuery("number_p_f_ni_ns_mv", getRandomStringArrayWithFloats(20, false), false); + } + @Test public void testFloatPointFieldMultiValuedReturn() throws Exception { testPointFieldMultiValuedReturn("number_p_f_mv", "float", getSequentialStringArrayWithDoubles(20)); @@ -467,22 +562,35 @@ public class TestPointFields extends SolrTestCaseJ4 { doTestSetQueries("number_p_f_ni_dv", getRandomStringArrayWithFloats(10, false), false); } + @Test + public void testFloatPointFieldNotIndexed() throws Exception { + doTestFieldNotIndexed("number_p_f_ni", getSequentialStringArrayWithDoubles(10)); + doTestFieldNotIndexed("number_p_f_ni_mv", getSequentialStringArrayWithDoubles(10)); + } + // Long @Test public void testLongPointFieldExactQuery() throws Exception { doTestIntPointFieldExactQuery("number_p_l", true); doTestIntPointFieldExactQuery("number_p_l_mv", true); + doTestIntPointFieldExactQuery("number_p_l_dv", true); + doTestIntPointFieldExactQuery("number_p_l_mv_dv", true); doTestIntPointFieldExactQuery("number_p_l_ni_dv", true); doTestIntPointFieldExactQuery("number_p_l_ni_ns_dv", true); + doTestIntPointFieldExactQuery("number_p_l_ni_dv_ns", true); doTestIntPointFieldExactQuery("number_p_l_ni_mv_dv", true); } + @Test + public void testLongPointFieldNonSearchableExactQuery() throws Exception { + doTestIntPointFieldExactQuery("number_p_l_ni", true, false); + doTestIntPointFieldExactQuery("number_p_l_ni_ns", true, false); + } + @Test public void testLongPointFieldReturn() throws Exception { testPointFieldReturn("number_p_l", "long", new String[]{"0", "-1", "2", "3", "43", "52", "-60", "74", "80", "99", String.valueOf(Long.MAX_VALUE)}); - clearIndex(); - assertU(commit()); testPointFieldReturn("number_p_l_dv_ns", "long", new String[]{"0", "-1", "2", "3", "43", "52", "-60", "74", "80", "99", String.valueOf(Long.MAX_VALUE)}); } @@ -493,6 +601,13 @@ public class TestPointFields extends SolrTestCaseJ4 { doTestIntPointFieldRangeQuery("number_p_l_dv", "long", true); } + @Test + public void testLongPointFieldNonSearchableRangeQuery() throws Exception { + doTestPointFieldNonSearchableRangeQuery("number_p_l_ni", "3333333333"); + doTestPointFieldNonSearchableRangeQuery("number_p_l_ni_ns", "3333333333"); + doTestPointFieldNonSearchableRangeQuery("number_p_l_ni_ns_mv", "3333333333", "-4444444444"); + } + @Test public void testLongPointFieldSort() throws Exception { doTestPointFieldSort("number_p_l", "number_p_l_dv", new String[]{String.valueOf(Integer.MIN_VALUE), @@ -532,6 +647,12 @@ public class TestPointFields extends SolrTestCaseJ4 { testPointFieldMultiValuedExactQuery("number_p_l_ni_mv_dv", getSequentialStringArrayWithInts(20)); } + @Test + public void testLongPointFieldMultiValuedNonSearchableExactQuery() throws Exception { + testPointFieldMultiValuedExactQuery("number_p_l_ni_mv", getSequentialStringArrayWithInts(20), false); + testPointFieldMultiValuedExactQuery("number_p_l_ni_ns_mv", getSequentialStringArrayWithInts(20), false); + } + @Test public void testLongPointFieldMultiValuedReturn() throws Exception { testPointFieldMultiValuedReturn("number_p_l_mv", "long", getSequentialStringArrayWithInts(20)); @@ -588,6 +709,12 @@ public class TestPointFields extends SolrTestCaseJ4 { doTestSetQueries("number_p_l_mv", getRandomStringArrayWithLongs(10, false), true); doTestSetQueries("number_p_l_ni_dv", getRandomStringArrayWithLongs(10, false), false); } + + @Test + public void testLongPointFieldNotIndexed() throws Exception { + doTestFieldNotIndexed("number_p_l_ni", getSequentialStringArrayWithInts(10)); + doTestFieldNotIndexed("number_p_l_ni_mv", getSequentialStringArrayWithInts(10)); + } // Date @@ -595,18 +722,24 @@ public class TestPointFields extends SolrTestCaseJ4 { public void testDatePointFieldExactQuery() throws Exception { doTestDatePointFieldExactQuery("number_p_dt", "1995-12-31T23:59:59Z"); doTestDatePointFieldExactQuery("number_p_dt_mv", "2015-12-31T23:59:59Z-1DAY"); + doTestDatePointFieldExactQuery("number_p_dt_dv", "2000-12-31T23:59:59Z+3DAYS"); + doTestDatePointFieldExactQuery("number_p_dt_mv_dv", "2000-12-31T23:59:59Z+3DAYS"); doTestDatePointFieldExactQuery("number_p_dt_ni_dv", "2000-12-31T23:59:59Z+3DAYS"); doTestDatePointFieldExactQuery("number_p_dt_ni_ns_dv", "1995-12-31T23:59:59Z-1MONTH"); doTestDatePointFieldExactQuery("number_p_dt_ni_mv_dv", "1995-12-31T23:59:59Z+2MONTHS"); } + @Test + public void testDatePointFieldNonSearchableExactQuery() throws Exception { + doTestDatePointFieldExactQuery("number_p_dt_ni", "1995-12-31T23:59:59Z", false); + doTestDatePointFieldExactQuery("number_p_dt_ni_ns", "1995-12-31T23:59:59Z", false); + + } @Test public void testDatePointFieldReturn() throws Exception { testPointFieldReturn("number_p_dt", "date", new String[]{"1995-12-31T23:59:59Z", "1994-02-28T23:59:59Z", "2015-12-31T23:59:59Z", "2000-10-31T23:59:59Z", "1999-12-31T12:59:59Z"}); - clearIndex(); - assertU(commit()); testPointFieldReturn("number_p_dt_dv_ns", "date", new String[]{"1995-12-31T23:59:59Z", "1994-02-28T23:59:59Z", "2015-12-31T23:59:59Z", "2000-10-31T23:59:59Z", "1999-12-31T12:59:59Z"}); @@ -617,6 +750,13 @@ public class TestPointFields extends SolrTestCaseJ4 { doTestDatePointFieldRangeQuery("number_p_dt"); doTestDatePointFieldRangeQuery("number_p_dt_ni_ns_dv"); } + + @Test + public void testDatePointFieldNonSearchableRangeQuery() throws Exception { + doTestPointFieldNonSearchableRangeQuery("number_p_dt_ni", "1995-12-31T23:59:59Z"); + doTestPointFieldNonSearchableRangeQuery("number_p_dt_ni_ns", "1995-12-31T23:59:59Z"); + doTestPointFieldNonSearchableRangeQuery("number_p_dt_ni_ns_mv", "1995-12-31T23:59:59Z", "2000-10-31T23:59:59Z"); + } @Test public void testDatePointFieldSort() throws Exception { @@ -653,6 +793,12 @@ public class TestPointFields extends SolrTestCaseJ4 { testPointFieldMultiValuedExactQuery("number_p_dt_ni_mv_dv", getSequentialStringArrayWithDates(20)); } + @Test + public void testDatePointFieldMultiValuedNonSearchableExactQuery() throws Exception { + testPointFieldMultiValuedExactQuery("number_p_dt_ni_mv", getSequentialStringArrayWithDates(20), false); + testPointFieldMultiValuedExactQuery("number_p_dt_ni_ns_mv", getSequentialStringArrayWithDates(20), false); + } + @Test public void testDatePointFieldMultiValuedReturn() throws Exception { testPointFieldMultiValuedReturn("number_p_dt_mv", "date", getSequentialStringArrayWithDates(20)); @@ -709,6 +855,13 @@ public class TestPointFields extends SolrTestCaseJ4 { doTestSetQueries("number_p_dt_ni_dv", getRandomStringArrayWithDates(10, false), false); } + + @Test + public void testDatePointFieldNotIndexed() throws Exception { + doTestFieldNotIndexed("number_p_dt_ni", getSequentialStringArrayWithDates(10)); + doTestFieldNotIndexed("number_p_dt_ni_mv", getSequentialStringArrayWithDates(10)); + } + @Test public void testIndexOrDocValuesQuery() throws Exception { String[] fieldTypeNames = new String[]{"_p_i", "_p_l", "_p_d", "_p_f"}; @@ -730,6 +883,30 @@ public class TestPointFields extends SolrTestCaseJ4 { } } + public void testInternals() throws IOException { + String[] types = new String[]{"i", "l", "f", "d"}; + String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; + Set typesTested = new HashSet<>(); + for (String type:types) { + for (String suffix:suffixes) { + doTestInternals("number_p_" + type + suffix, getSequentialStringArrayWithInts(10)); + typesTested.add("*_p_" + type + suffix); + } + } + for (String suffix:suffixes) { + doTestInternals("number_p_dt" + suffix, getSequentialStringArrayWithDates(10)); + typesTested.add("*_p_dt" + suffix); + } + + Set typesToTest = new HashSet<>(); + for (DynamicField dynField:h.getCore().getLatestSchema().getDynamicFields()) { + if (dynField.getPrototype().getType() instanceof PointField) { + typesToTest.add(dynField.getRegex()); + } + } + assertEquals("Missing types in the test", typesTested, typesToTest); + } + // Helper methods private String[] getRandomStringArrayWithDoubles(int length, boolean sorted) { @@ -870,24 +1047,61 @@ public class TestPointFields extends SolrTestCaseJ4 { return stringArr; } - private void doTestIntPointFieldExactQuery(String field, boolean testLong) throws Exception { + private void doTestFieldNotIndexed(String field, String[] values) throws IOException { + assert values.length == 10; + // test preconditions + SchemaField sf = h.getCore().getLatestSchema().getField(field); + assertFalse("Field should be indexed=false", sf.indexed()); + assertFalse("Field should be docValues=false", sf.hasDocValues()); + + for (int i=0; i < 10; i++) { + assertU(adoc("id", String.valueOf(i), field, values[i])); + } + assertU(commit()); + assertQ(req("q", "*:*"), "//*[@numFound='10']"); + assertQ("Can't search on index=false docValues=false field", req("q", field + ":[* TO *]"), "//*[@numFound='0']"); + IndexReader ir; + RefCounted ref = null; + try { + ref = h.getCore().getSearcher(); + ir = ref.get().getIndexReader(); + assertEquals("Field " + field + " should have no point values", 0, PointValues.size(ir, field)); + } finally { + ref.decref(); + } + } + + + private void doTestIntPointFieldExactQuery(final String field, final boolean testLong) throws Exception { + doTestIntPointFieldExactQuery(field, testLong, true); + } + + /** + * @param field the field to use for indexing and searching against + * @param testLong set to true if "field" is expected to support long values, false if only integers + * @param searchable set to true if searches against "field" should succeed, false if field is only stored and searches should always get numFound=0 + */ + private void doTestIntPointFieldExactQuery(final String field, final boolean testLong, final boolean searchable) throws Exception { + final String MATCH_ONE = "//*[@numFound='" + (searchable ? "1" : "0") + "']"; + final String MATCH_TWO = "//*[@numFound='" + (searchable ? "2" : "0") + "']"; + for (int i=0; i < 10; i++) { assertU(adoc("id", String.valueOf(i), field, String.valueOf(i+1))); } assertU(commit()); for (int i = 0; i < 10; i++) { assertQ(req("q", field + ":"+(i+1), "fl", "id, " + field), - "//*[@numFound='1']"); + MATCH_ONE); } for (int i = 0; i < 10; i++) { - assertQ(req("q", field + ":" + (i+1) + " OR " + field + ":" + ((i+1)%10 + 1)), "//*[@numFound='2']"); + assertQ(req("debug", "true", "q", field + ":" + (i+1) + " OR " + field + ":" + ((i+1)%10 + 1)), MATCH_TWO); } assertU(adoc("id", String.valueOf(Integer.MAX_VALUE), field, String.valueOf(Integer.MAX_VALUE))); assertU(commit()); assertQ(req("q", field + ":"+Integer.MAX_VALUE, "fl", "id, " + field), - "//*[@numFound='1']"); + MATCH_ONE); if (testLong) { for (long i = (long)Integer.MAX_VALUE; i < (long)Integer.MAX_VALUE + 10; i++) { @@ -896,12 +1110,12 @@ public class TestPointFields extends SolrTestCaseJ4 { assertU(commit()); for (long i = (long)Integer.MAX_VALUE; i < (long)Integer.MAX_VALUE + 10; i++) { assertQ(req("q", field + ":"+(i+1), "fl", "id, " + field), - "//*[@numFound='1']"); + MATCH_ONE); } assertU(adoc("id", String.valueOf(Long.MAX_VALUE), field, String.valueOf(Long.MAX_VALUE))); assertU(commit()); assertQ(req("q", field + ":"+Long.MAX_VALUE, "fl", "id, " + field), - "//*[@numFound='1']"); + MATCH_ONE); } clearIndex(); @@ -937,6 +1151,21 @@ public class TestPointFields extends SolrTestCaseJ4 { "//doc/" + type + "[@name='" + field + "'][.='" + values[i] + "']"); } } + clearIndex(); + assertU(commit()); + } + + private void doTestPointFieldNonSearchableRangeQuery(String fieldName, String... values) throws Exception { + for (int i = 9; i >= 0; i--) { + SolrInputDocument doc = sdoc("id", String.valueOf(i)); + for (String value : values) { + doc.addField(fieldName, value); + } + assertU(adoc(doc)); + } + assertU(commit()); + assertQ(req("q", fieldName + ":[* TO *]", "fl", "id, " + fieldName, "sort", "id asc"), + "//*[@numFound='0']"); } private void doTestIntPointFieldRangeQuery(String fieldName, String type, boolean testLong) throws Exception { @@ -1185,8 +1414,23 @@ public class TestPointFields extends SolrTestCaseJ4 { req("q", "*:*", "fl", "id, " + field, "stats", "true", "stats.field", field), SolrException.ErrorCode.BAD_REQUEST); } - - private void testPointFieldMultiValuedExactQuery(String fieldName, String[] numbers) throws Exception { + + + private void testPointFieldMultiValuedExactQuery(final String fieldName, final String[] numbers) throws Exception { + testPointFieldMultiValuedExactQuery(fieldName, numbers, true); + } + + /** + * @param fieldName the field to use for indexing and searching against + * @param numbers list of 20 values to index in 10 docs (pairwise) + * @param searchable set to true if searches against "field" should succeed, false if field is only stored and searches should always get numFound=0 + */ + private void testPointFieldMultiValuedExactQuery(final String fieldName, final String[] numbers, + final boolean searchable) throws Exception { + + final String MATCH_ONE = "//*[@numFound='" + (searchable ? "1" : "0") + "']"; + final String MATCH_TWO = "//*[@numFound='" + (searchable ? "2" : "0") + "']"; + assert numbers != null && numbers.length == 20; assertTrue(h.getCore().getLatestSchema().getField(fieldName).multiValued()); assertTrue(h.getCore().getLatestSchema().getField(fieldName).getType() instanceof PointField); @@ -1197,18 +1441,20 @@ public class TestPointFields extends SolrTestCaseJ4 { for (int i = 0; i < 20; i++) { if (h.getCore().getLatestSchema().getField(fieldName).getType() instanceof DatePointField) { assertQ(req("q", fieldName + ":\"" + numbers[i] + "\""), - "//*[@numFound='1']"); + MATCH_ONE); } else { assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-")), - "//*[@numFound='1']"); + MATCH_ONE); } } for (int i = 0; i < 20; i++) { if (h.getCore().getLatestSchema().getField(fieldName).getType() instanceof DatePointField) { - assertQ(req("q", fieldName + ":\"" + numbers[i] + "\"" + " OR " + fieldName + ":\"" + numbers[(i+1)%10]+"\""), "//*[@numFound='2']"); + assertQ(req("q", fieldName + ":\"" + numbers[i] + "\"" + " OR " + fieldName + ":\"" + numbers[(i+1)%10]+"\""), + MATCH_TWO); } else { - assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-") + " OR " + fieldName + ":" + numbers[(i+1)%10].replace("-", "\\-")), "//*[@numFound='2']"); + assertQ(req("q", fieldName + ":" + numbers[i].replace("-", "\\-") + " OR " + fieldName + ":" + numbers[(i+1)%10].replace("-", "\\-")), + MATCH_TWO); } } } @@ -1248,7 +1494,7 @@ public class TestPointFields extends SolrTestCaseJ4 { } 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()); @@ -1605,20 +1851,32 @@ public class TestPointFields extends SolrTestCaseJ4 { "//result/doc[1]/" + type + "[@name='" + field + "'][.='3']"); } - private void doTestFloatPointFieldExactQuery(String field) throws Exception { + + private void doTestFloatPointFieldExactQuery(final String field) throws Exception { + doTestFloatPointFieldExactQuery(field, true); + } + /** + * @param field the field to use for indexing and searching against + * @param searchable set to true if searches against "field" should succeed, false if field is only stored and searches should always get numFound=0 + */ + private void doTestFloatPointFieldExactQuery(String field, final boolean searchable) throws Exception { + final String MATCH_ONE = "//*[@numFound='" + (searchable ? "1" : "0") + "']"; + final String MATCH_TWO = "//*[@numFound='" + (searchable ? "2" : "0") + "']"; + for (int i=0; i < 10; i++) { assertU(adoc("id", String.valueOf(i), field, String.valueOf(i + "." + i))); } assertU(commit()); for (int i = 0; i < 9; i++) { assertQ(req("q", field + ":"+(i+1) + "." + (i+1), "fl", "id, " + field), - "//*[@numFound='1']"); + MATCH_ONE); } for (int i = 0; i < 9; i++) { String num1 = (i+1) + "." + (i+1); String num2 = ((i+1)%9 + 1) + "." + ((i+1)%9 + 1); - assertQ(req("q", field + ":" + num1 + " OR " + field + ":" + num2), "//*[@numFound='2']"); + assertQ(req("q", field + ":" + num1 + " OR " + field + ":" + num2), + MATCH_TWO); } clearIndex(); @@ -1628,7 +1886,7 @@ public class TestPointFields extends SolrTestCaseJ4 { assertU(adoc("id", "random_number ", field, String.valueOf(rand))); //always the same id to override assertU(commit()); assertQ(req("q", field + ":" + rand, "fl", "id, " + field), - "//*[@numFound='1']"); + MATCH_ONE); } clearIndex(); assertU(commit()); @@ -2008,8 +2266,20 @@ public class TestPointFields extends SolrTestCaseJ4 { "//lst[@name='facet_counts']/lst[@name='facet_ranges']/lst[@name='" + nonDocValuesField + "']/lst[@name='counts']/int[@name='-10'][.='0']"); } - - private void doTestDatePointFieldExactQuery(String field, String baseDate) throws Exception { + + private void doTestDatePointFieldExactQuery(final String field, final String baseDate) throws Exception { + doTestDatePointFieldExactQuery(field, baseDate, true); + } + + /** + * @param field the field to use for indexing and searching against + * @param baseDate basic value to use for indexing and searching + * @param searchable set to true if searches against "field" should succeed, false if field is only stored and searches should always get numFound=0 + */ + private void doTestDatePointFieldExactQuery(final String field, final String baseDate, final boolean searchable) throws Exception { + final String MATCH_ONE = "//*[@numFound='" + (searchable ? "1" : "0") + "']"; + final String MATCH_TWO = "//*[@numFound='" + (searchable ? "2" : "0") + "']"; + for (int i=0; i < 10; i++) { assertU(adoc("id", String.valueOf(i), field, String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, i+1))); } @@ -2017,20 +2287,21 @@ public class TestPointFields extends SolrTestCaseJ4 { for (int i = 0; i < 10; i++) { String date = String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, i+1); assertQ(req("q", field + ":\""+date+"\"", "fl", "id, " + field), - "//*[@numFound='1']"); + MATCH_ONE); } for (int i = 0; i < 10; i++) { String date1 = String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, i+1); String date2 = String.format(Locale.ROOT, "%s+%dMINUTES", baseDate, ((i+1)%10 + 1)); assertQ(req("q", field + ":\"" + date1 + "\"" - + " OR " + field + ":\"" + date2 + "\""), "//*[@numFound='2']"); + + " OR " + field + ":\"" + date2 + "\""), + MATCH_TWO); } clearIndex(); assertU(commit()); } - + private void doTestDatePointFieldRangeQuery(String fieldName) throws Exception { String baseDate = "1995-12-31T10:59:59Z"; for (int i = 9; i >= 0; i--) { @@ -2362,6 +2633,204 @@ public class TestPointFields extends SolrTestCaseJ4 { "count(//result/doc[1]/arr[@name='" + field + "']/" + type + ")=0"); } + + private void doTestInternals(String field, String[] values) throws IOException { + assertTrue(h.getCore().getLatestSchema().getField(field).getType() instanceof PointField); + for (int i=0; i < 10; i++) { + assertU(adoc("id", String.valueOf(i), field, values[i])); + } + assertU(commit()); + IndexReader ir; + RefCounted ref = null; + SchemaField sf = h.getCore().getLatestSchema().getField(field); + boolean ignoredField = !(sf.indexed() || sf.stored() || sf.hasDocValues()); + try { + ref = h.getCore().getSearcher(); + SolrIndexSearcher searcher = ref.get(); + ir = searcher.getIndexReader(); + if (sf.indexed()) { + assertEquals("Field " + field + " should have point values", 10, PointValues.size(ir, field)); + } else { + assertEquals("Field " + field + " should have no point values", 0, PointValues.size(ir, field)); + } + if (ignoredField) { + assertTrue("Field " + field + " should not have docValues", + DocValues.getSortedNumeric(searcher.getSlowAtomicReader(), field).nextDoc() == DocIdSetIterator.NO_MORE_DOCS); + assertTrue("Field " + field + " should not have docValues", + DocValues.getNumeric(searcher.getSlowAtomicReader(), field).nextDoc() == DocIdSetIterator.NO_MORE_DOCS); + assertTrue("Field " + field + " should not have docValues", + DocValues.getSorted(searcher.getSlowAtomicReader(), field).nextDoc() == DocIdSetIterator.NO_MORE_DOCS); + assertTrue("Field " + field + " should not have docValues", + DocValues.getBinary(searcher.getSlowAtomicReader(), field).nextDoc() == DocIdSetIterator.NO_MORE_DOCS); + } else { + if (sf.hasDocValues()) { + if (sf.multiValued()) { + assertFalse("Field " + field + " should have docValues", + DocValues.getSortedNumeric(searcher.getSlowAtomicReader(), field).nextDoc() == DocIdSetIterator.NO_MORE_DOCS); + } else { + assertFalse("Field " + field + " should have docValues", + DocValues.getNumeric(searcher.getSlowAtomicReader(), field).nextDoc() == DocIdSetIterator.NO_MORE_DOCS); + } + } else { + expectThrows(IllegalStateException.class, ()->DocValues.getSortedNumeric(searcher.getSlowAtomicReader(), field)); + expectThrows(IllegalStateException.class, ()->DocValues.getNumeric(searcher.getSlowAtomicReader(), field)); + } + expectThrows(IllegalStateException.class, ()->DocValues.getSorted(searcher.getSlowAtomicReader(), field)); + expectThrows(IllegalStateException.class, ()->DocValues.getBinary(searcher.getSlowAtomicReader(), field)); + } + for (LeafReaderContext leave:ir.leaves()) { + LeafReader reader = leave.reader(); + for (int i = 0; i < reader.numDocs(); i++) { + Document doc = reader.document(i, Collections.singleton(field)); + if (sf.stored()) { + assertNotNull(doc.get(field)); + } else { + assertNull(doc.get(field)); + } + } + } + } finally { + ref.decref(); + } + } + + public void testNonReturnable() throws Exception { + + doTestNonReturnable("foo_p_i_ni_ns", "42"); + doTestNonReturnable("foo_p_i_ni_ns_mv", "42", "666"); + + doTestNonReturnable("foo_p_l_ni_ns", "3333333333"); + doTestNonReturnable("foo_p_l_ni_ns_mv", "3333333333", "-4444444444"); + + doTestNonReturnable("foo_p_f_ni_ns", "42.3"); + doTestNonReturnable("foo_p_f_ni_ns_mv", "42.3", "-66.6"); + + doTestNonReturnable("foo_p_d_ni_ns", "42.3"); + doTestNonReturnable("foo_p_d_ni_ns_mv", "42.3", "-66.6"); + + doTestNonReturnable("foo_p_dt_ni_ns", "1995-12-31T23:59:59Z"); + doTestNonReturnable("foo_p_dt_ni_ns_mv", "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z+3DAYS"); + + } + + @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-10437") + public void testNonReturnableDocValues() throws Exception { + // TODO: once SOLR-10437 is resolved, this test method can be folded into testNonReturnable() + + // these fields are stored=false, docValues=true, useDocValuesAsStored=false and yet they are + // still returned and failing this test. + + doTestNonReturnable("foo_p_i_ni_dv_ns", "42"); + doTestNonReturnable("foo_p_l_ni_dv_ns", "3333333333"); + doTestNonReturnable("foo_p_f_ni_dv_ns", "42.3"); + doTestNonReturnable("foo_p_d_ni_dv_ns", "42.3"); + doTestNonReturnable("foo_p_dt_ni_dv_ns", "1995-12-31T23:59:59Z"); + } + + public void doTestNonReturnable(final String fieldName, final String... values) throws Exception { + for (int i=0; i < 10; i++) { + SolrInputDocument doc = sdoc("id", String.valueOf(i)); + for (String value : values) { + doc.addField(fieldName, value); + } + assertU(adoc(doc)); + } + assertU(commit()); + assertQ(req("q", "*:*", "rows", "100", "fl", "id," + fieldName), + "//*[@numFound='10']", + "count(//doc)=10", // exactly 10 docs in response + "count(//doc/*)=10", // exactly 10 fields across all docs + "count(//doc/*[@name!='id'])=0"); // no field in any doc other then 'id' + clearIndex(); + assertU(commit()); + } + + public void testWhiteboxCreateFields() throws Exception { + // TODO: we should have a "coverage" assert that we're looping over all the dynamic (point) fields in the schema + + String[] typeNames = new String[]{"i", "l", "f", "d", "dt"}; + String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; + Class[] expectedClasses = new Class[]{IntPoint.class, LongPoint.class, FloatPoint.class, DoublePoint.class, LongPoint.class}; + + Date dateToTest = new Date(); + Object[][] values = new Object[][] { + {42, "42"}, + {42, "42"}, + {42.123, "42.123"}, + {12345.6789, "12345.6789"}, + {dateToTest, new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss'Z'", Locale.ROOT).format(dateToTest), "NOW"} // "NOW" won't be equal to the other dates + }; + + Set typesTested = new HashSet<>(); + for (int i = 0; i < typeNames.length; i++) { + for (String suffix:suffixes) { + doWhiteboxCreateFields("whitebox_p_" + typeNames[i] + suffix, expectedClasses[i], values[i]); + typesTested.add("*_p_" + typeNames[i] + suffix); + } + } + Set typesToTest = new HashSet<>(); + for (DynamicField dynField:h.getCore().getLatestSchema().getDynamicFields()) { + if (dynField.getPrototype().getType() instanceof PointField) { + typesToTest.add(dynField.getRegex()); + } + } + assertEquals("Missing types in the test", typesTested, typesToTest); + } + + /** + * Calls {@link #callAndCheckCreateFields} on each of the specified values. + * This is a convinience method for testing the same fieldname with multiple inputs. + * + * @see #callAndCheckCreateFields + */ + private void doWhiteboxCreateFields(final String fieldName, final Class pointType, final Object... values) throws Exception { + + for (Object value : values) { + // ideally we should require that all input values be diff forms of the same logical value + // (ie '"42"' vs 'new Integer(42)') and assert that each produces an equivilent list of IndexableField objects + // but that doesn't seem to work -- appears not all IndexableField classes override Object.equals? + final List result = callAndCheckCreateFields(fieldName, pointType, value); + assertNotNull(value + " => null", result); + } + } + + + /** + * Calls {@link SchemaField#createFields} on the specified value for the specified field name, and asserts + * that the results match the SchemaField propeties, with an additional check that the pointType + * is included if and only if the SchemaField is "indexed" + */ + private List callAndCheckCreateFields(final String fieldName, final Class pointType, final Object value) throws Exception { + final SchemaField sf = h.getCore().getLatestSchema().getField(fieldName); + final List results = sf.createFields(value); + final Set resultSet = new LinkedHashSet<>(results); + assertEquals("duplicates found in results? " + results.toString(), + results.size(), resultSet.size()); + + final Set> resultClasses = new HashSet<>(); + for (IndexableField f : results) { + resultClasses.add(f.getClass()); + + if (!sf.hasDocValues() ) { + assertFalse(f.toString(), + (f instanceof NumericDocValuesField) || + (f instanceof SortedNumericDocValuesField)); + } + } + assertEquals(fieldName + " stored? Result Fields: " + Arrays.toString(results.toArray()), + sf.stored(), resultClasses.contains(StoredField.class)); + assertEquals(fieldName + " indexed? Result Fields: " + Arrays.toString(results.toArray()), + sf.indexed(), resultClasses.contains(pointType)); + if (sf.multiValued()) { + assertEquals(fieldName + " docvalues? Result Fields: " + Arrays.toString(results.toArray()), + sf.hasDocValues(), resultClasses.contains(SortedNumericDocValuesField.class)); + } else { + assertEquals(fieldName + " docvalues? Result Fields: " + Arrays.toString(results.toArray()), + sf.hasDocValues(), resultClasses.contains(NumericDocValuesField.class)); + } + + return results; + } } From 865f9b2b0155ed7b9326c1a79000fde935ee00d2 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Thu, 6 Apr 2017 21:17:02 -0400 Subject: [PATCH 131/563] SOLR-10444: SQL interface does not use client cache --- .../src/java/org/apache/solr/handler/StreamHandler.java | 4 ++++ .../src/java/org/apache/solr/handler/sql/SolrTable.java | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java index 8f123ec9481..6f373f65b0a 100644 --- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java +++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java @@ -115,6 +115,10 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware, return PermissionNameProvider.Name.READ_PERM; } + public static SolrClientCache getClientCache() { + return clientCache; + } + public void inform(SolrCore core) { /* The stream factory will always contain the zkUrl for the given collection 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 b7f552b6adf..aa873cc61be 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 @@ -28,6 +28,7 @@ import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.TranslatableTable; import org.apache.calcite.schema.impl.AbstractTableQueryable; import org.apache.calcite.util.Pair; +import org.apache.solr.client.solrj.io.SolrClientCache; import org.apache.solr.client.solrj.io.comp.ComparatorOrder; import org.apache.solr.client.solrj.io.comp.FieldComparator; import org.apache.solr.client.solrj.io.comp.MultipleFieldComparator; @@ -51,6 +52,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory; import org.apache.solr.client.solrj.io.stream.metrics.*; import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.handler.StreamHandler; import java.io.IOException; import java.util.*; @@ -158,6 +160,10 @@ class SolrTable extends AbstractQueryableTable implements TranslatableTable { throw new RuntimeException(e); } + StreamContext streamContext = new StreamContext(); + streamContext.setSolrClientCache(StreamHandler.getClientCache()); + tupleStream.setStreamContext(streamContext); + final TupleStream finalStream = tupleStream; return new AbstractEnumerable() { From a0d22297ce2498797cfe45862c9fe04ce872661d Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Thu, 6 Apr 2017 21:33:47 -0400 Subject: [PATCH 132/563] SOLR-10444: Fix precommit --- solr/core/src/java/org/apache/solr/handler/sql/SolrTable.java | 1 - 1 file changed, 1 deletion(-) 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 aa873cc61be..f86368096a6 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 @@ -28,7 +28,6 @@ import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.schema.TranslatableTable; import org.apache.calcite.schema.impl.AbstractTableQueryable; import org.apache.calcite.util.Pair; -import org.apache.solr.client.solrj.io.SolrClientCache; import org.apache.solr.client.solrj.io.comp.ComparatorOrder; import org.apache.solr.client.solrj.io.comp.FieldComparator; import org.apache.solr.client.solrj.io.comp.MultipleFieldComparator; From c05ab96dc4454c00779aad2cd6989812c9b13a2e Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Thu, 6 Apr 2017 22:06:07 -0400 Subject: [PATCH 133/563] SOLR-10341, SOLR-10444: Update CHANGES.txt --- solr/CHANGES.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 0bf2c54a28f..95edf899cde 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -227,6 +227,10 @@ Bug Fixes * SOLR-10425: Fix indexed="false" on numeric PointFields (Tomás Fernández Löbbe, hossman) +* SOLR-10341: SQL AVG function mis-interprets field type. (Joel Bernstein) + +* SOLR-10444: SQL interface does not use client cache. (Joel Bernstein) + ================== 6.5.0 ================== Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. From cbad533d7a44a5fd41f85756c791f3d7439861a2 Mon Sep 17 00:00:00 2001 From: Tommaso Teofili Date: Thu, 6 Apr 2017 19:05:52 +0200 Subject: [PATCH 134/563] LUCENE-6853 - renamed threshold to bias, initialize to avg tf --- .../BooleanPerceptronClassifier.java | 26 +++++++++---------- .../BooleanPerceptronClassifierTest.java | 14 ++++++---- 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/lucene/classification/src/java/org/apache/lucene/classification/BooleanPerceptronClassifier.java b/lucene/classification/src/java/org/apache/lucene/classification/BooleanPerceptronClassifier.java index 781a14ff6ee..928c0366770 100644 --- a/lucene/classification/src/java/org/apache/lucene/classification/BooleanPerceptronClassifier.java +++ b/lucene/classification/src/java/org/apache/lucene/classification/BooleanPerceptronClassifier.java @@ -58,7 +58,7 @@ import org.apache.lucene.util.fst.Util; */ public class BooleanPerceptronClassifier implements Classifier { - private final Double threshold; + private final Double bias; private final Terms textTerms; private final Analyzer analyzer; private final String textFieldName; @@ -72,14 +72,14 @@ public class BooleanPerceptronClassifier implements Classifier { * @param query a {@link Query} to eventually filter the docs used for training the classifier, or {@code null} * if all the indexed docs should be used * @param batchSize the size of the batch of docs to use for updating the perceptron weights - * @param threshold the threshold used for class separation + * @param bias the bias used for class separation * @param classFieldName the name of the field used as the output for the classifier * @param textFieldName the name of the field used as input for the classifier * @throws IOException if the building of the underlying {@link FST} fails and / or {@link TermsEnum} for the text field * cannot be found */ public BooleanPerceptronClassifier(IndexReader indexReader, Analyzer analyzer, Query query, Integer batchSize, - Double threshold, String classFieldName, String textFieldName) throws IOException { + Double bias, String classFieldName, String textFieldName) throws IOException { this.textTerms = MultiFields.getTerms(indexReader, textFieldName); if (textTerms == null) { @@ -89,18 +89,18 @@ public class BooleanPerceptronClassifier implements Classifier { this.analyzer = analyzer; this.textFieldName = textFieldName; - if (threshold == null || threshold == 0d) { - // automatic assign a threshold - long sumDocFreq = indexReader.getSumDocFreq(textFieldName); - if (sumDocFreq != -1) { - this.threshold = (double) sumDocFreq / 2d; + if (bias == null || bias == 0d) { + // automatic assign the bias to be the average total term freq + double t = (double) indexReader.getSumTotalTermFreq(textFieldName) / (double) indexReader.getDocCount(textFieldName); + if (t != -1) { + this.bias = t; } else { throw new IOException( - "threshold cannot be assigned since term vectors for field " + "bias cannot be assigned since term vectors for field " + textFieldName + " do not exist"); } } else { - this.threshold = threshold; + this.bias = bias; } // TODO : remove this map as soon as we have a writable FST @@ -173,7 +173,7 @@ public class BooleanPerceptronClassifier implements Classifier { // update weights Long previousValue = Util.get(fst, term); String termString = term.utf8ToString(); - weights.put(termString, previousValue == null ? 0 : previousValue + modifier * termFreqLocal); + weights.put(termString, previousValue == null ? 0 : Math.max(0, previousValue + modifier * termFreqLocal)); } } if (updateFST) { @@ -216,8 +216,8 @@ public class BooleanPerceptronClassifier implements Classifier { tokenStream.end(); } - double score = 1 - Math.exp(-1 * Math.abs(threshold - output.doubleValue()) / threshold); - return new ClassificationResult<>(output >= threshold, score); + double score = 1 - Math.exp(-1 * Math.abs(bias - output.doubleValue()) / bias); + return new ClassificationResult<>(output >= bias, score); } /** diff --git a/lucene/classification/src/test/org/apache/lucene/classification/BooleanPerceptronClassifierTest.java b/lucene/classification/src/test/org/apache/lucene/classification/BooleanPerceptronClassifierTest.java index 6ea92c03b08..ec059f75ac4 100644 --- a/lucene/classification/src/test/org/apache/lucene/classification/BooleanPerceptronClassifierTest.java +++ b/lucene/classification/src/test/org/apache/lucene/classification/BooleanPerceptronClassifierTest.java @@ -34,7 +34,9 @@ public class BooleanPerceptronClassifierTest extends ClassificationTestBase avgClassificationTime); // accuracy check disabled until LUCENE-6853 is fixed -// double accuracy = confusionMatrix.getAccuracy(); -// assertTrue(accuracy > 0d); + double accuracy = confusionMatrix.getAccuracy(); + assertTrue(accuracy > 0d); } finally { leafReader.close(); } From 276ccff751dc0f5cd0599dd5a6682553a0b58d7b Mon Sep 17 00:00:00 2001 From: Tommaso Teofili Date: Thu, 6 Apr 2017 19:13:50 +0200 Subject: [PATCH 135/563] LUCENE-6853 - re-enabled test classification measures for bpc --- .../BooleanPerceptronClassifierTest.java | 39 +++++++++++++++++-- 1 file changed, 36 insertions(+), 3 deletions(-) diff --git a/lucene/classification/src/test/org/apache/lucene/classification/BooleanPerceptronClassifierTest.java b/lucene/classification/src/test/org/apache/lucene/classification/BooleanPerceptronClassifierTest.java index ec059f75ac4..5ecf9c67465 100644 --- a/lucene/classification/src/test/org/apache/lucene/classification/BooleanPerceptronClassifierTest.java +++ b/lucene/classification/src/test/org/apache/lucene/classification/BooleanPerceptronClassifierTest.java @@ -19,8 +19,12 @@ package org.apache.lucene.classification; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.classification.utils.ConfusionMatrixGenerator; import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.MultiFields; import org.apache.lucene.index.Term; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.util.BytesRef; import org.junit.Test; /** @@ -90,16 +94,45 @@ public class BooleanPerceptronClassifierTest extends ClassificationTestBase avgClassificationTime); - // accuracy check disabled until LUCENE-6853 is fixed + + double f1 = confusionMatrix.getF1Measure(); + assertTrue(f1 >= 0d); + assertTrue(f1 <= 1d); + double accuracy = confusionMatrix.getAccuracy(); - assertTrue(accuracy > 0d); + assertTrue(accuracy >= 0d); + assertTrue(accuracy <= 1d); + + double recall = confusionMatrix.getRecall(); + assertTrue(recall >= 0d); + assertTrue(recall <= 1d); + + double precision = confusionMatrix.getPrecision(); + assertTrue(precision >= 0d); + assertTrue(precision <= 1d); + + Terms terms = MultiFields.getTerms(leafReader, booleanFieldName); + TermsEnum iterator = terms.iterator(); + BytesRef term; + while ((term = iterator.next()) != null) { + String s = term.utf8ToString(); + recall = confusionMatrix.getRecall(s); + assertTrue(recall >= 0d); + assertTrue(recall <= 1d); + precision = confusionMatrix.getPrecision(s); + assertTrue(precision >= 0d); + assertTrue(precision <= 1d); + double f1Measure = confusionMatrix.getF1Measure(s); + assertTrue(f1Measure >= 0d); + assertTrue(f1Measure <= 1d); + } } finally { leafReader.close(); } From f37fad206b0dbf32209bff3761179458a1ddf7bf Mon Sep 17 00:00:00 2001 From: Tommaso Teofili Date: Fri, 7 Apr 2017 10:58:49 +0200 Subject: [PATCH 136/563] LUCENE-5548 - improved testing for SNBC --- .../SimpleNaiveBayesClassifierTest.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/lucene/classification/src/test/org/apache/lucene/classification/SimpleNaiveBayesClassifierTest.java b/lucene/classification/src/test/org/apache/lucene/classification/SimpleNaiveBayesClassifierTest.java index 2b4873d93e0..0e05d4fe8b9 100644 --- a/lucene/classification/src/test/org/apache/lucene/classification/SimpleNaiveBayesClassifierTest.java +++ b/lucene/classification/src/test/org/apache/lucene/classification/SimpleNaiveBayesClassifierTest.java @@ -59,8 +59,10 @@ public class SimpleNaiveBayesClassifierTest extends ClassificationTestBase avgClassificationTime); + + double f1 = confusionMatrix.getF1Measure(); + assertTrue(f1 >= 0d); + assertTrue(f1 <= 1d); + double accuracy = confusionMatrix.getAccuracy(); assertTrue(accuracy >= 0d); assertTrue(accuracy <= 1d); From 94b83f1f7d85ffdf6b52709be9e5788020ee715f Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Fri, 7 Apr 2017 10:37:30 +0100 Subject: [PATCH 137/563] SOLR-10441: remove no longer used HttpShardHandlerFactory.USE_RETRIES --- .../solr/handler/component/HttpShardHandlerFactory.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java index 1c016c73326..4262c20658c 100644 --- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java +++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java @@ -124,10 +124,6 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org. // Configure if the threadpool favours fairness over throughput static final String INIT_FAIRNESS_POLICY = "fairnessPolicy"; - - // Turn on retries for certain IOExceptions, many of which can happen - // due to connection pooling limitations / races - static final String USE_RETRIES = "useRetries"; /** * Get {@link ShardHandler} that uses the default http client. From 206021578e7940c0c94aaadf659d72d73e96a075 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Fri, 7 Apr 2017 11:04:41 +0100 Subject: [PATCH 138/563] SOLR-10440: LBHttpSolrClient.doRequest is now always wrapped in a Mapped Diagnostic Context (MDC). --- solr/CHANGES.txt | 3 +++ .../solr/client/solrj/impl/LBHttpSolrClient.java | 11 ++++++++--- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 95edf899cde..9e70c2435c1 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -201,6 +201,9 @@ Other Changes * SOLR-10394: Rename getSortWithinGroup to getWithinGroupSort in search.grouping.Command class. (Judith Silverman, Christine Poerschke) +* SOLR-10440: LBHttpSolrClient.doRequest is now always wrapped in a Mapped Diagnostic Context (MDC). + (Christine Poerschke) + ================== 6.5.1 ================== Bug Fixes diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java index 3cb59cd061a..b0f1a505d0d 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/LBHttpSolrClient.java @@ -404,9 +404,14 @@ public class LBHttpSolrClient extends SolrClient { break; } - ex = doRequest(wrapper.client, req, rsp, isNonRetryable, true, wrapper.getKey()); - if (ex == null) { - return rsp; // SUCCESS + try { + MDC.put("LBHttpSolrClient.url", wrapper.client.getBaseURL()); + ex = doRequest(wrapper.client, req, rsp, isNonRetryable, true, wrapper.getKey()); + if (ex == null) { + return rsp; // SUCCESS + } + } finally { + MDC.remove("LBHttpSolrClient.url"); } } } From adabfdd9c2a50b4141f485655df0d048df21bd23 Mon Sep 17 00:00:00 2001 From: Christine Poerschke Date: Fri, 7 Apr 2017 11:11:23 +0100 Subject: [PATCH 139/563] =?UTF-8?q?SOLR-10264:=20Fixes=20multi-term=20syno?= =?UTF-8?q?nym=20parsing=20in=20ManagedSynonymFilterFactory.=20(J=C3=B6rg?= =?UTF-8?q?=20Rathlev,=20Steve=20Rowe,=20Christine=20Poerschke)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- solr/CHANGES.txt | 3 ++ .../analysis/ManagedSynonymFilterFactory.java | 7 ++-- .../solr/collection1/conf/schema-rest.xml | 2 +- .../TestManagedSynonymFilterFactory.java | 40 +++++++++++++++++++ 4 files changed, 48 insertions(+), 4 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 9e70c2435c1..62e91899b77 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -163,6 +163,9 @@ Bug Fixes * SOLR-10323: fix to SpellingQueryConverter to properly strip out colons in field-specific queries. (Amrit Sarkar via James Dyer) +* SOLR-10264: Fixes multi-term synonym parsing in ManagedSynonymFilterFactory. + (Jörg Rathlev, Steve Rowe, Christine Poerschke) + Other Changes ---------------------- diff --git a/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java index 3b5ce2e3fc4..ffbbb36dbd7 100644 --- a/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java +++ b/solr/core/src/java/org/apache/solr/rest/schema/analysis/ManagedSynonymFilterFactory.java @@ -33,6 +33,7 @@ import org.apache.lucene.analysis.synonym.SynonymFilterFactory; import org.apache.lucene.analysis.synonym.SynonymMap; import org.apache.lucene.analysis.util.ResourceLoader; import org.apache.lucene.util.CharsRef; +import org.apache.lucene.util.CharsRefBuilder; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.util.NamedList; @@ -358,9 +359,9 @@ public class ManagedSynonymFilterFactory extends BaseManagedTokenFilterFactory { for (String term : cpsm.mappings.keySet()) { for (String mapping : cpsm.mappings.get(term)) { // apply the case setting to match the behavior of the SynonymMap builder - String casedTerm = synonymManager.applyCaseSetting(ignoreCase, term); - String casedMapping = synonymManager.applyCaseSetting(ignoreCase, mapping); - add(new CharsRef(casedTerm), new CharsRef(casedMapping), false); + CharsRef casedTerm = analyze(synonymManager.applyCaseSetting(ignoreCase, term), new CharsRefBuilder()); + CharsRef casedMapping = analyze(synonymManager.applyCaseSetting(ignoreCase, mapping), new CharsRefBuilder()); + add(casedTerm, casedMapping, false); } } } diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml index aa4c21d2301..85c822a319a 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-rest.xml @@ -494,7 +494,7 @@ - + diff --git a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java index 26fcde1e44c..5cacc481e6d 100644 --- a/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java +++ b/solr/core/src/test/org/apache/solr/rest/schema/analysis/TestManagedSynonymFilterFactory.java @@ -16,6 +16,7 @@ */ package org.apache.solr.rest.schema.analysis; import java.io.File; +import java.net.URLEncoder; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -154,13 +155,30 @@ public class TestManagedSynonymFilterFactory extends RestTestBase { "count(/response/lst[@name='field']) = 1", "/response/lst[@name='responseHeader']/int[@name='status'] = '0'"); + // multi-term synonym logic - SOLR-10264 + final String multiTermOrigin; + final String multiTermSynonym; + if (random().nextBoolean()) { + multiTermOrigin = "hansestadt hamburg"; + multiTermSynonym = "hh"; + } else { + multiTermOrigin = "hh"; + multiTermSynonym = "hansestadt hamburg"; + } + // multi-term logic similar to the angry/mad logic (angry ~ origin, mad ~ synonym) + assertU(adoc(newFieldName, "I am a happy test today but yesterday I was angry", "id", "5150")); + assertU(adoc(newFieldName, multiTermOrigin+" is in North Germany.", "id", "040")); assertU(commit()); assertQ("/select?q=" + newFieldName + ":angry", "/response/lst[@name='responseHeader']/int[@name='status'] = '0'", "/response/result[@name='response'][@numFound='1']", "/response/result[@name='response']/doc/str[@name='id'][.='5150']"); + assertQ("/select?q=" + newFieldName + ":"+URLEncoder.encode(multiTermOrigin, "UTF-8"), + "/response/lst[@name='responseHeader']/int[@name='status'] = '0'", + "/response/result[@name='response'][@numFound='1']", + "/response/result[@name='response']/doc/str[@name='id'][.='040']"); // add a mapping that will expand a query for "mad" to match docs with "angry" syns = new HashMap<>(); @@ -172,12 +190,28 @@ public class TestManagedSynonymFilterFactory extends RestTestBase { assertJQ(endpoint, "/synonymMappings/managedMap/mad==['angry']"); + // add a mapping that will expand a query for "multi-term synonym" to match docs with "acronym" + syns = new HashMap<>(); + syns.put(multiTermSynonym, Arrays.asList(multiTermOrigin)); + assertJPut(endpoint, + JSONUtil.toJSON(syns), + "/responseHeader/status==0"); + + assertJQ(endpoint+"/"+URLEncoder.encode(multiTermSynonym, "UTF-8"), + "/"+multiTermSynonym+"==['"+multiTermOrigin+"']"); + // should not match as the synonym mapping between mad and angry does not // get applied until core reload assertQ("/select?q=" + newFieldName + ":mad", "/response/lst[@name='responseHeader']/int[@name='status'] = '0'", "/response/result[@name='response'][@numFound='0']"); + // should not match as the synonym mapping between "origin" and "synonym" + // was not added before the document was indexed + assertQ("/select?q=" + newFieldName + ":("+URLEncoder.encode(multiTermSynonym, "UTF-8") + ")&sow=false", + "/response/lst[@name='responseHeader']/int[@name='status'] = '0'", + "/response/result[@name='response'][@numFound='0']"); + restTestHarness.reload(); // now query for mad and we should see our test doc @@ -186,6 +220,12 @@ public class TestManagedSynonymFilterFactory extends RestTestBase { "/response/result[@name='response'][@numFound='1']", "/response/result[@name='response']/doc/str[@name='id'][.='5150']"); + // now query for "synonym" and we should see our test doc with "origin" + assertQ("/select?q=" + newFieldName + ":("+URLEncoder.encode(multiTermSynonym, "UTF-8") + ")&sow=false", + "/response/lst[@name='responseHeader']/int[@name='status'] = '0'", + "/response/result[@name='response'][@numFound='1']", + "/response/result[@name='response']/doc/str[@name='id'][.='040']"); + // test for SOLR-6015 syns = new HashMap<>(); syns.put("mb", Arrays.asList("megabyte")); From b2a631079ad2722f28cd6ee77779903e354d7a56 Mon Sep 17 00:00:00 2001 From: Joel Bernstein Date: Fri, 7 Apr 2017 12:36:19 -0400 Subject: [PATCH 140/563] Add version 6.5.1 --- lucene/core/src/java/org/apache/lucene/util/Version.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/lucene/core/src/java/org/apache/lucene/util/Version.java b/lucene/core/src/java/org/apache/lucene/util/Version.java index da6d653bce7..f6e6adcc5ed 100644 --- a/lucene/core/src/java/org/apache/lucene/util/Version.java +++ b/lucene/core/src/java/org/apache/lucene/util/Version.java @@ -101,6 +101,13 @@ public final class Version { @Deprecated public static final Version LUCENE_6_5_0 = new Version(6, 5, 0); + /** + * Match settings and bugs in Lucene's 6.5.1 release. + * @deprecated Use latest + */ + @Deprecated + public static final Version LUCENE_6_5_1 = new Version(6, 5, 1); + /** * Match settings and bugs in Lucene's 6.6.0 release. * @deprecated Use latest From ce75eb62e9fd3256bcab23e7f1d901fec7c65115 Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Fri, 7 Apr 2017 11:36:22 -0700 Subject: [PATCH 141/563] SOLR-10437: Improve test coverage of useDocValuesAsStored=false --- .../solr/collection1/conf/schema-point.xml | 5 ++ .../apache/solr/schema/TestPointFields.java | 71 ++++++++++--------- .../solr/schema/TestUseDocValuesAsStored.java | 28 ++++++++ 3 files changed, 71 insertions(+), 33 deletions(-) diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml index 1a936a04c64..ae6a11ecfef 100644 --- a/solr/core/src/test-files/solr/collection1/conf/schema-point.xml +++ b/solr/core/src/test-files/solr/collection1/conf/schema-point.xml @@ -54,6 +54,7 @@ + @@ -66,6 +67,7 @@ + @@ -78,6 +80,7 @@ + @@ -90,6 +93,7 @@ + @@ -102,6 +106,7 @@ + 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 de458445fe4..18ac3b257f5 100644 --- a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java +++ b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java @@ -17,6 +17,7 @@ package org.apache.solr.schema; import java.io.IOException; +import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Collections; import java.util.Date; @@ -57,7 +58,6 @@ import org.junit.Test; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.ibm.icu.text.SimpleDateFormat; /** * Tests for PointField functionality @@ -885,7 +885,7 @@ public class TestPointFields extends SolrTestCaseJ4 { public void testInternals() throws IOException { String[] types = new String[]{"i", "l", "f", "d"}; - String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; + String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_dv_ns_mv", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; Set typesTested = new HashSet<>(); for (String type:types) { for (String suffix:suffixes) { @@ -2696,38 +2696,36 @@ public class TestPointFields extends SolrTestCaseJ4 { public void testNonReturnable() throws Exception { - doTestNonReturnable("foo_p_i_ni_ns", "42"); - doTestNonReturnable("foo_p_i_ni_ns_mv", "42", "666"); + doTestReturnNonStored("foo_p_i_ni_ns", false, "42"); + doTestReturnNonStored("foo_p_i_ni_dv_ns", true, "42"); + doTestReturnNonStored("foo_p_i_ni_ns_mv", false, "42", "666"); + doTestReturnNonStored("foo_p_i_ni_dv_ns_mv", true, "42", "666"); - doTestNonReturnable("foo_p_l_ni_ns", "3333333333"); - doTestNonReturnable("foo_p_l_ni_ns_mv", "3333333333", "-4444444444"); + doTestReturnNonStored("foo_p_l_ni_ns", false, "3333333333"); + doTestReturnNonStored("foo_p_l_ni_dv_ns", true, "3333333333"); + doTestReturnNonStored("foo_p_l_ni_ns_mv", false, "3333333333", "-4444444444"); + doTestReturnNonStored("foo_p_l_ni_dv_ns_mv", true, "3333333333", "-4444444444"); - doTestNonReturnable("foo_p_f_ni_ns", "42.3"); - doTestNonReturnable("foo_p_f_ni_ns_mv", "42.3", "-66.6"); + doTestReturnNonStored("foo_p_f_ni_ns", false, "42.3"); + doTestReturnNonStored("foo_p_f_ni_dv_ns", true, "42.3"); + doTestReturnNonStored("foo_p_f_ni_ns_mv", false, "42.3", "-66.6"); + doTestReturnNonStored("foo_p_f_ni_dv_ns_mv", true, "42.3", "-66.6"); - doTestNonReturnable("foo_p_d_ni_ns", "42.3"); - doTestNonReturnable("foo_p_d_ni_ns_mv", "42.3", "-66.6"); - - doTestNonReturnable("foo_p_dt_ni_ns", "1995-12-31T23:59:59Z"); - doTestNonReturnable("foo_p_dt_ni_ns_mv", "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z+3DAYS"); + doTestReturnNonStored("foo_p_d_ni_ns", false, "42.3"); + doTestReturnNonStored("foo_p_d_ni_dv_ns", true, "42.3"); + doTestReturnNonStored("foo_p_d_ni_ns_mv", false, "42.3", "-66.6"); + doTestReturnNonStored("foo_p_d_ni_dv_ns_mv", true, "42.3", "-66.6"); + doTestReturnNonStored("foo_p_dt_ni_ns", false, "1995-12-31T23:59:59Z"); + doTestReturnNonStored("foo_p_dt_ni_dv_ns", true, "1995-12-31T23:59:59Z"); + doTestReturnNonStored("foo_p_dt_ni_ns_mv", false, "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z+3DAYS"); + doTestReturnNonStored("foo_p_dt_ni_dv_ns_mv", true, "1995-12-31T23:59:59Z", "2000-12-31T23:59:59Z+3DAYS"); } - @AwaitsFix(bugUrl="https://issues.apache.org/jira/browse/SOLR-10437") - public void testNonReturnableDocValues() throws Exception { - // TODO: once SOLR-10437 is resolved, this test method can be folded into testNonReturnable() - - // these fields are stored=false, docValues=true, useDocValuesAsStored=false and yet they are - // still returned and failing this test. - - doTestNonReturnable("foo_p_i_ni_dv_ns", "42"); - doTestNonReturnable("foo_p_l_ni_dv_ns", "3333333333"); - doTestNonReturnable("foo_p_f_ni_dv_ns", "42.3"); - doTestNonReturnable("foo_p_d_ni_dv_ns", "42.3"); - doTestNonReturnable("foo_p_dt_ni_dv_ns", "1995-12-31T23:59:59Z"); - } - - public void doTestNonReturnable(final String fieldName, final String... values) throws Exception { + public void doTestReturnNonStored(final String fieldName, boolean shouldReturnFieldIfRequested, final String... values) throws Exception { + final String RETURN_FIELD = "count(//doc/*[@name='" + fieldName + "'])=10"; + final String DONT_RETURN_FIELD = "count(//doc/*[@name='" + fieldName + "'])=0"; + assertFalse(h.getCore().getLatestSchema().getField(fieldName).stored()); for (int i=0; i < 10; i++) { SolrInputDocument doc = sdoc("id", String.valueOf(i)); for (String value : values) { @@ -2739,17 +2737,24 @@ public class TestPointFields extends SolrTestCaseJ4 { assertQ(req("q", "*:*", "rows", "100", "fl", "id," + fieldName), "//*[@numFound='10']", "count(//doc)=10", // exactly 10 docs in response - "count(//doc/*)=10", // exactly 10 fields across all docs - "count(//doc/*[@name!='id'])=0"); // no field in any doc other then 'id' + (shouldReturnFieldIfRequested?RETURN_FIELD:DONT_RETURN_FIELD)); // no field in any doc other then 'id' + + assertQ(req("q", "*:*", "rows", "100", "fl", "*"), + "//*[@numFound='10']", + "count(//doc)=10", // exactly 10 docs in response + DONT_RETURN_FIELD); // no field in any doc other then 'id' + + assertQ(req("q", "*:*", "rows", "100"), + "//*[@numFound='10']", + "count(//doc)=10", // exactly 10 docs in response + DONT_RETURN_FIELD); // no field in any doc other then 'id' clearIndex(); assertU(commit()); } public void testWhiteboxCreateFields() throws Exception { - // TODO: we should have a "coverage" assert that we're looping over all the dynamic (point) fields in the schema - String[] typeNames = new String[]{"i", "l", "f", "d", "dt"}; - String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; + String[] suffixes = new String[]{"", "_dv", "_mv", "_mv_dv", "_ni", "_ni_dv", "_ni_dv_ns", "_ni_dv_ns_mv", "_ni_mv", "_ni_mv_dv", "_ni_ns", "_ni_ns_mv", "_dv_ns", "_ni_ns_dv", "_dv_ns_mv"}; Class[] expectedClasses = new Class[]{IntPoint.class, LongPoint.class, FloatPoint.class, DoublePoint.class, LongPoint.class}; Date dateToTest = new Date(); diff --git a/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java b/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java index 48a3f2266bc..9cababc1044 100644 --- a/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java +++ b/solr/core/src/test/org/apache/solr/schema/TestUseDocValuesAsStored.java @@ -250,6 +250,34 @@ public class TestUseDocValuesAsStored extends AbstractBadConfigTestBase { + "{'id':'myid6','test_s_dvo':'hello'}" + "]"); } + + @Test + public void testUseDocValuesAsStoredFalse() throws Exception { + SchemaField sf = h.getCore().getLatestSchema().getField("nonstored_dv_str"); + assertNotNull(sf); + assertTrue(sf.hasDocValues()); + assertFalse(sf.useDocValuesAsStored()); + assertFalse(sf.stored()); + assertU(adoc("id", "myid", "nonstored_dv_str", "dont see me")); + assertU(commit()); + + assertJQ(req("q", "id:myid"), + "/response/docs==[" + + "{'id':'myid'}" + + "]"); + assertJQ(req("q", "id:myid", "fl", "*"), + "/response/docs==[" + + "{'id':'myid'}" + + "]"); + assertJQ(req("q", "id:myid", "fl", "id,nonstored_dv_*"), + "/response/docs==[" + + "{'id':'myid'}" + + "]"); + assertJQ(req("q", "id:myid", "fl", "id,nonstored_dv_str"), + "/response/docs==[" + + "{'id':'myid','nonstored_dv_str':'dont see me'}" + + "]"); + } public void testManagedSchema() throws Exception { IndexSchema oldSchema = h.getCore().getLatestSchema(); From d671273bc396faa4c3815e0a4614e88d3e0373e1 Mon Sep 17 00:00:00 2001 From: Tomas Fernandez Lobbe Date: Fri, 7 Apr 2017 14:11:25 -0700 Subject: [PATCH 142/563] SOLR-10443: Improvements to TestPointFields * Fixes testInternals, index needs to be cleaned after each field * Validate that SolrQueryParser generates a PointInSetQuery when possible --- .../apache/solr/schema/TestPointFields.java | 68 +++++++++++++------ 1 file changed, 49 insertions(+), 19 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 18ac3b257f5..e5dc99091c8 100644 --- a/solr/core/src/test/org/apache/solr/schema/TestPointFields.java +++ b/solr/core/src/test/org/apache/solr/schema/TestPointFields.java @@ -19,7 +19,6 @@ package org.apache.solr.schema; import java.io.IOException; import java.text.SimpleDateFormat; import java.util.Arrays; -import java.util.Collections; import java.util.Date; import java.util.HashSet; import java.util.LinkedHashSet; @@ -48,8 +47,10 @@ import org.apache.lucene.search.PointRangeQuery; import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.params.CommonParams; import org.apache.solr.schema.IndexSchema.DynamicField; import org.apache.solr.search.SolrIndexSearcher; +import org.apache.solr.search.SolrQueryParser; import org.apache.solr.util.DateMathParser; import org.apache.solr.util.RefCounted; import org.junit.After; @@ -218,9 +219,9 @@ public class TestPointFields extends SolrTestCaseJ4 { @Test public void testIntPointSetQuery() throws Exception { - doTestSetQueries("number_p_i", getRandomStringArrayWithInts(10, false), false); - doTestSetQueries("number_p_i_mv", getRandomStringArrayWithInts(10, false), true); - doTestSetQueries("number_p_i_ni_dv", getRandomStringArrayWithInts(10, false), false); + doTestSetQueries("number_p_i", getRandomStringArrayWithInts(20, false), false); + doTestSetQueries("number_p_i_mv", getRandomStringArrayWithInts(20, false), true); + doTestSetQueries("number_p_i_ni_dv", getRandomStringArrayWithInts(20, false), false); } // DoublePointField @@ -408,9 +409,9 @@ public class TestPointFields extends SolrTestCaseJ4 { @Test public void testDoublePointSetQuery() throws Exception { - doTestSetQueries("number_p_d", getRandomStringArrayWithDoubles(10, false), false); - doTestSetQueries("number_p_d_mv", getRandomStringArrayWithDoubles(10, false), true); - doTestSetQueries("number_p_d_ni_dv", getRandomStringArrayWithDoubles(10, false), false); + doTestSetQueries("number_p_d", getRandomStringArrayWithDoubles(20, false), false); + doTestSetQueries("number_p_d_mv", getRandomStringArrayWithDoubles(20, false), true); + doTestSetQueries("number_p_d_ni_dv", getRandomStringArrayWithDoubles(20, false), false); } // Float @@ -557,9 +558,9 @@ public class TestPointFields extends SolrTestCaseJ4 { @Test public void testFloatPointSetQuery() throws Exception { - doTestSetQueries("number_p_f", getRandomStringArrayWithFloats(10, false), false); - doTestSetQueries("number_p_f_mv", getRandomStringArrayWithFloats(10, false), true); - doTestSetQueries("number_p_f_ni_dv", getRandomStringArrayWithFloats(10, false), false); + doTestSetQueries("number_p_f", getRandomStringArrayWithFloats(20, false), false); + doTestSetQueries("number_p_f_mv", getRandomStringArrayWithFloats(20, false), true); + doTestSetQueries("number_p_f_ni_dv", getRandomStringArrayWithFloats(20, false), false); } @Test @@ -705,9 +706,9 @@ public class TestPointFields extends SolrTestCaseJ4 { @Test public void testLongPointSetQuery() throws Exception { - doTestSetQueries("number_p_l", getRandomStringArrayWithLongs(10, false), false); - doTestSetQueries("number_p_l_mv", getRandomStringArrayWithLongs(10, false), true); - doTestSetQueries("number_p_l_ni_dv", getRandomStringArrayWithLongs(10, false), false); + doTestSetQueries("number_p_l", getRandomStringArrayWithLongs(20, false), false); + doTestSetQueries("number_p_l_mv", getRandomStringArrayWithLongs(20, false), true); + doTestSetQueries("number_p_l_ni_dv", getRandomStringArrayWithLongs(20, false), false); } @Test @@ -850,9 +851,9 @@ public class TestPointFields extends SolrTestCaseJ4 { @Test public void testDatePointSetQuery() throws Exception { - doTestSetQueries("number_p_dt", getRandomStringArrayWithDates(10, false), false); - doTestSetQueries("number_p_dt_mv", getRandomStringArrayWithDates(10, false), true); - doTestSetQueries("number_p_dt_ni_dv", getRandomStringArrayWithDates(10, false), false); + doTestSetQueries("number_p_dt", getRandomStringArrayWithDates(20, false), false); + doTestSetQueries("number_p_dt_mv", getRandomStringArrayWithDates(20, false), true); + doTestSetQueries("number_p_dt_ni_dv", getRandomStringArrayWithDates(20, false), false); } @@ -2087,7 +2088,8 @@ public class TestPointFields extends SolrTestCaseJ4 { assertU(adoc("id", String.valueOf(i), fieldName, values[i])); } assertU(commit()); - assertTrue(h.getCore().getLatestSchema().getField(fieldName).getType() instanceof PointField); + SchemaField sf = h.getCore().getLatestSchema().getField(fieldName); + assertTrue(sf.getType() instanceof PointField); for (int i = 0; i < values.length; i++) { assertQ(req("q", "{!term f='" + fieldName + "'}" + values[i], "fl", "id," + fieldName), @@ -2099,6 +2101,27 @@ public class TestPointFields extends SolrTestCaseJ4 { "//*[@numFound='2']"); } + assertTrue(values.length > SolrQueryParser.TERMS_QUERY_THRESHOLD); + int numTerms = SolrQueryParser.TERMS_QUERY_THRESHOLD + 1; + StringBuilder builder = new StringBuilder(fieldName + ":("); + for (int i = 0; i < numTerms; i++) { + if (sf.getType().getNumberType() == NumberType.DATE) { + builder.append(String.valueOf(values[i]).replace(":", "\\:") + ' '); + } else { + builder.append(String.valueOf(values[i]).replace("-", "\\-") + ' '); + } + } + builder.append(')'); + if (sf.indexed()) { // SolrQueryParser should also be generating a PointInSetQuery if indexed + assertQ(req(CommonParams.DEBUG, CommonParams.QUERY, "q", "*:*", "fq", builder.toString(), "fl", "id," + fieldName), + "//*[@numFound='" + numTerms + "']", + "//*[@name='parsed_filter_queries']/str[.='(" + getSetQueryToString(fieldName, values, numTerms) + ")']"); + } else { + // Won't use PointInSetQuery if the fiels is not indexed, but should match the same docs + assertQ(req(CommonParams.DEBUG, CommonParams.QUERY, "q", "*:*", "fq", builder.toString(), "fl", "id," + fieldName), + "//*[@numFound='" + numTerms + "']"); + } + if (multiValued) { clearIndex(); assertU(commit()); @@ -2118,6 +2141,11 @@ public class TestPointFields extends SolrTestCaseJ4 { } } + private String getSetQueryToString(String fieldName, String[] values, int numTerms) { + SchemaField sf = h.getCore().getLatestSchema().getField(fieldName); + return sf.getType().getSetQuery(null, sf, Arrays.asList(Arrays.copyOf(values, numTerms))).toString(); + } + private void doTestDoublePointFieldMultiValuedRangeFacet(String docValuesField, String nonDocValuesField) throws Exception { for (int i = 0; i < 10; i++) { assertU(adoc("id", String.valueOf(i), docValuesField, String.valueOf(i), docValuesField, String.valueOf(i + 10), @@ -2681,9 +2709,9 @@ public class TestPointFields extends SolrTestCaseJ4 { for (LeafReaderContext leave:ir.leaves()) { LeafReader reader = leave.reader(); for (int i = 0; i < reader.numDocs(); i++) { - Document doc = reader.document(i, Collections.singleton(field)); + Document doc = reader.document(i); if (sf.stored()) { - assertNotNull(doc.get(field)); + assertNotNull("Field " + field + " not found. Doc: " + doc, doc.get(field)); } else { assertNull(doc.get(field)); } @@ -2692,6 +2720,8 @@ public class TestPointFields extends SolrTestCaseJ4 { } finally { ref.decref(); } + clearIndex(); + assertU(commit()); } public void testNonReturnable() throws Exception { From 550a32495257634859ac6eae3858cbce71be06f2 Mon Sep 17 00:00:00 2001 From: Noble Paul Date: Mon, 10 Apr 2017 06:43:27 +0930 Subject: [PATCH 143/563] SOLR-10429: UpdateRequest#getRoutes()should copy the response parser --- solr/CHANGES.txt | 2 ++ .../org/apache/solr/client/solrj/request/UpdateRequest.java | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 62e91899b77..ec25b90b4e5 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -207,6 +207,8 @@ Other Changes * SOLR-10440: LBHttpSolrClient.doRequest is now always wrapped in a Mapped Diagnostic Context (MDC). (Christine Poerschke) +* SOLR-10429: UpdateRequest#getRoutes()should copy the response parser (noble) + ================== 6.5.1 ================== Bug Fixes diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java index 142710a0425..0d9867cc86c 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/UpdateRequest.java @@ -269,7 +269,7 @@ public class UpdateRequest extends AbstractUpdateRequest { return null; } String leaderUrl = urls.get(0); - LBHttpSolrClient.Req request = (LBHttpSolrClient.Req) routes + LBHttpSolrClient.Req request = routes .get(leaderUrl); if (request == null) { UpdateRequest updateRequest = new UpdateRequest(); @@ -278,6 +278,7 @@ public class UpdateRequest extends AbstractUpdateRequest { updateRequest.setParams(params); updateRequest.setPath(getPath()); updateRequest.setBasicAuthCredentials(getBasicAuthUser(), getBasicAuthPassword()); + updateRequest.setResponseParser(getResponseParser()); request = new LBHttpSolrClient.Req(updateRequest, urls); routes.put(leaderUrl, request); } From caf329717ef643b2a5a8b7ef0bfb7bd5e096c849 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Tue, 28 Mar 2017 19:52:53 +0100 Subject: [PATCH 144/563] LUCENE-7701: Refactor grouping collectors --- lucene/CHANGES.txt | 3 + lucene/MIGRATE.txt | 13 + .../lucene/search/TopFieldCollector.java | 4 +- .../grouping/AllGroupHeadsCollector.java | 243 +++++- .../search/grouping/AllGroupsCollector.java | 42 +- .../grouping/BlockGroupingCollector.java | 2 +- .../grouping/DistinctValuesCollector.java | 92 ++- .../grouping/FirstPassGroupingCollector.java | 63 +- .../lucene/search/grouping/GroupReducer.java | 112 +++ .../lucene/search/grouping/GroupSelector.java | 73 ++ .../lucene/search/grouping/Grouper.java | 56 -- .../search/grouping/GroupingSearch.java | 24 +- .../grouping/SecondPassGroupingCollector.java | 146 +--- .../{term => }/TermGroupFacetCollector.java | 17 +- .../search/grouping/TermGroupSelector.java | 114 +++ .../search/grouping/TopGroupsCollector.java | 116 +++ .../grouping/ValueSourceGroupSelector.java | 86 ++ .../FunctionAllGroupHeadsCollector.java | 159 ---- .../function/FunctionAllGroupsCollector.java | 82 -- .../FunctionDistinctValuesCollector.java | 85 -- .../FunctionFirstPassGroupingCollector.java | 86 -- .../grouping/function/FunctionGrouper.java | 69 -- .../FunctionSecondPassGroupingCollector.java | 80 -- .../grouping/function/package-info.java | 21 - .../lucene/search/grouping/package-info.java | 34 +- .../term/TermAllGroupHeadsCollector.java | 767 ------------------ .../grouping/term/TermAllGroupsCollector.java | 128 --- .../term/TermDistinctValuesCollector.java | 162 ---- .../term/TermFirstPassGroupingCollector.java | 96 --- .../search/grouping/term/TermGrouper.java | 81 -- .../term/TermSecondPassGroupingCollector.java | 91 --- .../search/grouping/term/package-info.java | 21 - .../grouping/AllGroupHeadsCollectorTest.java | 13 +- .../grouping/AllGroupsCollectorTest.java | 20 +- .../grouping/DistinctValuesCollectorTest.java | 86 +- .../grouping/GroupFacetCollectorTest.java | 1 - .../lucene/search/grouping/TestGrouping.java | 89 +- .../response/SolrVelocityResourceLoader.java | 12 +- .../org/apache/solr/request/SimpleFacets.java | 7 +- .../java/org/apache/solr/search/Grouping.java | 57 +- .../solr/search/grouping/CommandHandler.java | 10 +- .../command/SearchGroupsFieldCommand.java | 26 +- .../command/TopGroupsFieldCommand.java | 40 +- .../org/apache/solr/TestGroupingSearch.java | 4 +- 44 files changed, 1103 insertions(+), 2430 deletions(-) create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java rename lucene/grouping/src/java/org/apache/lucene/search/grouping/{term => }/TermGroupFacetCollector.java (97%) create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java create mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGrouper.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java delete mode 100644 lucene/grouping/src/java/org/apache/lucene/search/grouping/term/package-info.java diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 1f3f30c79e4..cd25deeac6b 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -47,6 +47,9 @@ API Changes * LUCENE-7734: FieldType's copy constructor was widened to accept any IndexableFieldType. (David Smiley) +* LUCENE-7701: Grouping collectors have been refactored, such that groups are + now defined by a GroupSelector implementation. (Alan Woodward) + Bug Fixes * LUCENE-7626: IndexWriter will no longer accept broken token offsets diff --git a/lucene/MIGRATE.txt b/lucene/MIGRATE.txt index 1ccf56f6278..c7936a4bd7a 100644 --- a/lucene/MIGRATE.txt +++ b/lucene/MIGRATE.txt @@ -61,3 +61,16 @@ do not take deleted docs and doc values updates into account. Index-time boosts are not supported anymore. As a replacement, index-time scoring factors should be indexed in a doc value field and combined with the score at query time using FunctionScoreQuery for instance. + +## Grouping collector refactoring (LUCENE-7701) + +Groups are now defined by GroupSelector classes, making it easier to define new +types of groups. Rather than having term or function specific collection +classes, FirstPassGroupingCollector, AllGroupsCollector and +AllGroupHeadsCollector are now concrete classes taking a GroupSelector. + +SecondPassGroupingCollector is no longer specifically aimed at +collecting TopDocs for each group, but instead takes a GroupReducer that will +perform any type of reduction on the top groups collected on a first-pass. To +reproduce the old behaviour of SecondPassGroupingCollector, you should instead +use TopGroupsCollector. diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java index 1ec322fe5d9..97589c430de 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java @@ -320,11 +320,9 @@ public abstract class TopFieldCollector extends TopDocsCollector { * trackDocScores to true as well. * @return a {@link TopFieldCollector} instance which will sort the results by * the sort criteria. - * @throws IOException if there is a low-level I/O error */ public static TopFieldCollector create(Sort sort, int numHits, - boolean fillFields, boolean trackDocScores, boolean trackMaxScore) - throws IOException { + boolean fillFields, boolean trackDocScores, boolean trackMaxScore) { return create(sort, numHits, null, fillFields, trackDocScores, trackMaxScore); } diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java index b5fbdc36f1a..503b9529bd6 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupHeadsCollector.java @@ -18,27 +18,62 @@ package org.apache.lucene.search.grouping; import java.io.IOException; import java.util.Collection; +import java.util.HashMap; +import java.util.Map; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.FieldComparator; +import org.apache.lucene.search.LeafFieldComparator; +import org.apache.lucene.search.Scorer; import org.apache.lucene.search.SimpleCollector; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; import org.apache.lucene.util.FixedBitSet; /** - * This collector specializes in collecting the most relevant document (group head) for each group that match the query. + * This collector specializes in collecting the most relevant document (group head) for each + * group that matches the query. + * + * Clients should create new collectors by calling {@link #newCollector(GroupSelector, Sort)} * * @lucene.experimental */ @SuppressWarnings({"unchecked","rawtypes"}) public abstract class AllGroupHeadsCollector extends SimpleCollector { + private final GroupSelector groupSelector; + protected final Sort sort; + protected final int[] reversed; protected final int compIDXEnd; - protected final TemporalResult temporalResult; - protected AllGroupHeadsCollector(int numberOfSorts) { - this.reversed = new int[numberOfSorts]; - this.compIDXEnd = numberOfSorts - 1; - temporalResult = new TemporalResult(); + protected Map> heads = new HashMap<>(); + + protected LeafReaderContext context; + protected Scorer scorer; + + /** + * Create a new AllGroupHeadsCollector based on the type of within-group Sort required + * @param selector a GroupSelector to define the groups + * @param sort the within-group sort to use to choose the group head document + * @param the group value type + */ + public static AllGroupHeadsCollector newCollector(GroupSelector selector, Sort sort) { + if (sort.equals(Sort.RELEVANCE)) + return new ScoringGroupHeadsCollector<>(selector, sort); + return new SortingGroupHeadsCollector<>(selector, sort); + } + + private AllGroupHeadsCollector(GroupSelector selector, Sort sort) { + this.groupSelector = selector; + this.sort = sort; + this.reversed = new int[sort.getSort().length]; + final SortField[] sortFields = sort.getSort(); + for (int i = 0; i < sortFields.length; i++) { + reversed[i] = sortFields[i].getReverse() ? -1 : 1; + } + this.compIDXEnd = this.reversed.length - 1; } /** @@ -78,35 +113,28 @@ public abstract class AllGroupHeadsCollector extends SimpleCollector { return getCollectedGroupHeads().size(); } - /** - * Returns the group head and puts it into {@link #temporalResult}. - * If the group head wasn't encountered before then it will be added to the collected group heads. - *

    - * The {@link TemporalResult#stop} property will be true if the group head wasn't encountered before - * otherwise false. - * - * @param doc The document to retrieve the group head for. - * @throws IOException If I/O related errors occur - */ - protected abstract void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException; - /** * Returns the collected group heads. * Subsequent calls should return the same group heads. * * @return the collected group heads */ - protected abstract Collection> getCollectedGroupHeads(); + protected Collection> getCollectedGroupHeads() { + return heads.values(); + } @Override public void collect(int doc) throws IOException { - retrieveGroupHeadAndAddIfNotExist(doc); - if (temporalResult.stop) { + groupSelector.advanceTo(doc); + T groupValue = groupSelector.currentValue(); + if (heads.containsKey(groupValue) == false) { + groupValue = groupSelector.copyValue(); + heads.put(groupValue, newGroupHead(doc, groupValue, context, scorer)); return; } - GroupHead groupHead = temporalResult.groupHead; - // Ok now we need to check if the current doc is more relevant then current doc for this group + GroupHead groupHead = heads.get(groupValue); + // Ok now we need to check if the current doc is more relevant than top doc for this group for (int compIDX = 0; ; compIDX++) { final int c = reversed[compIDX] * groupHead.compare(compIDX, doc); if (c < 0) { @@ -125,17 +153,33 @@ public abstract class AllGroupHeadsCollector extends SimpleCollector { groupHead.updateDocHead(doc); } - /** - * Contains the result of group head retrieval. - * To prevent new object creations of this class for every collect. - */ - protected class TemporalResult { - - public GroupHead groupHead; - public boolean stop; - + @Override + public boolean needsScores() { + return sort.needsScores(); } + @Override + protected void doSetNextReader(LeafReaderContext context) throws IOException { + groupSelector.setNextReader(context); + this.context = context; + for (GroupHead head : heads.values()) { + head.setNextReader(context); + } + } + + @Override + public void setScorer(Scorer scorer) throws IOException { + this.scorer = scorer; + for (GroupHead head : heads.values()) { + head.setScorer(scorer); + } + } + + /** + * Create a new GroupHead for the given group value, initialized with a doc, context and scorer + */ + protected abstract GroupHead newGroupHead(int doc, T value, LeafReaderContext context, Scorer scorer) throws IOException; + /** * Represents a group head. A group head is the most relevant document for a particular group. * The relevancy is based is usually based on the sort. @@ -147,11 +191,29 @@ public abstract class AllGroupHeadsCollector extends SimpleCollector { public final T groupValue; public int doc; - protected GroupHead(T groupValue, int doc) { + protected int docBase; + + /** + * Create a new GroupHead for the given value + */ + protected GroupHead(T groupValue, int doc, int docBase) { this.groupValue = groupValue; - this.doc = doc; + this.doc = doc + docBase; + this.docBase = docBase; } + /** + * Called for each segment + */ + protected void setNextReader(LeafReaderContext ctx) throws IOException { + this.docBase = ctx.docBase; + } + + /** + * Called for each segment + */ + protected abstract void setScorer(Scorer scorer) throws IOException; + /** * Compares the specified document for a specified comparator against the current most relevant document. * @@ -173,4 +235,117 @@ public abstract class AllGroupHeadsCollector extends SimpleCollector { } + /** + * General implementation using a {@link FieldComparator} to select the group head + */ + private static class SortingGroupHeadsCollector extends AllGroupHeadsCollector { + + protected SortingGroupHeadsCollector(GroupSelector selector, Sort sort) { + super(selector, sort); + } + + @Override + protected GroupHead newGroupHead(int doc, T value, LeafReaderContext ctx, Scorer scorer) throws IOException { + return new SortingGroupHead<>(sort, value, doc, ctx, scorer); + } + } + + private static class SortingGroupHead extends GroupHead { + + final FieldComparator[] comparators; + final LeafFieldComparator[] leafComparators; + + protected SortingGroupHead(Sort sort, T groupValue, int doc, LeafReaderContext context, Scorer scorer) throws IOException { + super(groupValue, doc, context.docBase); + final SortField[] sortFields = sort.getSort(); + comparators = new FieldComparator[sortFields.length]; + leafComparators = new LeafFieldComparator[sortFields.length]; + for (int i = 0; i < sortFields.length; i++) { + comparators[i] = sortFields[i].getComparator(1, i); + leafComparators[i] = comparators[i].getLeafComparator(context); + leafComparators[i].setScorer(scorer); + leafComparators[i].copy(0, doc); + leafComparators[i].setBottom(0); + } + } + + @Override + public void setNextReader(LeafReaderContext ctx) throws IOException { + super.setNextReader(ctx); + for (int i = 0; i < comparators.length; i++) { + leafComparators[i] = comparators[i].getLeafComparator(ctx); + } + } + + @Override + protected void setScorer(Scorer scorer) throws IOException { + for (LeafFieldComparator c : leafComparators) { + c.setScorer(scorer); + } + } + + @Override + public int compare(int compIDX, int doc) throws IOException { + return leafComparators[compIDX].compareBottom(doc); + } + + @Override + public void updateDocHead(int doc) throws IOException { + for (LeafFieldComparator comparator : leafComparators) { + comparator.copy(0, doc); + comparator.setBottom(0); + } + this.doc = doc + docBase; + } + } + + /** + * Specialized implementation for sorting by score + */ + private static class ScoringGroupHeadsCollector extends AllGroupHeadsCollector { + + protected ScoringGroupHeadsCollector(GroupSelector selector, Sort sort) { + super(selector, sort); + } + + @Override + protected GroupHead newGroupHead(int doc, T value, LeafReaderContext context, Scorer scorer) throws IOException { + return new ScoringGroupHead<>(scorer, value, doc, context.docBase); + } + } + + private static class ScoringGroupHead extends GroupHead { + + private Scorer scorer; + private float topScore; + + protected ScoringGroupHead(Scorer scorer, T groupValue, int doc, int docBase) throws IOException { + super(groupValue, doc, docBase); + assert scorer.docID() == doc; + this.scorer = scorer; + this.topScore = scorer.score(); + } + + @Override + protected void setScorer(Scorer scorer) { + this.scorer = scorer; + } + + @Override + protected int compare(int compIDX, int doc) throws IOException { + assert scorer.docID() == doc; + assert compIDX == 0; + float score = scorer.score(); + int c = Float.compare(score, topScore); + if (c > 0) + topScore = score; + return c; + } + + @Override + protected void updateDocHead(int doc) throws IOException { + this.doc = doc + docBase; + } + } + } diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java index af697af5a84..8434534df4d 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/AllGroupsCollector.java @@ -18,23 +18,34 @@ package org.apache.lucene.search.grouping; import java.io.IOException; import java.util.Collection; +import java.util.HashSet; +import java.util.Set; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.SimpleCollector; -import org.apache.lucene.util.BytesRef; /** * A collector that collects all groups that match the * query. Only the group value is collected, and the order * is undefined. This collector does not determine * the most relevant document of a group. - *

    - * This is an abstract version. Concrete implementations define - * what a group actually is and how it is internally collected. * * @lucene.experimental */ -public abstract class AllGroupsCollector extends SimpleCollector { +public class AllGroupsCollector extends SimpleCollector { + + private final GroupSelector groupSelector; + + private final Set groups = new HashSet(); + + /** + * Create a new AllGroupsCollector + * @param groupSelector the GroupSelector to determine groups + */ + public AllGroupsCollector(GroupSelector groupSelector) { + this.groupSelector = groupSelector; + } /** * Returns the total number of groups for the executed search. @@ -49,17 +60,30 @@ public abstract class AllGroupsCollector extends SimpleCollector { /** * Returns the group values *

    - * This is an unordered collections of group values. For each group that matched the query there is a {@link BytesRef} - * representing a group value. + * This is an unordered collections of group values. * * @return the group values */ - public abstract Collection getGroups(); + public Collection getGroups() { + return groups; + } - // Empty not necessary @Override public void setScorer(Scorer scorer) throws IOException {} + @Override + protected void doSetNextReader(LeafReaderContext context) throws IOException { + groupSelector.setNextReader(context); + } + + @Override + public void collect(int doc) throws IOException { + groupSelector.advanceTo(doc); + if (groups.contains(groupSelector.currentValue())) + return; + groups.add(groupSelector.copyValue()); + } + @Override public boolean needsScores() { return false; // the result is unaffected by relevancy diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java index c965042a855..a50fda103d9 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java @@ -50,7 +50,7 @@ import org.apache.lucene.util.PriorityQueue; * being that the documents in each group must always be * indexed as a block. This collector also fills in * TopGroups.totalGroupCount without requiring the separate - * {@link org.apache.lucene.search.grouping.term.TermAllGroupsCollector}. However, this collector does + * {@link org.apache.lucene.search.grouping.AllGroupsCollector}. However, this collector does * not fill in the groupValue of each group; this field * will always be null. * diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java index 54d752c5876..103b0d26a70 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/DistinctValuesCollector.java @@ -16,10 +16,14 @@ */ package org.apache.lucene.search.grouping; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.SimpleCollector; /** @@ -27,33 +31,99 @@ import org.apache.lucene.search.SimpleCollector; * * @lucene.experimental */ -public abstract class DistinctValuesCollector extends SimpleCollector { +public class DistinctValuesCollector extends SecondPassGroupingCollector { + + /** + * Create a DistinctValuesCollector + * @param groupSelector the group selector to determine the top-level groups + * @param groups the top-level groups to collect for + * @param valueSelector a group selector to determine which values to collect per-group + */ + public DistinctValuesCollector(GroupSelector groupSelector, Collection> groups, + GroupSelector valueSelector) { + super(groupSelector, groups, new DistinctValuesReducer<>(valueSelector)); + } + + private static class ValuesCollector extends SimpleCollector { + + final GroupSelector valueSelector; + final Set values = new HashSet<>(); + + private ValuesCollector(GroupSelector valueSelector) { + this.valueSelector = valueSelector; + } + + @Override + public void collect(int doc) throws IOException { + if (valueSelector.advanceTo(doc) == GroupSelector.State.ACCEPT) { + R value = valueSelector.currentValue(); + if (values.contains(value) == false) + values.add(valueSelector.copyValue()); + } + else { + if (values.contains(null) == false) + values.add(null); + } + } + + @Override + protected void doSetNextReader(LeafReaderContext context) throws IOException { + valueSelector.setNextReader(context); + } + + @Override + public boolean needsScores() { + return false; + } + } + + private static class DistinctValuesReducer extends GroupReducer> { + + final GroupSelector valueSelector; + + private DistinctValuesReducer(GroupSelector valueSelector) { + this.valueSelector = valueSelector; + } + + @Override + public boolean needsScores() { + return false; + } + + @Override + protected ValuesCollector newCollector() { + return new ValuesCollector<>(valueSelector); + } + } /** * Returns all unique values for each top N group. * * @return all unique values for each top N group */ - public abstract List> getGroups(); + public List> getGroups() { + List> counts = new ArrayList<>(); + for (SearchGroup group : groups) { + @SuppressWarnings("unchecked") + ValuesCollector vc = (ValuesCollector) groupReducer.getCollector(group.groupValue); + counts.add(new GroupCount<>(group.groupValue, vc.values)); + } + return counts; + } /** * Returned by {@link DistinctValuesCollector#getGroups()}, * representing the value and set of distinct values for the group. */ - public static class GroupCount { + public static class GroupCount { public final T groupValue; - public final Set uniqueValues; + public final Set uniqueValues; - public GroupCount(T groupValue) { + public GroupCount(T groupValue, Set values) { this.groupValue = groupValue; - this.uniqueValues = new HashSet<>(); + this.uniqueValues = values; } } - @Override - public boolean needsScores() { - return false; // not needed to fetch all values - } - } diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java index 02bb1a27637..bd47adbb1f7 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/FirstPassGroupingCollector.java @@ -33,15 +33,16 @@ import org.apache.lucene.search.SortField; /** FirstPassGroupingCollector is the first of two passes necessary * to collect grouped hits. This pass gathers the top N sorted - * groups. Concrete subclasses define what a group is and how it - * is internally collected. + * groups. Groups are defined by a {@link GroupSelector} * *

    See {@link org.apache.lucene.search.grouping} for more * details including a full code example.

    * * @lucene.experimental */ -abstract public class FirstPassGroupingCollector extends SimpleCollector { +public class FirstPassGroupingCollector extends SimpleCollector { + + private final GroupSelector groupSelector; private final FieldComparator[] comparators; private final LeafFieldComparator[] leafComparators; @@ -60,16 +61,18 @@ abstract public class FirstPassGroupingCollector extends SimpleCollector { /** * Create the first pass collector. * - * @param groupSort The {@link Sort} used to sort the + * @param groupSelector a GroupSelector used to defined groups + * @param groupSort The {@link Sort} used to sort the * groups. The top sorted document within each group * according to groupSort, determines how that group * sorts against other groups. This must be non-null, * ie, if you want to groupSort by relevance use * Sort.RELEVANCE. - * @param topNGroups How many top groups to keep. + * @param topNGroups How many top groups to keep. */ @SuppressWarnings({"unchecked", "rawtypes"}) - public FirstPassGroupingCollector(Sort groupSort, int topNGroups) { + public FirstPassGroupingCollector(GroupSelector groupSelector, Sort groupSort, int topNGroups) { + this.groupSelector = groupSelector; if (topNGroups < 1) { throw new IllegalArgumentException("topNGroups must be >= 1 (got " + topNGroups + ")"); } @@ -133,7 +136,7 @@ abstract public class FirstPassGroupingCollector extends SimpleCollector { if (upto++ < groupOffset) { continue; } - //System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString())); + // System.out.println(" group=" + (group.groupValue == null ? "null" : group.groupValue.toString())); SearchGroup searchGroup = new SearchGroup<>(); searchGroup.groupValue = group.groupValue; if (fillFields) { @@ -155,14 +158,11 @@ abstract public class FirstPassGroupingCollector extends SimpleCollector { } } - @Override - public void collect(int doc) throws IOException { - //System.out.println("FP.collect doc=" + doc); - + private boolean isCompetitive(int doc) throws IOException { // If orderedGroups != null we already have collected N groups and // can short circuit by comparing this document to the bottom group, // without having to find what group this document belongs to. - + // Even if this document belongs to a group in the top N, we'll know that // we don't have to update that group. @@ -173,7 +173,7 @@ abstract public class FirstPassGroupingCollector extends SimpleCollector { final int c = reversed[compIDX] * leafComparators[compIDX].compareBottom(doc); if (c < 0) { // Definitely not competitive. So don't even bother to continue - return; + return false; } else if (c > 0) { // Definitely competitive. break; @@ -181,15 +181,24 @@ abstract public class FirstPassGroupingCollector extends SimpleCollector { // Here c=0. If we're at the last comparator, this doc is not // competitive, since docs are visited in doc Id order, which means // this doc cannot compete with any other document in the queue. - return; + return false; } } } + return true; + } + + @Override + public void collect(int doc) throws IOException { + + if (isCompetitive(doc) == false) + return; // TODO: should we add option to mean "ignore docs that // don't have the group field" (instead of stuffing them // under null group)? - final T groupValue = getDocGroupValue(doc); + groupSelector.advanceTo(doc); + T groupValue = groupSelector.currentValue(); final CollectedSearchGroup group = groupMap.get(groupValue); @@ -207,7 +216,7 @@ abstract public class FirstPassGroupingCollector extends SimpleCollector { // Add a new CollectedSearchGroup: CollectedSearchGroup sg = new CollectedSearchGroup<>(); - sg.groupValue = copyDocGroupValue(groupValue, null); + sg.groupValue = groupSelector.copyValue(); sg.comparatorSlot = groupMap.size(); sg.topDoc = docBase + doc; for (LeafFieldComparator fc : leafComparators) { @@ -233,7 +242,7 @@ abstract public class FirstPassGroupingCollector extends SimpleCollector { groupMap.remove(bottomGroup.groupValue); // reuse the removed CollectedSearchGroup - bottomGroup.groupValue = copyDocGroupValue(groupValue, bottomGroup.groupValue); + bottomGroup.groupValue = groupSelector.copyValue(); bottomGroup.topDoc = docBase + doc; for (LeafFieldComparator fc : leafComparators) { @@ -338,25 +347,15 @@ abstract public class FirstPassGroupingCollector extends SimpleCollector { for (int i=0; i getGroupSelector() { + return groupSelector; + } } diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java new file mode 100644 index 00000000000..4366e9123f7 --- /dev/null +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupReducer.java @@ -0,0 +1,112 @@ +/* + * 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.lucene.search.grouping; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Scorer; + +/** + * Concrete implementations of this class define what to collect for individual + * groups during the second-pass of a grouping search. + * + * Each group is assigned a Collector returned by {@link #newCollector()}, and + * {@link LeafCollector#collect(int)} is called for each document that is in + * a group + * + * @see SecondPassGroupingCollector + * + * @param the type of the value used for grouping + * @param the type of {@link Collector} used to reduce each group + */ +public abstract class GroupReducer { + + private final Map> groups = new HashMap<>(); + + /** + * Define which groups should be reduced. + * + * Called by {@link SecondPassGroupingCollector} + */ + public void setGroups(Collection> groups) { + for (SearchGroup group : groups) { + this.groups.put(group.groupValue, new GroupCollector<>(newCollector())); + } + } + + /** + * Whether or not this reducer requires collected documents to be scored + */ + public abstract boolean needsScores(); + + /** + * Creates a new Collector for each group + */ + protected abstract C newCollector(); + + /** + * Get the Collector for a given group + */ + public final C getCollector(T value) { + return groups.get(value).collector; + } + + /** + * Collect a given document into a given group + * @throws IOException on error + */ + public final void collect(T value, int doc) throws IOException { + GroupCollector collector = groups.get(value); + collector.leafCollector.collect(doc); + } + + /** + * Set the Scorer on all group collectors + */ + public final void setScorer(Scorer scorer) throws IOException { + for (GroupCollector collector : groups.values()) { + collector.leafCollector.setScorer(scorer); + } + } + + /** + * Called when the parent {@link SecondPassGroupingCollector} moves to a new segment + */ + public final void setNextReader(LeafReaderContext ctx) throws IOException { + for (GroupCollector collector : groups.values()) { + collector.leafCollector = collector.collector.getLeafCollector(ctx); + } + } + + private static final class GroupCollector { + + final C collector; + LeafCollector leafCollector; + + private GroupCollector(C collector) { + this.collector = collector; + } + } + +} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.java new file mode 100644 index 00000000000..dbb09329f8a --- /dev/null +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupSelector.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.lucene.search.grouping; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.lucene.index.LeafReaderContext; + +/** + * Defines a group, for use by grouping collectors + * + * A GroupSelector acts as an iterator over documents. For each segment, clients + * should call {@link #setNextReader(LeafReaderContext)}, and then {@link #advanceTo(int)} + * for each matching document. + * + * @param the type of the group value + */ +public abstract class GroupSelector { + + /** + * What to do with the current value + */ + public enum State { SKIP, ACCEPT } + + /** + * Set the LeafReaderContext + */ + public abstract void setNextReader(LeafReaderContext readerContext) throws IOException; + + /** + * Advance the GroupSelector's iterator to the given document + */ + public abstract State advanceTo(int doc) throws IOException; + + /** + * Get the group value of the current document + * + * N.B. this object may be reused, for a persistent version use {@link #copyValue()} + */ + public abstract T currentValue(); + + /** + * @return a copy of the group value of the current document + */ + public abstract T copyValue(); + + /** + * Set a restriction on the group values returned by this selector + * + * If the selector is positioned on a document whose group value is not contained + * within this set, then {@link #advanceTo(int)} will return {@link State#SKIP} + * + * @param groups a set of {@link SearchGroup} objects to limit selections to + */ + public abstract void setGroups(Collection> groups); + +} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java deleted file mode 100644 index 2ff79a1331b..00000000000 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/Grouper.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.lucene.search.grouping; - -import java.io.IOException; -import java.util.Collection; - -import org.apache.lucene.search.Sort; - -/** - * A factory object to create first and second-pass collectors, run by a {@link GroupingSearch} - * @param the type the group value - */ -public abstract class Grouper { - - /** - * Create a first-pass collector - * @param sort the order in which groups should be returned - * @param count how many groups to return - */ - public abstract FirstPassGroupingCollector getFirstPassCollector(Sort sort, int count) throws IOException; - - /** - * Create an {@link AllGroupsCollector} - */ - public abstract AllGroupsCollector getAllGroupsCollector(); - - /** - * Create an {@link AllGroupHeadsCollector} - * @param sort a within-group sort order to determine which doc is the group head - */ - public abstract AllGroupHeadsCollector getGroupHeadsCollector(Sort sort); - - /** - * Create a second-pass collector - */ - public abstract SecondPassGroupingCollector getSecondPassCollector( - Collection> groups, Sort groupSort, Sort withinGroupSort, - int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException; - -} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java index f4319d578d3..a36917d742a 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/GroupingSearch.java @@ -30,8 +30,6 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; import org.apache.lucene.search.Weight; -import org.apache.lucene.search.grouping.function.FunctionGrouper; -import org.apache.lucene.search.grouping.term.TermGrouper; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.mutable.MutableValue; @@ -43,7 +41,7 @@ import org.apache.lucene.util.mutable.MutableValue; */ public class GroupingSearch { - private final Grouper grouper; + private final GroupSelector grouper; private final Query groupEndDocs; private Sort groupSort = Sort.RELEVANCE; @@ -71,11 +69,7 @@ public class GroupingSearch { * @param groupField The name of the field to group by. */ public GroupingSearch(String groupField) { - this(new TermGrouper(groupField, 128), null); - } - - public GroupingSearch(String groupField, int initialSize) { - this(new TermGrouper(groupField, initialSize), null); + this(new TermGroupSelector(groupField), null); } /** @@ -86,7 +80,7 @@ public class GroupingSearch { * @param valueSourceContext The context of the specified groupFunction */ public GroupingSearch(ValueSource groupFunction, Map valueSourceContext) { - this(new FunctionGrouper(groupFunction, valueSourceContext), null); + this(new ValueSourceGroupSelector(groupFunction, valueSourceContext), null); } /** @@ -99,7 +93,7 @@ public class GroupingSearch { this(null, groupEndDocs); } - private GroupingSearch(Grouper grouper, Query groupEndDocs) { + private GroupingSearch(GroupSelector grouper, Query groupEndDocs) { this.grouper = grouper; this.groupEndDocs = groupEndDocs; } @@ -129,10 +123,10 @@ public class GroupingSearch { protected TopGroups groupByFieldOrFunction(IndexSearcher searcher, Query query, int groupOffset, int groupLimit) throws IOException { int topN = groupOffset + groupLimit; - final FirstPassGroupingCollector firstPassCollector = grouper.getFirstPassCollector(groupSort, topN); - final AllGroupsCollector allGroupsCollector = allGroups ? grouper.getAllGroupsCollector() : null; + final FirstPassGroupingCollector firstPassCollector = new FirstPassGroupingCollector(grouper, groupSort, topN); + final AllGroupsCollector allGroupsCollector = allGroups ? new AllGroupsCollector(grouper) : null; final AllGroupHeadsCollector allGroupHeadsCollector - = allGroupHeads ? grouper.getGroupHeadsCollector(sortWithinGroup) : null; + = allGroupHeads ? AllGroupHeadsCollector.newCollector(grouper, sortWithinGroup) : null; final Collector firstRound = MultiCollector.wrap(firstPassCollector, allGroupsCollector, allGroupHeadsCollector); @@ -158,8 +152,8 @@ public class GroupingSearch { } int topNInsideGroup = groupDocsOffset + groupDocsLimit; - SecondPassGroupingCollector secondPassCollector - = grouper.getSecondPassCollector(topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup, + TopGroupsCollector secondPassCollector + = new TopGroupsCollector(grouper, topSearchGroups, groupSort, sortWithinGroup, topNInsideGroup, includeScores, includeMaxScore, fillSortFields); if (cachedCollector != null && cachedCollector.isCached()) { diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java index f8feb756f54..c54c8eee4a8 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/SecondPassGroupingCollector.java @@ -18,152 +18,82 @@ package org.apache.lucene.search.grouping; import java.io.IOException; import java.util.Collection; -import java.util.HashMap; -import java.util.Map; import java.util.Objects; import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.search.LeafCollector; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.SimpleCollector; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.TopDocsCollector; -import org.apache.lucene.search.TopFieldCollector; -import org.apache.lucene.search.TopScoreDocCollector; /** - * SecondPassGroupingCollector is the second of two passes - * necessary to collect grouped docs. This pass gathers the - * top N documents per top group computed from the - * first pass. Concrete subclasses define what a group is and how it - * is internally collected. + * SecondPassGroupingCollector runs over an already collected set of + * groups, further applying a {@link GroupReducer} to each group * - *

    See {@link org.apache.lucene.search.grouping} for more - * details including a full code example.

    + * @see TopGroupsCollector + * @see DistinctValuesCollector * * @lucene.experimental */ -public abstract class SecondPassGroupingCollector extends SimpleCollector { +public class SecondPassGroupingCollector extends SimpleCollector { - private final Collection> groups; - private final Sort groupSort; - private final Sort withinGroupSort; - private final int maxDocsPerGroup; - private final boolean needsScores; - protected final Map> groupMap; + protected final GroupSelector groupSelector; + protected final Collection> groups; + protected final GroupReducer groupReducer; - protected SearchGroupDocs[] groupDocs; + protected int totalHitCount; + protected int totalGroupedHitCount; - private int totalHitCount; - private int totalGroupedHitCount; - - public SecondPassGroupingCollector(Collection> groups, Sort groupSort, Sort withinGroupSort, - int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) - throws IOException { + /** + * Create a new SecondPassGroupingCollector + * @param groupSelector the GroupSelector that defines groups for this search + * @param groups the groups to collect documents for + * @param reducer the reducer to apply to each group + */ + public SecondPassGroupingCollector(GroupSelector groupSelector, Collection> groups, GroupReducer reducer) { //System.out.println("SP init"); if (groups.isEmpty()) { throw new IllegalArgumentException("no groups to collect (groups is empty)"); } - this.groups = Objects.requireNonNull(groups); - this.groupSort = Objects.requireNonNull(groupSort); - this.withinGroupSort = Objects.requireNonNull(withinGroupSort); - this.maxDocsPerGroup = maxDocsPerGroup; - this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores(); + this.groupSelector = Objects.requireNonNull(groupSelector); + this.groupSelector.setGroups(groups); - this.groupMap = new HashMap<>(groups.size()); - for (SearchGroup group : groups) { - //System.out.println(" prep group=" + (group.groupValue == null ? "null" : group.groupValue.utf8ToString())); - final TopDocsCollector collector; - if (withinGroupSort.equals(Sort.RELEVANCE)) { // optimize to use TopScoreDocCollector - // Sort by score - collector = TopScoreDocCollector.create(maxDocsPerGroup); - } else { - // Sort by fields - collector = TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores); - } - groupMap.put(group.groupValue, new SearchGroupDocs<>(group.groupValue, collector)); - } + this.groups = Objects.requireNonNull(groups); + this.groupReducer = reducer; + reducer.setGroups(groups); + } + + /** + * @return the GroupSelector used in this collector + */ + public GroupSelector getGroupSelector() { + return groupSelector; } @Override public boolean needsScores() { - return needsScores; + return groupReducer.needsScores(); } @Override public void setScorer(Scorer scorer) throws IOException { - for (SearchGroupDocs group : groupMap.values()) { - group.leafCollector.setScorer(scorer); - } + groupReducer.setScorer(scorer); } @Override public void collect(int doc) throws IOException { totalHitCount++; - SearchGroupDocs group = retrieveGroup(doc); - if (group != null) { - totalGroupedHitCount++; - group.leafCollector.collect(doc); - } + if (groupSelector.advanceTo(doc) == GroupSelector.State.SKIP) + return; + totalGroupedHitCount++; + T value = groupSelector.currentValue(); + groupReducer.collect(value, doc); } - /** - * Returns the group the specified doc belongs to or null if no group could be retrieved. - * - * @param doc The specified doc - * @return the group the specified doc belongs to or null if no group could be retrieved - * @throws IOException If an I/O related error occurred - */ - protected abstract SearchGroupDocs retrieveGroup(int doc) throws IOException; - @Override protected void doSetNextReader(LeafReaderContext readerContext) throws IOException { - //System.out.println("SP.setNextReader"); - for (SearchGroupDocs group : groupMap.values()) { - group.leafCollector = group.collector.getLeafCollector(readerContext); - } + groupReducer.setNextReader(readerContext); + groupSelector.setNextReader(readerContext); } - public TopGroups getTopGroups(int withinGroupOffset) { - @SuppressWarnings({"unchecked","rawtypes"}) - final GroupDocs[] groupDocsResult = (GroupDocs[]) new GroupDocs[groups.size()]; - - int groupIDX = 0; - float maxScore = Float.MIN_VALUE; - for(SearchGroup group : groups) { - final SearchGroupDocs groupDocs = groupMap.get(group.groupValue); - final TopDocs topDocs = groupDocs.collector.topDocs(withinGroupOffset, maxDocsPerGroup); - groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN, - topDocs.getMaxScore(), - topDocs.totalHits, - topDocs.scoreDocs, - groupDocs.groupValue, - group.sortValues); - maxScore = Math.max(maxScore, topDocs.getMaxScore()); - } - - return new TopGroups<>(groupSort.getSort(), - withinGroupSort.getSort(), - totalHitCount, totalGroupedHitCount, groupDocsResult, - maxScore); - } - - - // TODO: merge with SearchGroup or not? - // ad: don't need to build a new hashmap - // disad: blows up the size of SearchGroup if we need many of them, and couples implementations - public class SearchGroupDocs { - - public final T groupValue; - public final TopDocsCollector collector; - public LeafCollector leafCollector; - - public SearchGroupDocs(T groupValue, TopDocsCollector collector) { - this.groupValue = groupValue; - this.collector = collector; - } - } } diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java similarity index 97% rename from lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java rename to lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java index cee327cbf47..39d28a57996 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupFacetCollector.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.lucene.search.grouping.term; +package org.apache.lucene.search.grouping; import java.io.IOException; import java.util.ArrayList; @@ -25,7 +25,6 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SortedDocValues; import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.search.grouping.GroupFacetCollector; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.SentinelIntSet; @@ -401,15 +400,15 @@ public abstract class TermGroupFacetCollector extends GroupFacetCollector { } } } -} -class GroupedFacetHit { + private static class GroupedFacetHit { - final BytesRef groupValue; - final BytesRef facetValue; + final BytesRef groupValue; + final BytesRef facetValue; - GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) { - this.groupValue = groupValue; - this.facetValue = facetValue; + GroupedFacetHit(BytesRef groupValue, BytesRef facetValue) { + this.groupValue = groupValue; + this.facetValue = facetValue; + } } } diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java new file mode 100644 index 00000000000..5b8f77c2ce8 --- /dev/null +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TermGroupSelector.java @@ -0,0 +1,114 @@ +/* + * 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.lucene.search.grouping; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefHash; + +/** + * A GroupSelector implementation that groups via SortedDocValues + */ +public class TermGroupSelector extends GroupSelector { + + private final String field; + private final BytesRefHash values = new BytesRefHash(); + private final Map ordsToGroupIds = new HashMap<>(); + + private SortedDocValues docValues; + private int groupId; + + private boolean secondPass; + private boolean includeEmpty; + + /** + * Create a new TermGroupSelector + * @param field the SortedDocValues field to use for grouping + */ + public TermGroupSelector(String field) { + this.field = field; + } + + @Override + public void setNextReader(LeafReaderContext readerContext) throws IOException { + this.docValues = DocValues.getSorted(readerContext.reader(), field); + this.ordsToGroupIds.clear(); + BytesRef scratch = new BytesRef(); + for (int i = 0; i < values.size(); i++) { + values.get(i, scratch); + int ord = this.docValues.lookupTerm(scratch); + if (ord >= 0) + ordsToGroupIds.put(ord, i); + } + } + + @Override + public State advanceTo(int doc) throws IOException { + if (this.docValues.advanceExact(doc) == false) { + groupId = -1; + return includeEmpty ? State.ACCEPT : State.SKIP; + } + int ord = docValues.ordValue(); + if (ordsToGroupIds.containsKey(ord)) { + groupId = ordsToGroupIds.get(ord); + return State.ACCEPT; + } + if (secondPass) + return State.SKIP; + groupId = values.add(docValues.binaryValue()); + ordsToGroupIds.put(ord, groupId); + return State.ACCEPT; + } + + private BytesRef scratch = new BytesRef(); + + @Override + public BytesRef currentValue() { + if (groupId == -1) + return null; + values.get(groupId, scratch); + return scratch; + } + + @Override + public BytesRef copyValue() { + if (groupId == -1) + return null; + return BytesRef.deepCopyOf(currentValue()); + } + + @Override + public void setGroups(Collection> searchGroups) { + this.values.clear(); + this.values.reinit(); + for (SearchGroup sg : searchGroups) { + if (sg.groupValue == null) + includeEmpty = true; + else + this.values.add(sg.groupValue); + } + this.secondPass = true; + } +} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java new file mode 100644 index 00000000000..b6c71d37a8d --- /dev/null +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/TopGroupsCollector.java @@ -0,0 +1,116 @@ +/* + * 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.lucene.search.grouping; + +import java.util.Collection; +import java.util.Objects; +import java.util.function.Supplier; + +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.TopDocsCollector; +import org.apache.lucene.search.TopFieldCollector; +import org.apache.lucene.search.TopScoreDocCollector; + +/** + * A second-pass collector that collects the TopDocs for each group, and + * returns them as a {@link TopGroups} object + * + * @param the type of the group value + */ +public class TopGroupsCollector extends SecondPassGroupingCollector { + + private final Sort groupSort; + private final Sort withinGroupSort; + private final int maxDocsPerGroup; + + /** + * Create a new TopGroupsCollector + * @param groupSelector the group selector used to define groups + * @param groups the groups to collect TopDocs for + * @param groupSort the order in which groups are returned + * @param withinGroupSort the order in which documents are sorted in each group + * @param maxDocsPerGroup the maximum number of docs to collect for each group + * @param getScores if true, record the scores of all docs in each group + * @param getMaxScores if true, record the maximum score for each group + * @param fillSortFields if true, record the sort field values for all docs + */ + public TopGroupsCollector(GroupSelector groupSelector, Collection> groups, Sort groupSort, Sort withinGroupSort, + int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) { + super(groupSelector, groups, + new TopDocsReducer<>(withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields)); + this.groupSort = Objects.requireNonNull(groupSort); + this.withinGroupSort = Objects.requireNonNull(withinGroupSort); + this.maxDocsPerGroup = maxDocsPerGroup; + + } + + private static class TopDocsReducer extends GroupReducer> { + + private final Supplier> supplier; + private final boolean needsScores; + + TopDocsReducer(Sort withinGroupSort, + int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) { + this.needsScores = getScores || getMaxScores || withinGroupSort.needsScores(); + this.supplier = withinGroupSort == Sort.RELEVANCE ? + () -> TopScoreDocCollector.create(maxDocsPerGroup) : + () -> TopFieldCollector.create(withinGroupSort, maxDocsPerGroup, fillSortFields, getScores, getMaxScores); + } + + @Override + public boolean needsScores() { + return needsScores; + } + + @Override + protected TopDocsCollector newCollector() { + return supplier.get(); + } + } + + /** + * Get the TopGroups recorded by this collector + * @param withinGroupOffset the offset within each group to start collecting documents + */ + public TopGroups getTopGroups(int withinGroupOffset) { + @SuppressWarnings({"unchecked","rawtypes"}) + final GroupDocs[] groupDocsResult = (GroupDocs[]) new GroupDocs[groups.size()]; + + int groupIDX = 0; + float maxScore = Float.MIN_VALUE; + for(SearchGroup group : groups) { + TopDocsCollector collector = (TopDocsCollector) groupReducer.getCollector(group.groupValue); + final TopDocs topDocs = collector.topDocs(withinGroupOffset, maxDocsPerGroup); + groupDocsResult[groupIDX++] = new GroupDocs<>(Float.NaN, + topDocs.getMaxScore(), + topDocs.totalHits, + topDocs.scoreDocs, + group.groupValue, + group.sortValues); + maxScore = Math.max(maxScore, topDocs.getMaxScore()); + } + + return new TopGroups<>(groupSort.getSort(), + withinGroupSort.getSort(), + totalHitCount, totalGroupedHitCount, groupDocsResult, + maxScore); + } + + +} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java new file mode 100644 index 00000000000..24901601885 --- /dev/null +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/ValueSourceGroupSelector.java @@ -0,0 +1,86 @@ +/* + * 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.lucene.search.grouping; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.queries.function.FunctionValues; +import org.apache.lucene.queries.function.ValueSource; +import org.apache.lucene.util.mutable.MutableValue; + +/** + * A GroupSelector that groups via a ValueSource + */ +public class ValueSourceGroupSelector extends GroupSelector { + + private final ValueSource valueSource; + private final Map context; + + private Set secondPassGroups; + + /** + * Create a new ValueSourceGroupSelector + * @param valueSource the ValueSource to group by + * @param context a context map for the ValueSource + */ + public ValueSourceGroupSelector(ValueSource valueSource, Map context) { + this.valueSource = valueSource; + this.context = context; + } + + private FunctionValues.ValueFiller filler; + + @Override + public void setNextReader(LeafReaderContext readerContext) throws IOException { + FunctionValues values = valueSource.getValues(context, readerContext); + this.filler = values.getValueFiller(); + } + + @Override + public State advanceTo(int doc) throws IOException { + this.filler.fillValue(doc); + if (secondPassGroups != null) { + if (secondPassGroups.contains(filler.getValue()) == false) + return State.SKIP; + } + return State.ACCEPT; + } + + @Override + public MutableValue currentValue() { + return filler.getValue(); + } + + @Override + public MutableValue copyValue() { + return filler.getValue().duplicate(); + } + + @Override + public void setGroups(Collection> searchGroups) { + secondPassGroups = new HashSet<>(); + for (SearchGroup group : searchGroups) { + secondPassGroups.add(group.groupValue); + } + } +} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java deleted file mode 100644 index f4d46682497..00000000000 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.lucene.search.grouping.function; - -import java.io.IOException; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.queries.function.FunctionValues; -import org.apache.lucene.queries.function.ValueSource; -import org.apache.lucene.search.FieldComparator; -import org.apache.lucene.search.LeafFieldComparator; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; -import org.apache.lucene.search.grouping.AllGroupHeadsCollector; -import org.apache.lucene.util.mutable.MutableValue; - -/** - * An implementation of {@link AllGroupHeadsCollector} for retrieving the most relevant groups when grouping - * by {@link ValueSource}. - * - * @lucene.experimental - */ -public class FunctionAllGroupHeadsCollector extends AllGroupHeadsCollector { - - private final ValueSource groupBy; - private final Map vsContext; - private final Map groups; - private final Sort sortWithinGroup; - - private FunctionValues.ValueFiller filler; - private MutableValue mval; - private LeafReaderContext readerContext; - private Scorer scorer; - - /** - * Constructs a {@link FunctionAllGroupHeadsCollector} instance. - * - * @param groupBy The {@link ValueSource} to group by - * @param vsContext The ValueSource context - * @param sortWithinGroup The sort within a group - */ - public FunctionAllGroupHeadsCollector(ValueSource groupBy, Map vsContext, Sort sortWithinGroup) { - super(sortWithinGroup.getSort().length); - groups = new HashMap<>(); - this.sortWithinGroup = sortWithinGroup; - this.groupBy = groupBy; - this.vsContext = vsContext; - - final SortField[] sortFields = sortWithinGroup.getSort(); - for (int i = 0; i < sortFields.length; i++) { - reversed[i] = sortFields[i].getReverse() ? -1 : 1; - } - } - - @Override - protected void retrieveGroupHeadAndAddIfNotExist(int doc) throws IOException { - filler.fillValue(doc); - FunctionGroupHead groupHead = groups.get(mval); - if (groupHead == null) { - MutableValue groupValue = mval.duplicate(); - groupHead = new FunctionGroupHead(groupValue, sortWithinGroup, doc); - groups.put(groupValue, groupHead); - temporalResult.stop = true; - } else { - temporalResult.stop = false; - } - this.temporalResult.groupHead = groupHead; - } - - @Override - protected Collection getCollectedGroupHeads() { - return groups.values(); - } - - @Override - public void setScorer(Scorer scorer) throws IOException { - this.scorer = scorer; - for (FunctionGroupHead groupHead : groups.values()) { - for (LeafFieldComparator comparator : groupHead.leafComparators) { - comparator.setScorer(scorer); - } - } - } - - @Override - protected void doSetNextReader(LeafReaderContext context) throws IOException { - this.readerContext = context; - FunctionValues values = groupBy.getValues(vsContext, context); - filler = values.getValueFiller(); - mval = filler.getValue(); - - for (FunctionGroupHead groupHead : groups.values()) { - for (int i = 0; i < groupHead.comparators.length; i++) { - groupHead.leafComparators[i] = groupHead.comparators[i].getLeafComparator(context); - } - } - } - - /** Holds current head document for a single group. - * - * @lucene.experimental */ - public class FunctionGroupHead extends AllGroupHeadsCollector.GroupHead { - - final FieldComparator[] comparators; - final LeafFieldComparator[] leafComparators; - - @SuppressWarnings({"unchecked","rawtypes"}) - private FunctionGroupHead(MutableValue groupValue, Sort sort, int doc) throws IOException { - super(groupValue, doc + readerContext.docBase); - final SortField[] sortFields = sort.getSort(); - comparators = new FieldComparator[sortFields.length]; - leafComparators = new LeafFieldComparator[sortFields.length]; - for (int i = 0; i < sortFields.length; i++) { - comparators[i] = sortFields[i].getComparator(1, i); - leafComparators[i] = comparators[i].getLeafComparator(readerContext); - leafComparators[i].setScorer(scorer); - leafComparators[i].copy(0, doc); - leafComparators[i].setBottom(0); - } - } - - @Override - public int compare(int compIDX, int doc) throws IOException { - return leafComparators[compIDX].compareBottom(doc); - } - - @Override - public void updateDocHead(int doc) throws IOException { - for (LeafFieldComparator comparator : leafComparators) { - comparator.copy(0, doc); - comparator.setBottom(0); - } - this.doc = doc + readerContext.docBase; - } - } - - @Override - public boolean needsScores() { - return sortWithinGroup.needsScores(); - } -} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java deleted file mode 100644 index 1609d4d7bcf..00000000000 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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.lucene.search.grouping.function; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.queries.function.FunctionValues; -import org.apache.lucene.queries.function.ValueSource; -import org.apache.lucene.search.grouping.AllGroupsCollector; -import org.apache.lucene.util.mutable.MutableValue; - -import java.io.IOException; -import java.util.Collection; -import java.util.Map; -import java.util.SortedSet; -import java.util.TreeSet; - -/** - * A collector that collects all groups that match the - * query. Only the group value is collected, and the order - * is undefined. This collector does not determine - * the most relevant document of a group. - *

    - * Implementation detail: Uses {@link ValueSource} and {@link FunctionValues} to retrieve the - * field values to group by. - * - * @lucene.experimental - */ -public class FunctionAllGroupsCollector extends AllGroupsCollector { - - private final Map vsContext; - private final ValueSource groupBy; - private final SortedSet groups = new TreeSet<>(); - - private FunctionValues.ValueFiller filler; - private MutableValue mval; - - /** - * Constructs a {@link FunctionAllGroupsCollector} instance. - * - * @param groupBy The {@link ValueSource} to group by - * @param vsContext The ValueSource context - */ - public FunctionAllGroupsCollector(ValueSource groupBy, Map vsContext) { - this.vsContext = vsContext; - this.groupBy = groupBy; - } - - @Override - public Collection getGroups() { - return groups; - } - - @Override - public void collect(int doc) throws IOException { - filler.fillValue(doc); - if (!groups.contains(mval)) { - groups.add(mval.duplicate()); - } - } - - @Override - protected void doSetNextReader(LeafReaderContext context) throws IOException { - FunctionValues values = groupBy.getValues(vsContext, context); - filler = values.getValueFiller(); - mval = filler.getValue(); - } - -} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java deleted file mode 100644 index 69ead076d6f..00000000000 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.lucene.search.grouping.function; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.queries.function.FunctionValues; -import org.apache.lucene.queries.function.ValueSource; -import org.apache.lucene.search.grouping.DistinctValuesCollector; -import org.apache.lucene.search.grouping.SearchGroup; -import org.apache.lucene.util.mutable.MutableValue; - -/** - * Function based implementation of {@link DistinctValuesCollector}. - * - * @lucene.experimental - */ -public class FunctionDistinctValuesCollector extends DistinctValuesCollector { - - private final Map vsContext; - private final ValueSource groupSource; - private final ValueSource countSource; - private final Map> groupMap; - - private FunctionValues.ValueFiller groupFiller; - private FunctionValues.ValueFiller countFiller; - private MutableValue groupMval; - private MutableValue countMval; - - public FunctionDistinctValuesCollector(Map vsContext, ValueSource groupSource, ValueSource countSource, Collection> groups) { - this.vsContext = vsContext; - this.groupSource = groupSource; - this.countSource = countSource; - groupMap = new LinkedHashMap<>(); - for (SearchGroup group : groups) { - groupMap.put(group.groupValue, new GroupCount<>(group.groupValue)); - } - } - - @Override - public List> getGroups() { - return new ArrayList<>(groupMap.values()); - } - - @Override - public void collect(int doc) throws IOException { - groupFiller.fillValue(doc); - GroupCount groupCount = groupMap.get(groupMval); - if (groupCount != null) { - countFiller.fillValue(doc); - groupCount.uniqueValues.add(countMval.duplicate()); - } - } - - @Override - protected void doSetNextReader(LeafReaderContext context) throws IOException { - FunctionValues values = groupSource.getValues(vsContext, context); - groupFiller = values.getValueFiller(); - groupMval = groupFiller.getValue(); - values = countSource.getValues(vsContext, context); - countFiller = values.getValueFiller(); - countMval = countFiller.getValue(); - } - -} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java deleted file mode 100644 index 85376e6a066..00000000000 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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.lucene.search.grouping.function; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.queries.function.FunctionValues; -import org.apache.lucene.queries.function.ValueSource; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.grouping.FirstPassGroupingCollector; -import org.apache.lucene.util.mutable.MutableValue; - -import java.io.IOException; -import java.util.Map; - -/** - * Concrete implementation of {@link FirstPassGroupingCollector} that groups based on - * {@link ValueSource} instances. - * - * @lucene.experimental - */ -public class FunctionFirstPassGroupingCollector extends FirstPassGroupingCollector { - - private final ValueSource groupByVS; - private final Map vsContext; - - private FunctionValues.ValueFiller filler; - private MutableValue mval; - - /** - * Creates a first pass collector. - * - * @param groupByVS The {@link ValueSource} instance to group by - * @param vsContext The ValueSource context - * @param groupSort The {@link Sort} used to sort the - * groups. The top sorted document within each group - * according to groupSort, determines how that group - * sorts against other groups. This must be non-null, - * ie, if you want to groupSort by relevance use - * Sort.RELEVANCE. - * @param topNGroups How many top groups to keep. - * @throws IOException When I/O related errors occur - */ - public FunctionFirstPassGroupingCollector(ValueSource groupByVS, Map vsContext, Sort groupSort, int topNGroups) throws IOException { - super(groupSort, topNGroups); - this.groupByVS = groupByVS; - this.vsContext = vsContext; - } - - @Override - protected MutableValue getDocGroupValue(int doc) throws IOException { - filler.fillValue(doc); - return mval; - } - - @Override - protected MutableValue copyDocGroupValue(MutableValue groupValue, MutableValue reuse) { - if (reuse != null) { - reuse.copy(groupValue); - return reuse; - } - return groupValue.duplicate(); - } - - @Override - protected void doSetNextReader(LeafReaderContext readerContext) throws IOException { - super.doSetNextReader(readerContext); - FunctionValues values = groupByVS.getValues(vsContext, readerContext); - filler = values.getValueFiller(); - mval = filler.getValue(); - } - -} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java deleted file mode 100644 index 5204dc26657..00000000000 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionGrouper.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.lucene.search.grouping.function; - -import java.io.IOException; -import java.util.Collection; -import java.util.Map; - -import org.apache.lucene.queries.function.ValueSource; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.grouping.AllGroupHeadsCollector; -import org.apache.lucene.search.grouping.AllGroupsCollector; -import org.apache.lucene.search.grouping.FirstPassGroupingCollector; -import org.apache.lucene.search.grouping.SecondPassGroupingCollector; -import org.apache.lucene.search.grouping.Grouper; -import org.apache.lucene.search.grouping.SearchGroup; -import org.apache.lucene.util.mutable.MutableValue; - -/** - * Collector factory for grouping by ValueSource - */ -public class FunctionGrouper extends Grouper { - - private final ValueSource valueSource; - private final Map context; - - /** - * Create a Grouper for the provided ValueSource and context - */ - public FunctionGrouper(ValueSource valueSource, Map context) { - this.valueSource = valueSource; - this.context = context; - } - - @Override - public FirstPassGroupingCollector getFirstPassCollector(Sort sort, int count) throws IOException { - return new FunctionFirstPassGroupingCollector(valueSource, context, sort, count); - } - - @Override - public AllGroupHeadsCollector getGroupHeadsCollector(Sort sort) { - return new FunctionAllGroupHeadsCollector(valueSource, context, sort); - } - - @Override - public AllGroupsCollector getAllGroupsCollector() { - return new FunctionAllGroupsCollector(valueSource, context); - } - - @Override - public SecondPassGroupingCollector getSecondPassCollector(Collection> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields) throws IOException { - return new FunctionSecondPassGroupingCollector(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields, valueSource, context); - } -} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java deleted file mode 100644 index 45f2b37887e..00000000000 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.lucene.search.grouping.function; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.queries.function.FunctionValues; -import org.apache.lucene.queries.function.ValueSource; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.grouping.SecondPassGroupingCollector; -import org.apache.lucene.search.grouping.SearchGroup; -import org.apache.lucene.util.mutable.MutableValue; -import org.apache.lucene.search.grouping.TopGroups; //javadoc - -import java.io.IOException; -import java.util.Collection; -import java.util.Map; - -/** - * Concrete implementation of {@link SecondPassGroupingCollector} that groups based on - * {@link ValueSource} instances. - * - * @lucene.experimental - */ -public class FunctionSecondPassGroupingCollector extends SecondPassGroupingCollector { - - private final ValueSource groupByVS; - private final Map vsContext; - - private FunctionValues.ValueFiller filler; - private MutableValue mval; - - /** - * Constructs a {@link FunctionSecondPassGroupingCollector} instance. - * - * @param searchGroups The {@link SearchGroup} instances collected during the first phase. - * @param groupSort The group sort - * @param withinGroupSort The sort inside a group - * @param maxDocsPerGroup The maximum number of documents to collect inside a group - * @param getScores Whether to include the scores - * @param getMaxScores Whether to include the maximum score - * @param fillSortFields Whether to fill the sort values in {@link TopGroups#withinGroupSort} - * @param groupByVS The {@link ValueSource} to group by - * @param vsContext The value source context - * @throws IOException IOException When I/O related errors occur - */ - public FunctionSecondPassGroupingCollector(Collection> searchGroups, Sort groupSort, Sort withinGroupSort, int maxDocsPerGroup, boolean getScores, boolean getMaxScores, boolean fillSortFields, ValueSource groupByVS, Map vsContext) throws IOException { - super(searchGroups, groupSort, withinGroupSort, maxDocsPerGroup, getScores, getMaxScores, fillSortFields); - this.groupByVS = groupByVS; - this.vsContext = vsContext; - } - - @Override - protected SearchGroupDocs retrieveGroup(int doc) throws IOException { - filler.fillValue(doc); - return groupMap.get(mval); - } - - @Override - protected void doSetNextReader(LeafReaderContext readerContext) throws IOException { - super.doSetNextReader(readerContext); - FunctionValues values = groupByVS.getValues(vsContext, readerContext); - filler = values.getValueFiller(); - mval = filler.getValue(); - } - -} diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java deleted file mode 100644 index 73588ce2463..00000000000 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/package-info.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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. - */ - -/** - * Support for grouping by {@link org.apache.lucene.queries.function.ValueSource}. - */ -package org.apache.lucene.search.grouping.function; diff --git a/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java b/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java index 824a98e31bf..7e3745e61f2 100644 --- a/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java +++ b/lucene/grouping/src/java/org/apache/lucene/search/grouping/package-info.java @@ -62,9 +62,9 @@ * * *

    The implementation is two-pass: the first pass ({@link - * org.apache.lucene.search.grouping.term.TermFirstPassGroupingCollector}) + * org.apache.lucene.search.grouping.FirstPassGroupingCollector}) * gathers the top groups, and the second pass ({@link - * org.apache.lucene.search.grouping.term.TermSecondPassGroupingCollector}) + * org.apache.lucene.search.grouping.SecondPassGroupingCollector}) * gathers documents within those groups. If the search is costly to * run you may want to use the {@link * org.apache.lucene.search.CachingCollector} class, which @@ -73,18 +73,17 @@ * hold all hits. Results are returned as a {@link * org.apache.lucene.search.grouping.TopGroups} instance.

    * - *

    - * This module abstracts away what defines group and how it is collected. All grouping collectors - * are abstract and have currently term based implementations. One can implement - * collectors that for example group on multiple fields. - *

    + *

    Groups are defined by {@link org.apache.lucene.search.grouping.GroupSelector} + * implementations:

    + *
      + *
    • {@link org.apache.lucene.search.grouping.TermGroupSelector} groups based on + * the value of a {@link org.apache.lucene.index.SortedDocValues} field
    • + *
    • {@link org.apache.lucene.search.grouping.ValueSourceGroupSelector} groups based on + * the value of a {@link org.apache.lucene.queries.function.ValueSource}
    • + *
    * *

    Known limitations:

    *