mirror of https://github.com/apache/lucene.git
SOLR-14545: Fix or suppress warnings in apache/solr/update
This commit is contained in:
parent
8663a425a8
commit
4dcdd750f2
|
@ -312,6 +312,8 @@ Other Changes
|
|||
|
||||
* SOLR-14543: Fix or suppress warnings in apache/solr/search (Erick Erickson)
|
||||
|
||||
* SOLR-14545: Fix or suppress warnings in apache/solr/update (Erick Erickson)
|
||||
|
||||
================== 8.5.2 ==================
|
||||
|
||||
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
|
||||
|
|
|
@ -61,7 +61,7 @@ public class OpenNLPLangDetectUpdateProcessorFactory extends UpdateRequestProces
|
|||
private SolrResourceLoader solrResourceLoader;
|
||||
|
||||
@Override
|
||||
public void init( NamedList args )
|
||||
public void init( @SuppressWarnings({"rawtypes"})NamedList args )
|
||||
{
|
||||
if (args != null) {
|
||||
Object o;
|
||||
|
|
|
@ -255,6 +255,7 @@ public class AddUpdateCommand extends UpdateCommand {
|
|||
flattenLabelled(unwrappedDocs, ((SolrInputDocument) val));
|
||||
continue;
|
||||
}
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Collection<SolrInputDocument> childrenList = ((Collection) val);
|
||||
for (SolrInputDocument child : childrenList) {
|
||||
flattenLabelled(unwrappedDocs, child);
|
||||
|
|
|
@ -202,6 +202,7 @@ public class CdcrUpdateLog extends UpdateLog {
|
|||
* Creates a new {@link org.apache.solr.update.CdcrUpdateLog.CdcrLogReader}
|
||||
* initialised with the current list of tlogs.
|
||||
*/
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public CdcrLogReader newLogReader() {
|
||||
return new CdcrLogReader(new ArrayList(logs), tlog);
|
||||
}
|
||||
|
@ -362,6 +363,7 @@ public class CdcrUpdateLog extends UpdateLog {
|
|||
// populate recent deleteByQuery commands
|
||||
for (int i=startingUpdates.deleteByQueryList.size()-1; i>=0; i--) {
|
||||
Update update = startingUpdates.deleteByQueryList.get(i);
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> dbq = (List<Object>) update.log.lookup(update.pointer);
|
||||
long version = (Long) dbq.get(1);
|
||||
String q = (String) dbq.get(2);
|
||||
|
@ -402,6 +404,7 @@ public class CdcrUpdateLog extends UpdateLog {
|
|||
Object o = tlogReader.next();
|
||||
if (o == null) break; // we reached the end of the tlog
|
||||
// should currently be a List<Oper,Ver,Doc/Id>
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List entry = (List) o;
|
||||
operationAndFlags = (Integer) entry.get(0);
|
||||
int oper = operationAndFlags & OPERATION_MASK;
|
||||
|
@ -729,6 +732,7 @@ public class CdcrUpdateLog extends UpdateLog {
|
|||
* Extracts the version number and converts it to its absolute form.
|
||||
*/
|
||||
private long getVersion(Object o) {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List entry = (List) o;
|
||||
// version is negative for delete, ensure that we are manipulating absolute version numbers
|
||||
return Math.abs((Long) entry.get(1));
|
||||
|
|
|
@ -63,6 +63,7 @@ public final class CommitTracker implements Runnable {
|
|||
// Not a big deal but it means we must take care of MDC logging here.
|
||||
private final ScheduledExecutorService scheduler =
|
||||
Executors.newScheduledThreadPool(1, new SolrNamedThreadFactory("commitScheduler"));
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
private ScheduledFuture pending;
|
||||
|
||||
// state
|
||||
|
|
|
@ -84,6 +84,7 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
|
|||
|
||||
private volatile Future<Boolean> cdcrBootstrapFuture;
|
||||
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
private volatile Callable cdcrBootstrapCallable;
|
||||
|
||||
@Deprecated
|
||||
|
@ -448,12 +449,13 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
|
|||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
public Callable getCdcrBootstrapCallable() {
|
||||
return cdcrBootstrapCallable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setCdcrBootstrapCallable(Callable cdcrBootstrapCallable) {
|
||||
public void setCdcrBootstrapCallable(@SuppressWarnings({"rawtypes"})Callable cdcrBootstrapCallable) {
|
||||
this.cdcrBootstrapCallable = cdcrBootstrapCallable;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -623,6 +623,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
if (cmd.expungeDeletes) expungeDeleteCommands.mark();
|
||||
}
|
||||
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Future[] waitSearcher = null;
|
||||
if (cmd.waitSearcher) {
|
||||
waitSearcher = new Future[1];
|
||||
|
|
|
@ -159,6 +159,7 @@ public class DocumentBuilder {
|
|||
// load each field value
|
||||
boolean hasField = false;
|
||||
try {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Iterator it = field.iterator();
|
||||
while (it.hasNext()) {
|
||||
Object v = it.next();
|
||||
|
|
|
@ -164,6 +164,7 @@ public class HdfsTransactionLog extends TransactionLog {
|
|||
return true;
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
private void readHeader(FastInputStream fis) throws IOException {
|
||||
// read existing header
|
||||
boolean closeFis = false;
|
||||
|
@ -488,6 +489,7 @@ public class HdfsTransactionLog extends TransactionLog {
|
|||
|
||||
long lastVersion = Long.MIN_VALUE;
|
||||
while ( (o = super.next()) != null) {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List entry = (List) o;
|
||||
long version = (Long) entry.get(UpdateLog.VERSION_IDX);
|
||||
version = Math.abs(version);
|
||||
|
|
|
@ -231,6 +231,7 @@ public class HdfsUpdateLog extends UpdateLog {
|
|||
// populate recent deleteByQuery commands
|
||||
for (int i = startingUpdates.deleteByQueryList.size() - 1; i >= 0; i--) {
|
||||
Update update = startingUpdates.deleteByQueryList.get(i);
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> dbq = (List<Object>) update.log.lookup(update.pointer);
|
||||
long version = (Long) dbq.get(1);
|
||||
String q = (String) dbq.get(2);
|
||||
|
|
|
@ -109,6 +109,7 @@ public class IndexFingerprint implements MapSerializable {
|
|||
throws IOException {
|
||||
SchemaField versionField = VersionInfo.getAndCheckVersionField(searcher.getSchema());
|
||||
ValueSource vs = versionField.getType().getValueSource(versionField, null);
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Map funcContext = ValueSource.newContext(searcher);
|
||||
vs.createWeight(funcContext, searcher);
|
||||
|
||||
|
@ -184,7 +185,7 @@ public class IndexFingerprint implements MapSerializable {
|
|||
return map;
|
||||
}
|
||||
|
||||
private static long getLong(Map m, String key, long def) {
|
||||
private static long getLong(@SuppressWarnings({"rawtypes"})Map m, String key, long def) {
|
||||
Object oval = m.get(key);
|
||||
return oval != null ? ((Number)oval).longValue() : def;
|
||||
}
|
||||
|
@ -194,6 +195,7 @@ public class IndexFingerprint implements MapSerializable {
|
|||
*/
|
||||
public static IndexFingerprint fromObject(Object o) {
|
||||
if (o instanceof IndexFingerprint) return (IndexFingerprint) o;
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Map map = null;
|
||||
if (o instanceof Map) {
|
||||
map = (Map) o;
|
||||
|
|
|
@ -216,6 +216,7 @@ public class PeerSync implements SolrMetricProducer {
|
|||
ShardResponse srsp = shardHandler.takeCompletedOrError();
|
||||
if (srsp == null) break;
|
||||
if (srsp.getException() == null) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Long> otherVersions = (List<Long>)srsp.getSolrResponse().getResponse().get("versions");
|
||||
if (otherVersions != null && !otherVersions.isEmpty()) {
|
||||
syncErrors.inc();
|
||||
|
@ -432,6 +433,7 @@ public class PeerSync implements SolrMetricProducer {
|
|||
|
||||
private boolean handleVersions(ShardResponse srsp) {
|
||||
// we retrieved the last N updates from the replica
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Long> otherVersions = (List<Long>)srsp.getSolrResponse().getResponse().get("versions");
|
||||
// TODO: how to handle short lists?
|
||||
|
||||
|
@ -516,6 +518,7 @@ public class PeerSync implements SolrMetricProducer {
|
|||
|
||||
private boolean handleUpdates(ShardResponse srsp) {
|
||||
// we retrieved the last N updates from the replica
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> updates = (List<Object>)srsp.getSolrResponse().getResponse().get("updates");
|
||||
|
||||
SyncShardRequest sreq = (SyncShardRequest) srsp.getShardRequest();
|
||||
|
@ -580,7 +583,9 @@ public class PeerSync implements SolrMetricProducer {
|
|||
if (!(o1 instanceof List)) return 1;
|
||||
if (!(o2 instanceof List)) return -1;
|
||||
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List lst1 = (List) o1;
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List lst2 = (List) o2;
|
||||
|
||||
long l1 = Math.abs((Long) lst1.get(1));
|
||||
|
@ -616,6 +621,7 @@ public class PeerSync implements SolrMetricProducer {
|
|||
for (Object obj : updates) {
|
||||
// should currently be a List<Oper,Ver,Doc/Id>
|
||||
o = obj;
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> entry = (List<Object>)o;
|
||||
|
||||
if (debug) {
|
||||
|
|
|
@ -228,6 +228,7 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
|
|||
|
||||
private MissedUpdatesRequest buildMissedUpdatesRequest(NamedList<Object> rsp) {
|
||||
// we retrieved the last N updates from the replica
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Long> otherVersions = (List<Long>)rsp.get("versions");
|
||||
if (log.isInfoEnabled()) {
|
||||
log.info("{} Received {} versions from {}", msg(), otherVersions.size(), leaderUrl);
|
||||
|
@ -264,6 +265,7 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
|
|||
|
||||
private boolean handleUpdates(NamedList<Object> rsp, long numRequestedUpdates, IndexFingerprint leaderFingerprint) {
|
||||
// missed updates from leader, it does not contains updates from bufferedUpdates
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> updates = (List<Object>)rsp.get("updates");
|
||||
|
||||
if (updates.size() < numRequestedUpdates) {
|
||||
|
@ -285,6 +287,7 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
|
|||
// TODO leader should do fingerprint and retrieve recent updates version in atomic
|
||||
if (leaderFingerprint != null) {
|
||||
boolean existDBIOrDBQInTheGap = updates.stream().anyMatch(e -> {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> u = (List<Object>) e;
|
||||
long version = (Long) u.get(1);
|
||||
int oper = (Integer)u.get(0) & UpdateLog.OPERATION_MASK;
|
||||
|
@ -294,6 +297,7 @@ public class PeerSyncWithLeader implements SolrMetricProducer {
|
|||
if (!existDBIOrDBQInTheGap) {
|
||||
// it is safe to use leaderFingerprint.maxVersionEncountered as cut point now.
|
||||
updates.removeIf(e -> {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> u = (List<Object>) e;
|
||||
long version = (Long) u.get(1);
|
||||
return version > leaderFingerprint.getMaxVersionEncountered();
|
||||
|
|
|
@ -428,6 +428,7 @@ public class SolrCmdDistributor implements Closeable {
|
|||
NamedList<Object> nl = brp.processResponse(inputStream, null);
|
||||
Object hdr = nl.get("responseHeader");
|
||||
if (hdr != null && hdr instanceof NamedList) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
NamedList<Object> hdrList = (NamedList<Object>) hdr;
|
||||
Object rfObj = hdrList.get(UpdateRequest.REPFACT);
|
||||
if (rfObj != null && rfObj instanceof Integer) {
|
||||
|
|
|
@ -247,6 +247,7 @@ public class SolrIndexConfig implements MapSerializable {
|
|||
|
||||
if (mergedSegmentWarmerInfo != null) {
|
||||
// TODO: add infostream -> normal logging system (there is an issue somewhere)
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
IndexReaderWarmer warmer = core.getResourceLoader().newInstance(mergedSegmentWarmerInfo.className,
|
||||
IndexReaderWarmer.class,
|
||||
null,
|
||||
|
@ -262,7 +263,7 @@ public class SolrIndexConfig implements MapSerializable {
|
|||
* Builds a MergePolicy using the configured MergePolicyFactory
|
||||
* or if no factory is configured uses the configured mergePolicy PluginInfo.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
private MergePolicy buildMergePolicy(SolrResourceLoader resourceLoader, IndexSchema schema) {
|
||||
|
||||
final String mpfClassName;
|
||||
|
@ -285,6 +286,7 @@ public class SolrIndexConfig implements MapSerializable {
|
|||
return mpf.getMergePolicy();
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
private MergeScheduler buildMergeScheduler(SolrResourceLoader resourceLoader) {
|
||||
String msClassName = mergeSchedulerInfo == null ? SolrIndexConfig.DEFAULT_MERGE_SCHEDULER_CLASSNAME : mergeSchedulerInfo.className;
|
||||
MergeScheduler scheduler = resourceLoader.newInstance(msClassName, MergeScheduler.class);
|
||||
|
@ -293,6 +295,7 @@ public class SolrIndexConfig implements MapSerializable {
|
|||
// LUCENE-5080: these two setters are removed, so we have to invoke setMaxMergesAndThreads
|
||||
// if someone has them configured.
|
||||
if (scheduler instanceof ConcurrentMergeScheduler) {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
NamedList args = mergeSchedulerInfo.initArgs.clone();
|
||||
Integer maxMergeCount = (Integer) args.remove("maxMergeCount");
|
||||
if (maxMergeCount == null) {
|
||||
|
|
|
@ -430,6 +430,7 @@ public class SolrIndexSplitter {
|
|||
}
|
||||
|
||||
private void openNewSearcher(SolrCore core) throws Exception {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Future[] waitSearcher = new Future[1];
|
||||
core.getSearcher(true, false, waitSearcher, true);
|
||||
if (waitSearcher[0] != null) {
|
||||
|
|
|
@ -266,10 +266,12 @@ public class TransactionLog implements Closeable {
|
|||
}
|
||||
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
private void readHeader(FastInputStream fis) throws IOException {
|
||||
// read existing header
|
||||
fis = fis != null ? fis : new ChannelFastInputStream(channel, 0);
|
||||
@SuppressWarnings("resource") final LogCodec codec = new LogCodec(resolver);
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Map header = (Map) codec.unmarshal(fis);
|
||||
|
||||
fis.readInt(); // skip size
|
||||
|
@ -306,10 +308,12 @@ public class TransactionLog implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
protected void writeLogHeader(LogCodec codec) throws IOException {
|
||||
long pos = fos.size();
|
||||
assert pos == 0;
|
||||
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Map header = new LinkedHashMap<String, Object>();
|
||||
header.put("SOLR_TLOG", 1); // a magic string + version number
|
||||
header.put("strings", globalStringList);
|
||||
|
@ -747,6 +751,7 @@ public class TransactionLog implements Closeable {
|
|||
|
||||
long lastVersion = Long.MIN_VALUE;
|
||||
while ((o = super.next()) != null) {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List entry = (List) o;
|
||||
long version = (Long) entry.get(UpdateLog.VERSION_IDX);
|
||||
version = Math.abs(version);
|
||||
|
|
|
@ -210,7 +210,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
|
|||
// keep track of deletes only... this is not updated on an add
|
||||
protected LinkedHashMap<BytesRef, LogPtr> oldDeletes = new LinkedHashMap<BytesRef, LogPtr>(numDeletesToKeep) {
|
||||
@Override
|
||||
protected boolean removeEldestEntry(Map.Entry eldest) {
|
||||
protected boolean removeEldestEntry(@SuppressWarnings({"rawtypes"})Map.Entry eldest) {
|
||||
return size() > numDeletesToKeep;
|
||||
}
|
||||
};
|
||||
|
@ -420,6 +420,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
|
|||
// populate recent deleteByQuery commands
|
||||
for (int i = startingUpdates.deleteByQueryList.size() - 1; i >= 0; i--) {
|
||||
Update update = startingUpdates.deleteByQueryList.get(i);
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> dbq = (List<Object>) update.log.lookup(update.pointer);
|
||||
long version = (Long) dbq.get(1);
|
||||
String q = (String) dbq.get(2);
|
||||
|
@ -894,14 +895,16 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
|
|||
* @return Returns 0 if a full document was found in the log, -1 if no full document was found. If full document was supposed
|
||||
* to be found in the tlogs, but couldn't be found (because the logs were rotated) then the prevPointer is returned.
|
||||
*/
|
||||
@SuppressWarnings({"unchecked"})
|
||||
synchronized public long applyPartialUpdates(BytesRef id, long prevPointer, long prevVersion,
|
||||
Set<String> onlyTheseFields, SolrDocumentBase latestPartialDoc) {
|
||||
Set<String> onlyTheseFields, @SuppressWarnings({"rawtypes"})SolrDocumentBase latestPartialDoc) {
|
||||
|
||||
SolrInputDocument partialUpdateDoc = null;
|
||||
|
||||
List<TransactionLog> lookupLogs = Arrays.asList(tlog, prevMapLog, prevMapLog2);
|
||||
while (prevPointer >= 0) {
|
||||
//go through each partial update and apply it on the incoming doc one after another
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List entry;
|
||||
entry = getEntryFromTLog(prevPointer, prevVersion, lookupLogs);
|
||||
if (entry == null) {
|
||||
|
@ -942,7 +945,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
|
|||
/**
|
||||
* Add all fields from olderDoc into newerDoc if not already present in newerDoc
|
||||
*/
|
||||
private void applyOlderUpdates(SolrDocumentBase newerDoc, SolrInputDocument olderDoc, Set<String> mergeFields) {
|
||||
private void applyOlderUpdates(@SuppressWarnings({"rawtypes"})SolrDocumentBase newerDoc, SolrInputDocument olderDoc, Set<String> mergeFields) {
|
||||
for (String fieldName : olderDoc.getFieldNames()) {
|
||||
// if the newerDoc has this field, then this field from olderDoc can be ignored
|
||||
if (!newerDoc.containsKey(fieldName) && (mergeFields == null || mergeFields.contains(fieldName))) {
|
||||
|
@ -959,6 +962,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
|
|||
*
|
||||
* @return The entry if found, otherwise null
|
||||
*/
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
private synchronized List getEntryFromTLog(long lookupPointer, long lookupVersion, List<TransactionLog> lookupLogs) {
|
||||
for (TransactionLog lookupLog : lookupLogs) {
|
||||
if (lookupLog != null && lookupLog.getLogSize() > lookupPointer) {
|
||||
|
@ -1255,6 +1259,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
|
|||
try {
|
||||
while ( (o = logReader.next()) != null ) {
|
||||
try {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List entry = (List)o;
|
||||
int operationAndFlags = (Integer) entry.get(0);
|
||||
int oper = operationAndFlags & OPERATION_MASK;
|
||||
|
@ -1491,6 +1496,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
|
|||
if (o==null) break;
|
||||
|
||||
// should currently be a List<Oper,Ver,Doc/Id>
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List entry = (List)o;
|
||||
|
||||
// TODO: refactor this out so we get common error handling
|
||||
|
@ -1875,6 +1881,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
|
|||
try {
|
||||
|
||||
// should currently be a List<Oper,Ver,Doc/Id>
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List entry = (List) o;
|
||||
operationAndFlags = (Integer) entry.get(UpdateLog.FLAGS_IDX);
|
||||
int oper = operationAndFlags & OPERATION_MASK;
|
||||
|
@ -2082,7 +2089,8 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
|
|||
* if it is UPDATE_INPLACE then the previous version will also be read from the entry
|
||||
* @param version Version already obtained from the entry.
|
||||
*/
|
||||
public static AddUpdateCommand convertTlogEntryToAddUpdateCommand(SolrQueryRequest req, List entry,
|
||||
public static AddUpdateCommand convertTlogEntryToAddUpdateCommand(SolrQueryRequest req,
|
||||
@SuppressWarnings({"rawtypes"})List entry,
|
||||
int operation, long version) {
|
||||
assert operation == UpdateLog.ADD || operation == UpdateLog.UPDATE_INPLACE;
|
||||
SolrInputDocument sdoc = (SolrInputDocument) entry.get(entry.size()-1);
|
||||
|
|
|
@ -219,6 +219,7 @@ public class VersionInfo {
|
|||
if (lookup < 0) return null; // this means the doc doesn't exist in the index yet
|
||||
|
||||
ValueSource vs = versionField.getType().getValueSource(versionField, null);
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Map context = ValueSource.newContext(searcher);
|
||||
vs.createWeight(context, searcher);
|
||||
FunctionValues fv = vs.getValues(context, searcher.getTopReaderContext().leaves().get((int) (lookup >> 32)));
|
||||
|
@ -254,6 +255,7 @@ public class VersionInfo {
|
|||
|
||||
long maxVersionInIndex = 0L;
|
||||
ValueSource vs = versionField.getType().getValueSource(versionField, null);
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Map funcContext = ValueSource.newContext(searcher);
|
||||
vs.createWeight(funcContext, searcher);
|
||||
// TODO: multi-thread this
|
||||
|
|
|
@ -46,7 +46,7 @@ public abstract class AbstractDefaultValueUpdateProcessorFactory
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
Object obj = args.remove("fieldName");
|
||||
if (null == obj && null == fieldName) {
|
||||
|
|
|
@ -158,7 +158,7 @@ public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcesso
|
|||
}
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
inclusions = FieldMutatingUpdateProcessorFactory.parseSelectorParams(args);
|
||||
validateSelectorParams(inclusions);
|
||||
inclusions.fieldNameMatchesSchemaField = false; // Explicitly (non-configurably) require unknown field names
|
||||
|
@ -192,8 +192,9 @@ public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcesso
|
|||
}
|
||||
}
|
||||
|
||||
private static List<TypeMapping> parseTypeMappings(NamedList args) {
|
||||
private static List<TypeMapping> parseTypeMappings(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
List<TypeMapping> typeMappings = new ArrayList<>();
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> typeMappingsParams = args.getAll(TYPE_MAPPING_PARAM);
|
||||
for (Object typeMappingObj : typeMappingsParams) {
|
||||
if (null == typeMappingObj) {
|
||||
|
@ -202,6 +203,7 @@ public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcesso
|
|||
if ( ! (typeMappingObj instanceof NamedList) ) {
|
||||
throw new SolrException(SERVER_ERROR, "'" + TYPE_MAPPING_PARAM + "' init param must be a <lst>");
|
||||
}
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
NamedList typeMappingNamedList = (NamedList)typeMappingObj;
|
||||
|
||||
Object fieldTypeObj = typeMappingNamedList.remove(FIELD_TYPE_PARAM);
|
||||
|
@ -218,6 +220,7 @@ public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcesso
|
|||
}
|
||||
String fieldType = fieldTypeObj.toString();
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Collection<String> valueClasses
|
||||
= typeMappingNamedList.removeConfigArgs(VALUE_CLASS_PARAM);
|
||||
if (valueClasses.isEmpty()) {
|
||||
|
@ -245,6 +248,7 @@ public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcesso
|
|||
if ( ! (copyFieldObj instanceof NamedList)) {
|
||||
throw new SolrException(SERVER_ERROR, "'" + COPY_FIELD_PARAM + "' init param must be a <lst>");
|
||||
}
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
NamedList copyFieldNamedList = (NamedList)copyFieldObj;
|
||||
// dest
|
||||
Object destObj = copyFieldNamedList.remove(DEST_PARAM);
|
||||
|
@ -556,6 +560,7 @@ public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcesso
|
|||
}
|
||||
|
||||
private boolean isImmutableConfigSet(SolrCore core) {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
NamedList args = core.getConfigSetProperties();
|
||||
Object immutable = args != null ? args.get(IMMUTABLE_CONFIGSET_ARG) : null;
|
||||
return immutable != null ? Boolean.parseBoolean(immutable.toString()) : false;
|
||||
|
|
|
@ -97,6 +97,7 @@ public class AtomicUpdateDocumentMerger {
|
|||
* @param toDoc the final SolrInputDocument that will be mutated with the values from the fromDoc atomic commands
|
||||
* @return toDoc with mutated values
|
||||
*/
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public SolrInputDocument merge(final SolrInputDocument fromDoc, SolrInputDocument toDoc) {
|
||||
for (SolrInputField sif : fromDoc.values()) {
|
||||
Object val = sif.getValue();
|
||||
|
@ -164,6 +165,7 @@ public class AtomicUpdateDocumentMerger {
|
|||
*
|
||||
* @return Return a set of fields that can be in-place updated.
|
||||
*/
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public static Set<String> computeInPlaceUpdatableFields(AddUpdateCommand cmd) throws IOException {
|
||||
SolrInputDocument sdoc = cmd.getSolrInputDocument();
|
||||
IndexSchema schema = cmd.getReq().getSchema();
|
||||
|
@ -415,7 +417,9 @@ public class AtomicUpdateDocumentMerger {
|
|||
* @param updateDoc the document that was sent as part of the Add Update Command
|
||||
* @return updated SolrInputDocument
|
||||
*/
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public SolrInputDocument updateDocInSif(SolrInputField updateSif, SolrInputDocument cmdDocWChildren, SolrInputDocument updateDoc) {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List sifToReplaceValues = (List) updateSif.getValues();
|
||||
final boolean wasList = updateSif.getValue() instanceof Collection;
|
||||
int index = getDocIndexFromCollection(cmdDocWChildren, sifToReplaceValues);
|
||||
|
@ -511,6 +515,7 @@ public class AtomicUpdateDocumentMerger {
|
|||
final String name = sif.getName();
|
||||
SolrInputField existingField = toDoc.get(name);
|
||||
if (existingField == null) return;
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
final Collection original = existingField.getValues();
|
||||
if (fieldVal instanceof Collection) {
|
||||
for (Object object : (Collection) fieldVal) {
|
||||
|
@ -546,6 +551,7 @@ public class AtomicUpdateDocumentMerger {
|
|||
private Collection<Pattern> preparePatterns(Object fieldVal) {
|
||||
final Collection<Pattern> patterns = new LinkedHashSet<>(1);
|
||||
if (fieldVal instanceof Collection) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Collection<Object> patternVals = (Collection<Object>) fieldVal;
|
||||
for (Object patternVal : patternVals) {
|
||||
patterns.add(Pattern.compile(patternVal.toString()));
|
||||
|
@ -568,6 +574,7 @@ public class AtomicUpdateDocumentMerger {
|
|||
if(!(obj instanceof Collection)) {
|
||||
return obj instanceof SolrDocumentBase;
|
||||
}
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Collection objValues = (Collection) obj;
|
||||
if(objValues.size() == 0) {
|
||||
return false;
|
||||
|
@ -575,7 +582,7 @@ public class AtomicUpdateDocumentMerger {
|
|||
return objValues.iterator().next() instanceof SolrDocumentBase;
|
||||
}
|
||||
|
||||
private void removeObj(Collection original, Object toRemove, String fieldName) {
|
||||
private void removeObj(@SuppressWarnings({"rawtypes"})Collection original, Object toRemove, String fieldName) {
|
||||
if(isChildDoc(toRemove)) {
|
||||
removeChildDoc(original, (SolrInputDocument) toRemove);
|
||||
} else {
|
||||
|
@ -583,7 +590,8 @@ public class AtomicUpdateDocumentMerger {
|
|||
}
|
||||
}
|
||||
|
||||
private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
private static void removeChildDoc(@SuppressWarnings({"rawtypes"})Collection original, SolrInputDocument docToRemove) {
|
||||
for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
|
||||
if(isDerivedFromDoc(doc, docToRemove)) {
|
||||
original.remove(doc);
|
||||
|
|
|
@ -31,7 +31,7 @@ public class CdcrUpdateProcessorFactory
|
|||
implements DistributingUpdateProcessorFactory {
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -63,7 +63,7 @@ public class ClassificationUpdateProcessorFactory extends UpdateRequestProcessor
|
|||
private ClassificationUpdateProcessorParams classificationParams;
|
||||
|
||||
@Override
|
||||
public void init(final NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})final NamedList args) {
|
||||
if (args != null) {
|
||||
params = args.toSolrParams();
|
||||
classificationParams = new ClassificationUpdateProcessorParams();
|
||||
|
|
|
@ -198,7 +198,7 @@ public class CloneFieldUpdateProcessorFactory
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
// high level (loose) check for which type of config we have.
|
||||
//
|
||||
|
@ -283,7 +283,7 @@ public class CloneFieldUpdateProcessorFactory
|
|||
* "source" and "dest" init params <em>do</em> exist.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
private void initSourceSelectorSyntax(NamedList args) {
|
||||
private void initSourceSelectorSyntax(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
// Full and complete syntax where source and dest are mandatory.
|
||||
//
|
||||
// source may be a single string or a selector.
|
||||
|
@ -307,6 +307,7 @@ public class CloneFieldUpdateProcessorFactory
|
|||
if (1 == sources.size()) {
|
||||
if (sources.get(0) instanceof NamedList) {
|
||||
// nested set of selector options
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
NamedList selectorConfig = (NamedList) args.remove(SOURCE_PARAM);
|
||||
|
||||
srcInclusions = parseSelectorParams(selectorConfig);
|
||||
|
@ -322,6 +323,7 @@ public class CloneFieldUpdateProcessorFactory
|
|||
throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM +
|
||||
"' child 'exclude' must be <lst/>");
|
||||
}
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
NamedList exc = (NamedList) excObj;
|
||||
srcExclusions.add(parseSelectorParams(exc));
|
||||
if (0 < exc.size()) {
|
||||
|
@ -353,6 +355,7 @@ public class CloneFieldUpdateProcessorFactory
|
|||
}
|
||||
|
||||
if (d instanceof NamedList) {
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
NamedList destList = (NamedList) d;
|
||||
|
||||
Object patt = destList.remove(PATTERN_PARAM);
|
||||
|
@ -469,7 +472,7 @@ public class CloneFieldUpdateProcessorFactory
|
|||
}
|
||||
|
||||
/** macro */
|
||||
private static SelectorParams parseSelectorParams(NamedList args) {
|
||||
private static SelectorParams parseSelectorParams(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
return FieldMutatingUpdateProcessorFactory.parseSelectorParams(args);
|
||||
}
|
||||
|
||||
|
|
|
@ -67,7 +67,7 @@ public final class ConcatFieldUpdateProcessorFactory extends FieldMutatingUpdate
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
Object d = args.remove("delimiter");
|
||||
if (null != d) delimiter = d.toString();
|
||||
|
||||
|
|
|
@ -54,7 +54,7 @@ public class DefaultValueUpdateProcessorFactory
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
Object obj = args.remove("value");
|
||||
if (null == obj) {
|
||||
|
|
|
@ -325,7 +325,9 @@ public class DocBasedVersionConstraintsProcessor extends UpdateRequestProcessor
|
|||
* @param userVersionFieldName Field name of the user versions being compared
|
||||
* @return True if acceptable, false if not.
|
||||
*/
|
||||
protected boolean newUpdateComparePasses(Comparable newUserVersion, Comparable oldUserVersion, String userVersionFieldName) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
protected boolean newUpdateComparePasses(@SuppressWarnings({"rawtypes"})Comparable newUserVersion,
|
||||
@SuppressWarnings({"rawtypes"})Comparable oldUserVersion, String userVersionFieldName) {
|
||||
return oldUserVersion.compareTo(newUserVersion) < 0;
|
||||
}
|
||||
|
||||
|
@ -355,6 +357,7 @@ public class DocBasedVersionConstraintsProcessor extends UpdateRequestProcessor
|
|||
SchemaField field,
|
||||
SolrIndexSearcher searcher) throws IOException {
|
||||
ValueSource vs = field.getType().getValueSource(field, null);
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
Map context = ValueSource.newContext(searcher);
|
||||
vs.createWeight(context, searcher);
|
||||
return vs.getValues(context, segmentContext);
|
||||
|
|
|
@ -105,7 +105,7 @@ public class DocBasedVersionConstraintsProcessorFactory extends UpdateRequestPro
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init( NamedList args ) {
|
||||
public void init( @SuppressWarnings({"rawtypes"})NamedList args ) {
|
||||
|
||||
Object tmp = args.remove("versionField");
|
||||
if (null == tmp) {
|
||||
|
|
|
@ -195,7 +195,8 @@ public final class DocExpirationUpdateProcessorFactory
|
|||
private SolrException confErr(final String msg, SolrException root) {
|
||||
return new SolrException(SERVER_ERROR, this.getClass().getSimpleName()+": "+msg, root);
|
||||
}
|
||||
private String removeArgStr(final NamedList args, final String arg, final String def,
|
||||
private String removeArgStr(@SuppressWarnings({"rawtypes"})final NamedList args,
|
||||
final String arg, final String def,
|
||||
final String errMsg) {
|
||||
|
||||
if (args.indexOf(arg,0) < 0) return def;
|
||||
|
@ -210,7 +211,7 @@ public final class DocExpirationUpdateProcessorFactory
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
deleteChainName = removeArgStr(args, DEL_CHAIN_NAME_CONF, null,
|
||||
"must be a <str> or <null/> for default chain");
|
||||
|
|
|
@ -53,7 +53,7 @@ public final class FieldLengthUpdateProcessorFactory extends FieldMutatingUpdate
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
// no length specific init args
|
||||
super.init(args);
|
||||
}
|
||||
|
|
|
@ -209,12 +209,14 @@ public abstract class FieldMutatingUpdateProcessor
|
|||
private static final class ConfigurableFieldNameSelectorHelper {
|
||||
|
||||
final SelectorParams params;
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
final Collection<Class> classes;
|
||||
|
||||
private ConfigurableFieldNameSelectorHelper(final SolrResourceLoader loader,
|
||||
final SelectorParams params) {
|
||||
this.params = params;
|
||||
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
final Collection<Class> classes = new ArrayList<>(params.typeClass.size());
|
||||
|
||||
for (String t : params.typeClass) {
|
||||
|
@ -267,8 +269,9 @@ public abstract class FieldMutatingUpdateProcessor
|
|||
* returns true if the Object 'o' is an instance of any class in
|
||||
* the Collection
|
||||
*/
|
||||
private static boolean instanceOfAny(Object o, Collection<Class> classes) {
|
||||
for (Class c : classes) {
|
||||
private static boolean instanceOfAny(Object o,
|
||||
@SuppressWarnings({"rawtypes"})Collection<Class> classes) {
|
||||
for (@SuppressWarnings({"rawtypes"})Class c : classes) {
|
||||
if ( c.isInstance(o) ) return true;
|
||||
}
|
||||
return false;
|
||||
|
|
|
@ -138,7 +138,9 @@ public abstract class FieldMutatingUpdateProcessorFactory
|
|||
"selector was never initialized, inform(SolrCore) never called???");
|
||||
}
|
||||
|
||||
public static SelectorParams parseSelectorParams(NamedList args) {
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
public static SelectorParams parseSelectorParams(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
SelectorParams params = new SelectorParams();
|
||||
|
||||
params.fieldName = new HashSet<>(args.removeConfigArgs("fieldName"));
|
||||
|
@ -168,8 +170,10 @@ public abstract class FieldMutatingUpdateProcessorFactory
|
|||
return params;
|
||||
}
|
||||
|
||||
public static Collection<SelectorParams> parseSelectorExclusionParams(NamedList args) {
|
||||
public static Collection<SelectorParams> parseSelectorExclusionParams(
|
||||
@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
Collection<SelectorParams> exclusions = new ArrayList<>();
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<Object> excList = args.getAll("exclude");
|
||||
for (Object excObj : excList) {
|
||||
if (null == excObj) {
|
||||
|
@ -180,6 +184,7 @@ public abstract class FieldMutatingUpdateProcessorFactory
|
|||
throw new SolrException (SolrException.ErrorCode.SERVER_ERROR,
|
||||
"'exclude' init param must be <lst/>");
|
||||
}
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
NamedList exc = (NamedList) excObj;
|
||||
exclusions.add(parseSelectorParams(exc));
|
||||
if (0 < exc.size()) {
|
||||
|
@ -203,7 +208,7 @@ public abstract class FieldMutatingUpdateProcessorFactory
|
|||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
inclusions = parseSelectorParams(args);
|
||||
exclusions = parseSelectorExclusionParams(args);
|
||||
|
|
|
@ -83,7 +83,7 @@ public class FieldNameMutatingUpdateProcessorFactory extends UpdateRequestProce
|
|||
}
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
sourcePattern = (String) args.get("pattern");
|
||||
replacement = (String) args.get("replacement");
|
||||
if(sourcePattern ==null || replacement == null) {
|
||||
|
|
|
@ -49,7 +49,7 @@ public class IgnoreCommitOptimizeUpdateProcessorFactory extends UpdateRequestPro
|
|||
protected boolean ignoreOptimizeOnly = false; // default behavior is to ignore commits and optimize
|
||||
|
||||
@Override
|
||||
public void init(final NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})final NamedList args) {
|
||||
SolrParams params = (args != null) ? args.toSolrParams() : null;
|
||||
if (params == null) {
|
||||
errorCode = ErrorCode.FORBIDDEN; // default is 403 error
|
||||
|
|
|
@ -48,7 +48,7 @@ public class IgnoreLargeDocumentProcessorFactory extends UpdateRequestProcessorF
|
|||
private long maxDocumentSize = 1024 * 1024;
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
maxDocumentSize = args.toSolrParams().required().getLong(LIMIT_SIZE_PARAM);
|
||||
args.remove(LIMIT_SIZE_PARAM);
|
||||
|
||||
|
@ -125,6 +125,7 @@ public class IgnoreLargeDocumentProcessorFactory extends UpdateRequestProcessorF
|
|||
return size;
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
static long estimate(Object obj) {
|
||||
if (obj instanceof SolrInputDocument) {
|
||||
return estimate((SolrInputDocument) obj);
|
||||
|
@ -162,7 +163,7 @@ public class IgnoreLargeDocumentProcessorFactory extends UpdateRequestProcessorF
|
|||
return size;
|
||||
}
|
||||
|
||||
private static long estimate(Collection collection) {
|
||||
private static long estimate(@SuppressWarnings({"rawtypes"})Collection collection) {
|
||||
if (collection.isEmpty()) return 0;
|
||||
long size = 0;
|
||||
for (Object obj : collection) {
|
||||
|
|
|
@ -59,6 +59,7 @@ public final class LastFieldValueUpdateProcessorFactory extends FieldValueSubset
|
|||
|
||||
if (values instanceof List) {
|
||||
// optimize index lookup
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
List l = (List)values;
|
||||
result = l.get(l.size()-1);
|
||||
} else if (values instanceof SortedSet) {
|
||||
|
|
|
@ -52,7 +52,7 @@ public class LogUpdateProcessorFactory extends UpdateRequestProcessorFactory imp
|
|||
int maxNumToLog = 10;
|
||||
int slowUpdateThresholdMillis = -1;
|
||||
@Override
|
||||
public void init( final NamedList args ) {
|
||||
public void init( @SuppressWarnings({"rawtypes"})final NamedList args ) {
|
||||
if( args != null ) {
|
||||
SolrParams params = args.toSolrParams();
|
||||
maxNumToLog = params.getInt( "maxNumToLog", maxNumToLog );
|
||||
|
|
|
@ -55,7 +55,7 @@ import static org.apache.solr.update.processor.FieldMutatingUpdateProcessor.SELE
|
|||
public final class MaxFieldValueUpdateProcessorFactory extends FieldValueSubsetUpdateProcessorFactory {
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public Collection pickSubset(Collection values) {
|
||||
Collection result = values;
|
||||
try {
|
||||
|
|
|
@ -55,7 +55,7 @@ import static org.apache.solr.update.processor.FieldMutatingUpdateProcessor.SELE
|
|||
public final class MinFieldValueUpdateProcessorFactory extends FieldValueSubsetUpdateProcessorFactory {
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public Collection pickSubset(Collection values) {
|
||||
Collection result = values;
|
||||
try {
|
||||
|
|
|
@ -103,7 +103,7 @@ public class ParseBooleanFieldUpdateProcessorFactory extends FieldMutatingUpdate
|
|||
}
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
Object caseSensitiveParam = args.remove(CASE_SENSITIVE_PARAM);
|
||||
if (null != caseSensitiveParam) {
|
||||
if (caseSensitiveParam instanceof Boolean) {
|
||||
|
@ -113,6 +113,7 @@ public class ParseBooleanFieldUpdateProcessorFactory extends FieldMutatingUpdate
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Collection<String> trueValuesParam = args.removeConfigArgs(TRUE_VALUES_PARAM);
|
||||
if ( ! trueValuesParam.isEmpty()) {
|
||||
trueValues.clear();
|
||||
|
@ -121,6 +122,7 @@ public class ParseBooleanFieldUpdateProcessorFactory extends FieldMutatingUpdate
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Collection<String> falseValuesParam = args.removeConfigArgs(FALSE_VALUES_PARAM);
|
||||
if ( ! falseValuesParam.isEmpty()) {
|
||||
falseValues.clear();
|
||||
|
|
|
@ -166,7 +166,7 @@ public class ParseDateFieldUpdateProcessorFactory extends FieldMutatingUpdatePro
|
|||
}
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
Locale locale;
|
||||
String localeParam = (String)args.remove(LOCALE_PARAM);
|
||||
|
@ -182,6 +182,7 @@ public class ParseDateFieldUpdateProcessorFactory extends FieldMutatingUpdatePro
|
|||
defaultTimeZone = ZoneId.of(defaultTimeZoneParam.toString());
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Collection<String> formatsParam = args.removeConfigArgs(FORMATS_PARAM);
|
||||
if (null != formatsParam) {
|
||||
for (String value : formatsParam) {
|
||||
|
|
|
@ -50,7 +50,7 @@ public abstract class ParseNumericFieldUpdateProcessorFactory extends FieldMutat
|
|||
protected Locale locale = Locale.ROOT;
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
String localeParam = (String)args.remove(LOCALE_PARAM);
|
||||
if (null != localeParam) {
|
||||
locale = LocaleUtils.toLocale(localeParam);
|
||||
|
|
|
@ -107,7 +107,7 @@ public class PreAnalyzedUpdateProcessorFactory extends FieldMutatingUpdateProces
|
|||
private String parserImpl;
|
||||
|
||||
@Override
|
||||
public void init(final NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})final NamedList args) {
|
||||
parserImpl = (String)args.get("parser");
|
||||
args.remove("parser");
|
||||
// initialize inclusion / exclusion patterns
|
||||
|
|
|
@ -79,7 +79,7 @@ public final class RegexReplaceProcessorFactory extends FieldMutatingUpdateProce
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
Object patternParam = args.remove(PATTERN_PARAM);
|
||||
|
||||
|
|
|
@ -88,6 +88,7 @@ public class RegexpBoostProcessor extends UpdateRequestProcessor {
|
|||
|
||||
try {
|
||||
synchronized (sharedObjectCache) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
List<BoostEntry> cachedBoostEntries =
|
||||
(List<BoostEntry>) sharedObjectCache.get(BOOST_ENTRIES_CACHE_KEY);
|
||||
|
||||
|
|
|
@ -49,7 +49,7 @@ public final class RemoveBlankFieldUpdateProcessorFactory extends FieldMutatingU
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
// no trim specific init args
|
||||
super.init(args);
|
||||
}
|
||||
|
|
|
@ -53,7 +53,7 @@ public class SignatureUpdateProcessorFactory
|
|||
private SolrParams params;
|
||||
|
||||
@Override
|
||||
public void init(final NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})final NamedList args) {
|
||||
if (args != null) {
|
||||
SolrParams params = args.toSolrParams();
|
||||
boolean enabled = params.getBool("enabled", true);
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.solr.update.AddUpdateCommand;
|
|||
*/
|
||||
public abstract class SimpleUpdateProcessorFactory extends UpdateRequestProcessorFactory {
|
||||
private String myName; // if classname==XyzUpdateProcessorFactory myName=Xyz
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
protected NamedList initArgs = new NamedList();
|
||||
private static ThreadLocal<SolrQueryRequest> REQ = new ThreadLocal<>();
|
||||
|
||||
|
@ -39,7 +40,7 @@ public abstract class SimpleUpdateProcessorFactory extends UpdateRequestProcesso
|
|||
}
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
super.init(args);
|
||||
this.initArgs = args;
|
||||
|
||||
|
|
|
@ -97,7 +97,7 @@ public class SkipExistingDocumentsProcessorFactory extends UpdateRequestProcesso
|
|||
private boolean skipUpdateIfMissing = true;
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
Object tmp = args.remove(PARAM_SKIP_INSERT_IF_EXISTS);
|
||||
if (null != tmp) {
|
||||
if (! (tmp instanceof Boolean) ) {
|
||||
|
|
|
@ -177,8 +177,9 @@ public class StatelessScriptUpdateProcessorFactory extends UpdateRequestProcesso
|
|||
private ScriptEngineCustomizer scriptEngineCustomizer;
|
||||
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
Collection<String> scripts =
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Collection<String> scripts =
|
||||
args.removeConfigArgs(SCRIPT_ARG);
|
||||
if (scripts.isEmpty()) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
|
|
|
@ -47,7 +47,7 @@ public final class TrimFieldUpdateProcessorFactory extends FieldMutatingUpdatePr
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
// no trim specific init args
|
||||
super.init(args);
|
||||
}
|
||||
|
|
|
@ -54,7 +54,7 @@ public final class TruncateFieldUpdateProcessorFactory
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
Object lengthParam = args.remove(MAX_LENGTH_PARAM);
|
||||
if (null == lengthParam) {
|
||||
|
|
|
@ -77,7 +77,7 @@ public class UUIDUpdateProcessorFactory extends UpdateRequestProcessorFactory {
|
|||
protected String fieldName = null;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void init(NamedList args) {
|
||||
public void init(@SuppressWarnings({"rawtypes"})NamedList args) {
|
||||
|
||||
Object obj = args.remove(FIELD_PARAM);
|
||||
if (null != obj) {
|
||||
|
|
|
@ -55,7 +55,7 @@ public class UniqFieldsUpdateProcessorFactory extends FieldValueSubsetUpdateProc
|
|||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public Collection pickSubset(Collection values) {
|
||||
Set<Object> uniqs = new HashSet<>();
|
||||
List<Object> result = new ArrayList<>(values.size());
|
||||
|
|
|
@ -121,6 +121,7 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
|
|||
* @see DistributedUpdateProcessorFactory
|
||||
*/
|
||||
@Override
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
public void init(PluginInfo info) {
|
||||
final String infomsg = "updateRequestProcessorChain \"" +
|
||||
(null != info.name ? info.name : "") + "\"" +
|
||||
|
@ -173,6 +174,7 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
|
|||
|
||||
}
|
||||
|
||||
@SuppressWarnings({"rawtypes"})
|
||||
private List<UpdateRequestProcessorFactory> createProcessors(PluginInfo info) {
|
||||
List<PluginInfo> processors = info.getChildren("processor");
|
||||
return processors.stream().map(it -> {
|
||||
|
@ -275,7 +277,8 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
|
|||
return result;
|
||||
}
|
||||
|
||||
private static void insertBefore(LinkedList<UpdateRequestProcessorFactory> urps, List<UpdateRequestProcessorFactory> newFactories, Class klas, int idx) {
|
||||
private static void insertBefore(LinkedList<UpdateRequestProcessorFactory> urps, List<UpdateRequestProcessorFactory> newFactories,
|
||||
@SuppressWarnings({"rawtypes"})Class klas, int idx) {
|
||||
if (newFactories.isEmpty()) return;
|
||||
for (int i = 0; i < urps.size(); i++) {
|
||||
if (klas.isInstance(urps.get(i))) {
|
||||
|
@ -306,6 +309,7 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
|
|||
p = core.getUpdateProcessors().get(s);
|
||||
}
|
||||
if (p == null) {
|
||||
@SuppressWarnings({"unchecked"})
|
||||
Class<UpdateRequestProcessorFactory> factoryClass = implicits.get(s);
|
||||
if(factoryClass != null) {
|
||||
PluginInfo pluginInfo = new PluginInfo("updateProcessor",
|
||||
|
@ -356,7 +360,7 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
|
|||
public static class LazyUpdateProcessorFactoryHolder extends PluginBag.PluginHolder<UpdateRequestProcessorFactory> {
|
||||
private volatile UpdateRequestProcessorFactory lazyFactory;
|
||||
|
||||
public LazyUpdateProcessorFactoryHolder(final PluginBag.PluginHolder holder) {
|
||||
public LazyUpdateProcessorFactoryHolder(@SuppressWarnings({"rawtypes"})final PluginBag.PluginHolder holder) {
|
||||
super(holder.getPluginInfo());
|
||||
lazyFactory = new LazyUpdateRequestProcessorFactory(holder);
|
||||
}
|
||||
|
@ -370,6 +374,7 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
|
|||
public static class LazyUpdateRequestProcessorFactory extends UpdateRequestProcessorFactory {
|
||||
private final PluginBag.PluginHolder<UpdateRequestProcessorFactory> holder;
|
||||
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public LazyUpdateRequestProcessorFactory(PluginBag.PluginHolder holder) {
|
||||
this.holder = holder;
|
||||
}
|
||||
|
@ -384,6 +389,7 @@ public final class UpdateRequestProcessorChain implements PluginInfoInitialized
|
|||
}
|
||||
}
|
||||
}
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public static final Map<String, Class> implicits = new ImmutableMap.Builder()
|
||||
.put(TemplateUpdateProcessorFactory.NAME, TemplateUpdateProcessorFactory.class)
|
||||
.put(AtomicUpdateProcessorFactory.NAME, AtomicUpdateProcessorFactory.class)
|
||||
|
|
|
@ -41,7 +41,7 @@ public abstract class UpdateRequestProcessorFactory implements NamedListInitiali
|
|||
public interface RunAlways {}
|
||||
|
||||
@Override
|
||||
public void init( NamedList args )
|
||||
public void init( @SuppressWarnings({"rawtypes"})NamedList args )
|
||||
{
|
||||
// could process the Node
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue