This is an automated email from the ASF dual-hosted git repository.
magibney pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/solr.git
The following commit(s) were added to refs/heads/branch_9x by this push:
new f0be4fa2671 SOLR-16962: Restore ability to configure tlog directory
(#1895)
f0be4fa2671 is described below
commit f0be4fa2671fcdb3d3922dd70597cb88caa4e149
Author: Michael Gibney <[email protected]>
AuthorDate: Thu Jun 13 12:15:24 2024 -0400
SOLR-16962: Restore ability to configure tlog directory (#1895)
Includes substantial refactoring and clarification of logic and definitions
around `ulog` and `tlog`. Refactoring largely focuses on UpdateLog and
HdfsUpdateLog. There is no anticipated change in existing behavior, other than
that the `dir` parameter to the `<updateLog>` element in `solrconfig.xml`,
which has long been silently ignored, will now be respected
(cherry picked from commit 6e0e6a571e13c7e6725c60a33869ccc4819a672c)
---
solr/CHANGES.txt | 5 +
.../java/org/apache/solr/cloud/ZkController.java | 2 +-
.../solr/cloud/api/collections/MoveReplicaCmd.java | 3 +-
.../src/java/org/apache/solr/core/SolrCore.java | 53 +++-
.../java/org/apache/solr/handler/IndexFetcher.java | 2 +-
.../apache/solr/handler/ReplicationHandler.java | 2 +-
.../apache/solr/update/DirectUpdateHandler2.java | 16 +-
.../java/org/apache/solr/update/UpdateHandler.java | 65 ++++-
.../src/java/org/apache/solr/update/UpdateLog.java | 320 +++++++++++++++------
.../solr/configsets/cloud-hdfs/conf/solrconfig.xml | 4 +-
.../org/apache/solr/cloud/TestCloudRecovery.java | 2 +-
.../org/apache/solr/cloud/TestPullReplica.java | 29 +-
.../org/apache/solr/cloud/TestTlogReplica.java | 8 +-
.../test/org/apache/solr/search/TestRecovery.java | 10 +-
.../org/apache/solr/update/CustomTLogDirTest.java | 172 +++++++++++
.../org/apache/solr/hdfs/HdfsDirectoryFactory.java | 19 +-
.../org/apache/solr/hdfs/update/HdfsUpdateLog.java | 188 +++++-------
.../apache/solr/hdfs/search/TestRecoveryHdfs.java | 14 +-
.../client/solrj/request/CoreAdminRequest.java | 6 +-
.../solr/client/solrj/request/TestCoreAdmin.java | 2 +-
20 files changed, 650 insertions(+), 272 deletions(-)
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 10537782b70..1389512cd78 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -62,6 +62,11 @@ Bug Fixes
* SOLR-17326: Generated v2 'SolrRequest' implementations now all serialize to
the correct response type (Christos Malliaridis via Jason Gerlowski)
+* SOLR-16962: Restore ability to configure tlog directory. Changes largely
focus on UpdateLog and
+ HdfsUpdateLog. There is no anticipated change in existing behavior, other
than that the `dir` parameter
+ to the `<updateLog>` element in `solrconfig.xml`, which has long been
silently ignored, will now be
+ respected (Michael Gibney, David Smiley)
+
Dependency Upgrades
---------------------
(No changes)
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 d982df85e5d..8c21bb3ebdb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1713,7 +1713,7 @@ public class ZkController implements Closeable {
props.put("dataDir", core.getDataDir());
UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
if (ulog != null) {
- props.put("ulogDir", ulog.getLogDir());
+ props.put("ulogDir", ulog.getUlogDir());
}
});
}
diff --git
a/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
b/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
index 8eb92aba7b7..cd5097993b4 100644
---
a/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
+++
b/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
@@ -46,7 +46,6 @@ import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
-import org.apache.solr.update.UpdateLog;
import org.apache.solr.util.TimeOut;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -261,7 +260,7 @@ public class MoveReplicaCmd implements
CollApiCmds.CollectionApiCommand {
SKIP_CREATE_REPLICA_IN_CLUSTER_STATE,
skipCreateReplicaInClusterState,
CoreAdminParams.ULOG_DIR,
- ulogDir.substring(0, ulogDir.lastIndexOf(UpdateLog.TLOG_NAME)),
+ ulogDir,
CoreAdminParams.DATA_DIR,
dataDir,
ZkStateReader.REPLICA_TYPE,
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 01dab69ce97..35efde508f7 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -159,6 +159,7 @@ import org.apache.solr.update.SolrCoreState;
import org.apache.solr.update.SolrCoreState.IndexWriterCloser;
import org.apache.solr.update.SolrIndexWriter;
import org.apache.solr.update.UpdateHandler;
+import org.apache.solr.update.UpdateLog;
import org.apache.solr.update.VersionInfo;
import org.apache.solr.update.processor.DistributedUpdateProcessorFactory;
import org.apache.solr.update.processor.LogUpdateProcessorFactory;
@@ -222,7 +223,6 @@ public class SolrCore implements SolrInfoBean, Closeable {
private volatile IndexSchema schema;
private final NamedList<?> configSetProperties;
private final String dataDir;
- private final String ulogDir;
private final UpdateHandler updateHandler;
private final SolrCoreState solrCoreState;
@@ -392,10 +392,6 @@ public class SolrCore implements SolrInfoBean, Closeable {
return dataDir;
}
- public String getUlogDir() {
- return ulogDir;
- }
-
public String getIndexDir() {
synchronized (searcherLock) {
if (_searcher == null) return getNewIndexDir();
@@ -1102,7 +1098,6 @@ public class SolrCore implements SolrInfoBean, Closeable {
}
this.dataDir = initDataDir(dataDir, solrConfig, coreDescriptor);
- this.ulogDir = initUpdateLogDir(coreDescriptor);
if (log.isInfoEnabled()) {
log.info("Opening new SolrCore at [{}], dataDir=[{}]",
getInstancePath(), this.dataDir);
@@ -1525,14 +1520,6 @@ public class SolrCore implements SolrInfoBean, Closeable
{
}
}
- private String initUpdateLogDir(CoreDescriptor coreDescriptor) {
- String updateLogDir = coreDescriptor.getUlogDir();
- if (updateLogDir == null) {
- updateLogDir =
coreDescriptor.getInstanceDir().resolve(dataDir).toString();
- }
- return updateLogDir;
- }
-
/**
* Close the core, if it is still in use waits until is no longer in use.
*
@@ -3302,6 +3289,22 @@ public class SolrCore implements SolrInfoBean, Closeable
{
log.error(
"Failed to flag data dir for removal for core: {} dir: {}", name,
getDataDir(), e);
}
+ // ulogDir may be outside dataDir and instanceDir, so we have to
explicitly ensure it's
+ // removed; ulog is most closely associated with data, so we bundle it
with logic under
+ // `deleteDataDir`
+ UpdateHandler uh = getUpdateHandler();
+ UpdateLog ulog;
+ if (uh != null && (ulog = uh.getUpdateLog()) != null) {
+ try {
+ directoryFactory.remove(ulog.getTlogDir(), true);
+ } catch (Exception e) {
+ log.error(
+ "Failed to flag tlog dir for removal for core: {} dir: {}",
+ name,
+ ulog.getTlogDir(),
+ e);
+ }
+ }
}
if (deleteInstanceDir) {
addCloseHook(
@@ -3327,7 +3330,8 @@ public class SolrCore implements SolrInfoBean, Closeable {
public static void deleteUnloadedCore(
CoreDescriptor cd, boolean deleteDataDir, boolean deleteInstanceDir) {
if (deleteDataDir) {
- Path dataDir = cd.getInstanceDir().resolve(cd.getDataDir());
+ Path instanceDir = cd.getInstanceDir();
+ Path dataDir = instanceDir.resolve(cd.getDataDir());
try {
PathUtils.deleteDirectory(dataDir);
} catch (IOException e) {
@@ -3337,6 +3341,25 @@ public class SolrCore implements SolrInfoBean, Closeable
{
dataDir.toAbsolutePath(),
e);
}
+ String ulogDir = cd.getUlogDir();
+ if (ulogDir != null) {
+ Path ulogDirPath = instanceDir.resolve(ulogDir);
+ if (!ulogDirPath.startsWith(dataDir)
+ && (!deleteInstanceDir || !ulogDirPath.startsWith(instanceDir))) {
+ // external ulogDir, we have to remove it explicitly
+ try {
+ Path tlogPath =
+ UpdateLog.ulogToTlogDir(cd.getName(), ulogDirPath,
instanceDir, dataDir.toString());
+ PathUtils.deleteDirectory(tlogPath);
+ } catch (IOException e) {
+ log.error(
+ "Failed to delete external ulog dir for core: {} ulogDir: {}",
+ cd.getName(),
+ ulogDirPath,
+ e);
+ }
+ }
+ }
}
if (deleteInstanceDir) {
try {
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 eac48745c30..662a94943c9 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -1540,7 +1540,7 @@ public class IndexFetcher {
*/
private boolean copyTmpTlogFiles2Tlog(File tmpTlogDir) {
Path tlogDir =
-
FileSystems.getDefault().getPath(solrCore.getUpdateHandler().getUpdateLog().getLogDir());
+
FileSystems.getDefault().getPath(solrCore.getUpdateHandler().getUpdateLog().getTlogDir());
Path backupTlogDir =
FileSystems.getDefault()
.getPath(tlogDir.getParent().toAbsolutePath().toString(),
tmpTlogDir.getName());
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 81a64865ac0..463fc9a046b 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -1741,7 +1741,7 @@ public class ReplicationHandler extends RequestHandlerBase
@Override
protected Path initFile() {
// if it is a tlog file read from tlog directory
- return Path.of(core.getUpdateHandler().getUpdateLog().getLogDir(),
tlogFileName);
+ return Path.of(core.getUpdateHandler().getUpdateLog().getTlogDir(),
tlogFileName);
}
}
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 c50709d13a2..d730cc137fb 100644
--- a/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
+++ b/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
@@ -118,7 +118,7 @@ public class DirectUpdateHandler2 extends UpdateHandler
private static final Logger log =
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public DirectUpdateHandler2(SolrCore core) {
- super(core);
+ super(core, null, false);
solrCoreState = core.getSolrCoreState();
@@ -156,10 +156,13 @@ public class DirectUpdateHandler2 extends UpdateHandler
commitWithinSoftCommit = false;
commitTracker.setOpenSearcher(true);
}
+ if (ulog != null) {
+ initUlog(true);
+ }
}
public DirectUpdateHandler2(SolrCore core, UpdateHandler updateHandler) {
- super(core, updateHandler.getUpdateLog());
+ super(core, updateHandler.getUpdateLog(), false);
solrCoreState = core.getSolrCoreState();
UpdateHandlerInfo updateHandlerInfo =
core.getSolrConfig().getUpdateHandlerInfo();
@@ -190,11 +193,14 @@ public class DirectUpdateHandler2 extends UpdateHandler
commitWithinSoftCommit = updateHandlerInfo.commitWithinSoftCommit;
- UpdateLog existingLog = updateHandler.getUpdateLog();
- if (this.ulog != null && this.ulog == existingLog) {
+ if (ulog != null) {
// If we are reusing the existing update log, inform the log that its
update handler has
// changed. We do this as late as possible.
- this.ulog.init(this, core);
+ // TODO: not sure _why_ we "do this as late as possible". Consider
simplifying by
+ // moving `ulog.init(UpdateHandler, SolrCore)` entirely into the
`UpdateHandler` ctor,
+ // avoiding the need for the `UpdateHandler(SolrCore, UpdateLog,
boolean)` ctor
+ // (with the extra boolean `initUlog` param).
+ initUlog(ulog != updateHandler.getUpdateLog());
}
}
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 cfcd32fb4cf..185dd0475ad 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateHandler.java
@@ -106,10 +106,37 @@ public abstract class UpdateHandler implements
SolrInfoBean {
}
public UpdateHandler(SolrCore core) {
- this(core, null);
+ this(core, null, true);
}
public UpdateHandler(SolrCore core, UpdateLog updateLog) {
+ this(core, updateLog, true);
+ }
+
+ /**
+ * Subclasses should call this ctor, with `initUlog=false` and should then,
as the last action in
+ * the subclass ctor, call {@link #initUlog(boolean)}.
+ *
+ * <p>NOTE: as an abstract class, if subclasses are supposed to always call
this with
+ * `initUlog=false`, we could simply never init ulog in this method, and
avoid the extra arg. But
+ * the arg is present for 3 reasons:
+ *
+ * <ol>
+ * <li>for backward compatibility with subclasses (plugins) that may have
called {@link
+ * UpdateHandler} ctor with the assumption that {@link #ulog}
<i>will</i> be initialized
+ * <li>to force subclass implementations to be aware that they must init
{@link #ulog}
+ * <li>because it's likely that deferring ulog init until the last action
of the top-level ctor
+ * is actually unnecessary (see below)
+ * </ol>
+ *
+ * <p>As noted in a comment in {@link
DirectUpdateHandler2#DirectUpdateHandler2(SolrCore,
+ * UpdateHandler)}, it's unclear why we are advised to defer ulog init until
the last action of
+ * the top-level ctor, as opposed to simply delegating init to the
base-class {@link
+ * UpdateHandler} ctor. If we were to follow this approach, this "extra-arg"
ctor could be removed
+ * in favor of {@link #UpdateHandler(SolrCore, UpdateLog)}, initializing any
non-null {@link
+ * #ulog} (and removing the {@link #initUlog(boolean)} helper method as
well).
+ */
+ public UpdateHandler(SolrCore core, UpdateLog updateLog, boolean initUlog) {
this.core = core;
idField = core.getLatestSchema().getUniqueKeyField();
idFieldType = idField != null ? idField.getType() : null;
@@ -122,7 +149,7 @@ public abstract class UpdateHandler implements SolrInfoBean
{
&&
!core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
if (updateLog == null
&& ulogPluginInfo != null
- && ulogPluginInfo.isEnabled()
+ && ulogPluginInfo.isEnabled() // useless; `getPluginInfo()` returns
null if !enabled
&& !skipUpdateLog) {
DirectoryFactory dirFactory = core.getDirectoryFactory();
@@ -134,17 +161,41 @@ public abstract class UpdateHandler implements
SolrInfoBean {
? dirFactory.newDefaultUpdateLog()
: core.getResourceLoader().newInstance(ulogPluginInfo,
UpdateLog.class, true);
- if (!core.isReloaded() && !dirFactory.isPersistent()) {
- ulog.clearLog(core, ulogPluginInfo);
- }
-
if (log.isInfoEnabled()) {
log.info("Using UpdateLog implementation: {}",
ulog.getClass().getName());
}
ulog.init(ulogPluginInfo);
- ulog.init(this, core);
+
+ if (initUlog) {
+ initUlog(true);
+ }
} else {
ulog = updateLog;
+ if (updateLog != null && initUlog) {
+ initUlog(false);
+ }
+ }
+ }
+
+ /**
+ * Helper method to init {@link #ulog}. As discussed in the javadocs for
{@link
+ * #UpdateHandler(SolrCore, UpdateLog, boolean)}, this should be called as
the last action of each
+ * top-level ctor.
+ *
+ * @param closeOnError if the calling context is responsible for creating
{@link #ulog}, then we
+ * should respond to an init failure by closing {@link #ulog}, and this
param should be set to
+ * <code>true</code>. If the calling context is <i>not</i> responsible
for creating {@link
+ * #ulog}, then references exist elsewhere and we should not close on
init error (set this
+ * param to <code>false</code>).
+ */
+ protected final void initUlog(boolean closeOnError) {
+ try {
+ ulog.init(this, core);
+ } catch (Throwable t) {
+ if (closeOnError) {
+ ulog.close(false, false);
+ }
+ throw t;
}
}
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 5f2d925c14e..4ebe4f311d1 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -53,9 +53,11 @@ import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Stream;
+import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.solr.common.SolrDocumentBase;
import org.apache.solr.common.SolrException;
@@ -69,9 +71,11 @@ import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.SolrNamedThreadFactory;
import org.apache.solr.common.util.SuppressForbidden;
import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.PluginInfo;
import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrInfoBean;
+import org.apache.solr.core.SolrPaths;
import org.apache.solr.metrics.SolrMetricProducer;
import org.apache.solr.metrics.SolrMetricsContext;
import org.apache.solr.request.LocalSolrQueryRequest;
@@ -245,10 +249,10 @@ public class UpdateLog implements PluginInfoInitialized,
SolrMetricProducer {
// Needs to be String because hdfs.Path is incompatible with nio.Path
protected String[] tlogFiles;
protected Path tlogDir;
+ protected Closeable releaseTlogDir;
protected Collection<String> globalStrings;
protected String dataDir;
- protected String lastDataDir;
protected VersionInfo versionInfo;
@@ -345,6 +349,31 @@ public class UpdateLog implements PluginInfoInitialized,
SolrMetricProducer {
} else return def;
}
+ public static Path ulogToTlogDir(
+ String coreName, Path ulogDirPath, Path instancePath, String
coreDataDir) {
+ boolean unscopedDataDir =
+ !ulogDirPath.startsWith(instancePath) &&
!ulogDirPath.startsWith(coreDataDir);
+
+ // if the ulog dataDir is unscoped (neither under core instanceDir, nor
core dataDir),
+ // then we must scope it to the core; otherwise, scope to purpose
(TLOG_NAME).
+ if (unscopedDataDir) {
+ Path tlog = ulogDirPath.resolve(coreName);
+ if (tlog.equals(instancePath)) {
+ throw new IllegalArgumentException(
+ "tlog path " + tlog + " conflicts with instance path " +
instancePath);
+ } else if (SolrPaths.normalizeDir(tlog.toString()).equals(coreDataDir)) {
+ // NOTE: use string comparison above because `coreDataDir` might not
be parseable
+ // as a valid Path (e.g., it might be an hdfs Path).
+ throw new IllegalArgumentException(
+ "tlog path " + tlog + " conflicts with core data dir " +
coreDataDir);
+ }
+ return tlog;
+ } else {
+ // the simple case
+ return ulogDirPath.resolve(TLOG_NAME);
+ }
+ }
+
@Override
public void init(PluginInfo info) {
dataDir = (String) info.initArgs.get("dir");
@@ -367,32 +396,179 @@ public class UpdateLog implements PluginInfoInitialized,
SolrMetricProducer {
numVersionBuckets);
}
- /* Note, when this is called, uhandler is not completely constructed.
- * This must be called when a new log is created, or
- * for an existing log whenever the core or update handler changes.
+ private final AtomicBoolean initialized = new AtomicBoolean();
+
+ private String resolveDataDir(SolrCore core) {
+ // we need to initialize with existing `this.dataDir`, because it may have
been
+ // set in `init(PluginInfo)` to the ulogDir specified in `<updateLog>`
element
+ // of `solrconfig.xml`
+ String dataDir = this.dataDir;
+
+ // ulogDir from CoreDescriptor overrides
+ String ulogDir = core.getCoreDescriptor().getUlogDir();
+
+ if (ulogDir != null) {
+ dataDir = ulogDir;
+ }
+
+ if (dataDir == null || dataDir.length() == 0) {
+ // NOTE: this method is called from within `UpdateHandler` ctor, and
this method is called
+ // _after_ `init(PluginInfo)`; so if ulogDir is specified in
`<updateLog>` element of
+ // `solrconfig.xml`, `dataDir` will _not_ be null here.
+ dataDir = core.getDataDir();
+ }
+
+ return resolveDataDir(core, dataDir);
+ }
+
+ /**
+ * This must be called when a new log is created, or for an existing log
whenever the core or
+ * update handler changes. It is called from the ctor of the specified
{@link UpdateHandler}, so
+ * the specified uhandler will not yet be completely constructed.
+ *
+ * <p>This method must be called <i>after</i> {@link #init(PluginInfo)} is
called.
*/
public void init(UpdateHandler uhandler, SolrCore core) {
- dataDir = core.getUlogDir();
-
this.uhandler = uhandler;
- usableForChildDocs = core.getLatestSchema().isUsableForChildDocs();
-
- if (dataDir.equals(lastDataDir)) {
+ // on a reopen, return early; less work to do.
+ if (!initialized.compareAndSet(false, true)) {
+ // NOTE: we may not strictly _need_ fancy concurrency constructs here,
since it looks like
+ // `init(UpdateHandler, SolrCore` is never actually called concurrently
in application code
+ // (`TestHdfsUpdateLog.testFSThreadSafety()`, introduced by SOLR-7113,
seems to be the only
+ // place that requires true thread safety from this method?).
+ if (debug) {
+ log.debug(
+ "UpdateHandler init: tlogDir={}, next id={} this is a reopen or
double init ... nothing else to do.",
+ getTlogDir(),
+ id);
+ }
versionInfo.reload();
core.getCoreMetricManager()
.registerMetricProducer(SolrInfoBean.Category.TLOG.toString(), this);
- if (debug) {
- log.debug(
- "UpdateHandler init: tlogDir={}, next id={} this is a
reopen...nothing else to do",
- tlogDir,
- id);
+ String reResolved = resolveDataDir(core);
+ if (dataDir == null || !dataDir.equals(reResolved)) {
+ // dataDir should already be initialized, and should not change. We
recompute every time
+ // in order to fail on attempted config changes, rather than simply
silently ignore them.
+ throw new IllegalStateException(
+ "dataDir should not change on reload! computed " + reResolved + ",
found " + dataDir);
}
return;
}
- lastDataDir = dataDir;
- tlogDir = Path.of(dataDir, TLOG_NAME);
+
+ dataDir = resolveDataDir(core);
+
+ initTlogDir(core);
+
+ usableForChildDocs = core.getLatestSchema().isUsableForChildDocs();
+
+ try {
+ versionInfo = new VersionInfo(this, numVersionBuckets);
+ } catch (SolrException e) {
+ log.error("Unable to use updateLog: ", e);
+ throw new SolrException(
+ SolrException.ErrorCode.SERVER_ERROR, "Unable to use updateLog: " +
e.getMessage(), e);
+ }
+
+ // TODO: these startingVersions assume that we successfully recover from
all non-complete tlogs.
+ try (RecentUpdates startingUpdates = getRecentUpdates()) {
+ startingVersions = startingUpdates.getVersions(numRecordsToKeep);
+
+ // populate recent deletes list (since we can't get that info from the
index)
+ for (int i = startingUpdates.deleteList.size() - 1; i >= 0; i--) {
+ DeleteUpdate du = startingUpdates.deleteList.get(i);
+ oldDeletes.put(new BytesRef(du.id), new LogPtr(-1, du.version));
+ }
+
+ // 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);
+ trackDeleteByQuery(q, version);
+ }
+ }
+
core.getCoreMetricManager().registerMetricProducer(SolrInfoBean.Category.TLOG.toString(),
this);
+ }
+
+ protected final void maybeClearLog(SolrCore core) {
+ if (!core.isReloaded() && !core.getDirectoryFactory().isPersistent()) {
+ clearLog();
+ }
+ }
+
+ /**
+ * Resolves any relative path wrt the highest core-scoped level (whatever
that means for a
+ * particular implementation). For most filesystems, this will be the core
instanceDir, but there
+ * are other cases; e.g., HdfsUpdateLog will resolve paths relative to the
core dataDir.
+ *
+ * <p>If the input path is already absolute, it will be returned unmodified.
+ *
+ * <p>This method should return the final, absolute, normalized path that
defines the location of
+ * the ulog dataDir. It should <i>not</i> bother to resolve the tlog dir,
nor do any work
+ * associated with initializing the tlog dir or its contents. Tlog dir
initialization takes place
+ * later (if necessary) in {@link #initTlogDir(SolrCore)}.
+ */
+ protected String resolveDataDir(SolrCore core, String path) {
+ if (Path.of(path).isAbsolute()) {
+ return path;
+ }
+ Path instancePath = core.getInstancePath();
+ Path absolute = instancePath.resolve(path);
+ if (!absolute.normalize().startsWith(instancePath.normalize())) {
+ // relative path spec should not be able to escape core-scoped
instanceDir
+ throw new SolrException(ErrorCode.SERVER_ERROR, "Illegal relative ulog
dir spec: " + path);
+ }
+ return absolute.toString();
+ }
+
+ /**
+ * Based on {@link #dataDir} (whose initialization must be complete before
this method is called),
+ * this method is responsible for, in order:
+ *
+ * <ol>
+ * <li>resolving the tlog dir (in an implementation-dependent way)
+ * <li>clearing any existing entries (if applicable) by calling {@link
#maybeClearLog(SolrCore)}
+ * <li>actually creating tlog files (and handling any existing tlog files,
if necessary)
+ * </ol>
+ *
+ * <p>Note: implementations of this method <i>must</i> call {@link
#maybeClearLog(SolrCore)} after
+ * resolving the tlog dir, and before creating any (or handling any
existing) tlog files.
+ */
+ protected void initTlogDir(SolrCore core) {
+ Path instancePath = core.getInstancePath();
+
+ Path dataDirPath = Path.of(dataDir);
+
+ // intentionally use assert side-effect assignment here.
`Path.startsWith()` fails if the
+ // argument is a different class, which is the case for
`SolrCore.getInstancePath()`,
+ // strictly in tests (lucene test-framework `FilterPath`).
+ assert (instancePath = Path.of(instancePath.toUri())).getClass()
+ == (dataDirPath = Path.of(dataDirPath.toUri())).getClass();
+
+ tlogDir = ulogToTlogDir(core.getName(), dataDirPath, instancePath,
core.getDataDir());
+
+ maybeClearLog(core);
+
+ // usage of tlog dir almost entirely bypasses `Directory` API; we only
need to do this so that
+ // we can remove the tlog dir via `DirectoryFactory.remove()`, which
understands how to delete
+ // in filesystem-specific ways.
+ DirectoryFactory df = core.getDirectoryFactory();
+ Directory d;
+ try {
+ d =
+ df.get(
+ tlogDir.toAbsolutePath().toString(),
+ DirectoryFactory.DirContext.DEFAULT,
+ DirectoryFactory.LOCK_TYPE_NONE);
+ this.releaseTlogDir = () -> df.release(d);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+
try {
Files.createDirectories(tlogDir);
} catch (IOException e) {
@@ -441,36 +617,6 @@ public class UpdateLog implements PluginInfoInitialized,
SolrMetricProducer {
newestLogsOnStartup.addFirst(ll);
if (newestLogsOnStartup.size() >= 2) break;
}
-
- try {
- versionInfo = new VersionInfo(this, numVersionBuckets);
- } catch (SolrException e) {
- log.error("Unable to use updateLog: ", e);
- throw new SolrException(
- SolrException.ErrorCode.SERVER_ERROR, "Unable to use updateLog: " +
e.getMessage(), e);
- }
-
- // TODO: these startingVersions assume that we successfully recover from
all non-complete tlogs.
- try (RecentUpdates startingUpdates = getRecentUpdates()) {
- startingVersions = startingUpdates.getVersions(numRecordsToKeep);
-
- // populate recent deletes list (since we can't get that info from the
index)
- for (int i = startingUpdates.deleteList.size() - 1; i >= 0; i--) {
- DeleteUpdate du = startingUpdates.deleteList.get(i);
- oldDeletes.put(new BytesRef(du.id), new LogPtr(-1, du.version));
- }
-
- // 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);
- trackDeleteByQuery(q, version);
- }
- }
-
core.getCoreMetricManager().registerMetricProducer(SolrInfoBean.Category.TLOG.toString(),
this);
}
@Override
@@ -521,10 +667,14 @@ public class UpdateLog implements PluginInfoInitialized,
SolrMetricProducer {
return new TransactionLog(tlogFile, globalStrings, openExisting);
}
- public String getLogDir() {
+ public String getTlogDir() {
return tlogDir.toAbsolutePath().toString();
}
+ public String getUlogDir() {
+ return dataDir;
+ }
+
public List<Long> getStartingVersions() {
return startingVersions;
}
@@ -1463,36 +1613,41 @@ public class UpdateLog implements
PluginInfoInitialized, SolrMetricProducer {
close(committed, false);
}
+ @SuppressWarnings("try")
public void close(boolean committed, boolean deleteOnClose) {
- recoveryExecutor.shutdown(); // no new tasks
+ try (Closeable c = releaseTlogDir) {
+ recoveryExecutor.shutdown(); // no new tasks
- synchronized (this) {
+ synchronized (this) {
- // Don't delete the old tlogs, we want to be able to replay from them
and retrieve old
- // versions
+ // Don't delete the old tlogs, we want to be able to replay from them
and retrieve old
+ // versions
- doClose(prevTlog, committed);
- doClose(tlog, committed);
+ doClose(prevTlog, committed);
+ doClose(tlog, committed);
- for (TransactionLog log : logs) {
- if (log == prevTlog || log == tlog) continue;
- log.deleteOnClose = false;
- log.decref();
- log.forceClose();
- }
+ for (TransactionLog log : logs) {
+ if (log == prevTlog || log == tlog) continue;
+ log.deleteOnClose = false;
+ log.decref();
+ log.forceClose();
+ }
- if (bufferTlog != null) {
- // should not delete bufferTlog on close, existing bufferTlog is a
sign for skip peerSync
- bufferTlog.deleteOnClose = false;
- bufferTlog.decref();
- bufferTlog.forceClose();
+ if (bufferTlog != null) {
+ // should not delete bufferTlog on close, existing bufferTlog is a
sign for skip peerSync
+ bufferTlog.deleteOnClose = false;
+ bufferTlog.decref();
+ bufferTlog.forceClose();
+ }
}
- }
- try {
- ExecutorUtil.shutdownAndAwaitTermination(recoveryExecutor);
- } catch (Exception e) {
- log.error("Exception shutting down recoveryExecutor", e);
+ try {
+ ExecutorUtil.shutdownAndAwaitTermination(recoveryExecutor);
+ } catch (Exception e) {
+ log.error("Exception shutting down recoveryExecutor", e);
+ }
+ } catch (IOException e) {
+ log.warn("exception releasing tlog dir", e);
}
}
@@ -2282,30 +2437,9 @@ public class UpdateLog implements PluginInfoInitialized,
SolrMetricProducer {
}
}
- protected String getTlogDir(SolrCore core, PluginInfo info) {
- String dataDir = (String) info.initArgs.get("dir");
-
- String ulogDir = core.getCoreDescriptor().getUlogDir();
- if (ulogDir != null) {
- dataDir = ulogDir;
- }
-
- if (dataDir == null || dataDir.length() == 0) {
- dataDir = core.getDataDir();
- }
-
- return dataDir + "/" + TLOG_NAME;
- }
-
- /**
- * Clears the logs on the file system. Only call before init.
- *
- * @param core the SolrCore
- * @param ulogPluginInfo the init info for the UpdateHandler
- */
- public void clearLog(SolrCore core, PluginInfo ulogPluginInfo) {
- if (ulogPluginInfo == null) return;
- Path tlogPath = Path.of(getTlogDir(core, ulogPluginInfo));
+ /** Clears the logs on the file system. Only call before init. */
+ public void clearLog() {
+ Path tlogPath = tlogDir;
if (Files.exists(tlogPath)) {
try (Stream<Path> paths = Files.walk(tlogPath)) {
paths
diff --git
a/solr/core/src/test-files/solr/configsets/cloud-hdfs/conf/solrconfig.xml
b/solr/core/src/test-files/solr/configsets/cloud-hdfs/conf/solrconfig.xml
index 3e54f30a902..39edc032c8b 100644
--- a/solr/core/src/test-files/solr/configsets/cloud-hdfs/conf/solrconfig.xml
+++ b/solr/core/src/test-files/solr/configsets/cloud-hdfs/conf/solrconfig.xml
@@ -35,7 +35,9 @@
<commitWithin>
<softCommit>${solr.commitwithin.softcommit:true}</softCommit>
</commitWithin>
- <updateLog></updateLog>
+ <updateLog>
+ <str name="dir">${solr.ulog.dir:}</str>
+ </updateLog>
</updateHandler>
<requestHandler name="/select" class="solr.SearchHandler">
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 c21777686a9..150ab7b5d27 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery.java
@@ -188,7 +188,7 @@ public class TestCloudRecovery extends SolrCloudTestCase {
Map<String, byte[]> contentFiles = new HashMap<>();
for (JettySolrRunner solrRunner : cluster.getJettySolrRunners()) {
for (SolrCore solrCore : solrRunner.getCoreContainer().getCores()) {
- File tlogFolder = new File(solrCore.getUlogDir(), UpdateLog.TLOG_NAME);
+ File tlogFolder = new
File(solrCore.getUpdateHandler().getUpdateLog().getTlogDir());
String[] tLogFiles = tlogFolder.list();
Arrays.sort(tLogFiles);
String lastTLogFile = tlogFolder.getAbsolutePath() + "/" +
tLogFiles[tLogFiles.length - 1];
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
index 7b0ae43ab3b..f5cf82c701e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
@@ -17,6 +17,7 @@
package org.apache.solr.cloud;
import com.carrotsearch.randomizedtesting.annotations.Repeat;
+import java.io.File;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
@@ -52,8 +53,10 @@ 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.TimeSource;
+import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrCore;
import org.apache.solr.embedded.JettySolrRunner;
+import org.apache.solr.update.UpdateLog;
import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
@@ -220,6 +223,27 @@ public class TestPullReplica extends SolrCloudTestCase {
}
}
+ /**
+ * For some tests (when we want to check for <i>absence</i> of tlog dir), we
need a standin for
+ * the common case where <code>core.getUpdateHandler().getUpdateLog() ==
null</code>. This method
+ * returns the actual tlog dir if an {@link UpdateLog} is configured on the
core's {@link
+ * org.apache.solr.update.UpdateHandler}; otherwise, falls back to the
legacy behavior: if {@link
+ * CoreDescriptor#getUlogDir()} is specified, returns the <code>tlog</code>
subdirectory of that;
+ * otherwise returns the <code>tlog</code> subdirectory within {@link
SolrCore#getDataDir()}.
+ * (NOTE: the last of these is by far the most common default location of
the tlog directory).
+ */
+ static File getHypotheticalTlogDir(SolrCore core) {
+ String ulogDir;
+ UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
+ if (ulog != null) {
+ return new File(ulog.getTlogDir());
+ } else if ((ulogDir = core.getCoreDescriptor().getUlogDir()) != null) {
+ return new File(ulogDir, UpdateLog.TLOG_NAME);
+ } else {
+ return new File(core.getDataDir(), UpdateLog.TLOG_NAME);
+ }
+ }
+
/**
* Asserts that Update logs don't exist for replicas of type {@link
* org.apache.solr.common.cloud.Replica.Type#PULL}
@@ -233,10 +257,11 @@ public class TestPullReplica extends SolrCloudTestCase {
try (SolrCore core =
cluster.getReplicaJetty(r).getCoreContainer().getCore(r.getCoreName())) {
assertNotNull(core);
+ File tlogDir = getHypotheticalTlogDir(core);
assertFalse(
"Update log should not exist for replicas of type Passive but
file is present: "
- + core.getUlogDir(),
- new java.io.File(core.getUlogDir()).exists());
+ + tlogDir,
+ tlogDir.exists());
}
}
}
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
index c13795dcfca..563ed686298 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
@@ -16,8 +16,11 @@
*/
package org.apache.solr.cloud;
+import static org.apache.solr.cloud.TestPullReplica.getHypotheticalTlogDir;
+
import com.carrotsearch.randomizedtesting.annotations.Repeat;
import com.codahale.metrics.Meter;
+import java.io.File;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
@@ -142,9 +145,8 @@ public class TestTlogReplica extends SolrCloudTestCase {
try {
core =
cluster.getReplicaJetty(r).getCoreContainer().getCore(r.getCoreName());
assertNotNull(core);
- assertTrue(
- "Update log should exist for replicas of type Append",
- new java.io.File(core.getUlogDir()).exists());
+ File tlogDir = getHypotheticalTlogDir(core);
+ assertTrue("Update log should exist for replicas of type Append",
tlogDir.exists());
} finally {
core.close();
}
diff --git a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
index 9fd46e00c58..6cb6747c716 100644
--- a/solr/core/src/test/org/apache/solr/search/TestRecovery.java
+++ b/solr/core/src/test/org/apache/solr/search/TestRecovery.java
@@ -1382,7 +1382,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
assertU(commit());
UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
+ File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getTlogDir());
h.close();
@@ -1538,7 +1538,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
+ File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getTlogDir());
clearIndex();
assertU(commit());
@@ -1604,7 +1604,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
TestInjection.skipIndexWriterCommitOnClose = true;
UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
+ File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getTlogDir());
clearIndex();
assertU(commit());
@@ -1688,7 +1688,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
+ File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getTlogDir());
clearIndex();
assertU(commit());
@@ -1920,7 +1920,7 @@ public class TestRecovery extends SolrTestCaseJ4 {
// stops the core, removes the transaction logs, restarts the core.
void deleteLogs() throws Exception {
UpdateLog ulog = h.getCore().getUpdateHandler().getUpdateLog();
- File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getLogDir());
+ File logDir = new
File(h.getCore().getUpdateHandler().getUpdateLog().getTlogDir());
h.close();
diff --git a/solr/core/src/test/org/apache/solr/update/CustomTLogDirTest.java
b/solr/core/src/test/org/apache/solr/update/CustomTLogDirTest.java
new file mode 100644
index 00000000000..5ca9a2d335b
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/CustomTLogDirTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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.update;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.lucene.tests.mockfile.FilterPath;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.util.EmbeddedSolrServerTestRule;
+import org.apache.solr.util.SolrClientTestRule;
+import org.junit.ClassRule;
+
+public class CustomTLogDirTest extends SolrTestCaseJ4 {
+
+ @ClassRule
+ public static final SolrClientTestRule solrClientTestRule =
+ new EmbeddedSolrServerTestRule() {
+ @Override
+ protected void before() {
+ System.setProperty("solr.directoryFactory",
"solr.NRTCachingDirectoryFactory");
+ solrClientTestRule.startSolr(LuceneTestCase.createTempDir());
+ }
+ };
+
+ private static final AtomicInteger collectionIdx = new AtomicInteger();
+
+ public void testExternal() throws Exception {
+ String collectionName = "coll" + collectionIdx.getAndIncrement();
+ SolrClient client = solrClientTestRule.getSolrClient(collectionName);
+
+ Path coreRootDir = ((EmbeddedSolrServer)
client).getCoreContainer().getCoreRootDirectory();
+
+ Path instanceDir = FilterPath.unwrap(coreRootDir.resolve(collectionName));
+
+ Path ulogDir = LuceneTestCase.createTempDir();
+ // absolute path spec that falls outside of the instance and data dirs for
the
+ // associated core, is assumed to already by namespaced by purpose (tlog).
We
+ // expect it to be further namespaced by core name.
+ Path resolvedTlogDir = ulogDir.resolve(collectionName);
+ validateTlogPath(client, instanceDir, ulogDir, resolvedTlogDir);
+ }
+
+ public void testRelative() throws Exception {
+ String collectionName = "coll" + collectionIdx.getAndIncrement();
+ SolrClient client = solrClientTestRule.getSolrClient(collectionName);
+
+ Path coreRootDir = ((EmbeddedSolrServer)
client).getCoreContainer().getCoreRootDirectory();
+
+ Path instanceDir = FilterPath.unwrap(coreRootDir.resolve(collectionName));
+
+ Path ulogDir = Path.of("relativeUlogDir");
+ // relative dir path spec is taken to be relative to instance dir, so we
expect it
+ // to be namespaced by purpose within the core.
+ Path resolvedTlogDir = instanceDir.resolve(ulogDir).resolve("tlog");
+ validateTlogPath(client, instanceDir, ulogDir, resolvedTlogDir);
+ }
+
+ public void testIllegalRelative() throws Exception {
+ Path ulogDir = Path.of("../");
+
+ Path configSet = LuceneTestCase.createTempDir();
+ System.setProperty("solr.test.sys.prop2", "proptwo");
+ System.setProperty("solr.ulog.dir", ulogDir.toString()); // picked up from
`solrconfig.xml`
+ SolrTestCaseJ4.copyMinConf(configSet.toFile(), null, "solrconfig.xml");
+
+ // relative dir path specs should not be able to "escape" the core-scoped
instance dir;
+ // check that this config is unsuccessful
+ expectThrows(
+ Exception.class,
+ () ->
+ solrClientTestRule
+ .newCollection("illegal")
+ .withConfigSet(configSet.toString())
+ .create());
+ }
+
+ public void testAbsoluteSubdir() throws Exception {
+ String collectionName = "coll" + collectionIdx.getAndIncrement();
+ SolrClient client = solrClientTestRule.getSolrClient(collectionName);
+
+ Path coreRootDir = ((EmbeddedSolrServer)
client).getCoreContainer().getCoreRootDirectory();
+
+ Path instanceDir = FilterPath.unwrap(coreRootDir.resolve(collectionName));
+
+ Path ulogDir = instanceDir.resolve("absoluteUlogDir");
+ // an absolute dir path spec, if it is contained within the instance dir,
is taken
+ // to already be namespaced to the core. We expect the tlog dir to be
namespaced by
+ // purpose (tlog) within core, just as is the case with relative path spec.
+ Path resolvedTlogDir = ulogDir.resolve("tlog");
+ validateTlogPath(client, instanceDir, ulogDir, resolvedTlogDir);
+ }
+
+ public void testDefault() throws Exception {
+ String collectionName = "coll" + collectionIdx.getAndIncrement();
+ SolrClient client = solrClientTestRule.getSolrClient(collectionName);
+
+ Path coreRootDir = ((EmbeddedSolrServer)
client).getCoreContainer().getCoreRootDirectory();
+
+ Path instanceDir = FilterPath.unwrap(coreRootDir.resolve(collectionName));
+
+ // whether the normal default ulog dir spec `[instanceDir]/data` is
configured
+ // implicitly or explicitly, we expect to find the tlog in the same place:
+ Path resolvedTlogDir = instanceDir.resolve("data/tlog");
+ validateTlogPath(client, instanceDir, null, resolvedTlogDir);
+ }
+
+ public void testExplicitDefault() throws Exception {
+ String collectionName = "coll" + collectionIdx.getAndIncrement();
+ SolrClient client = solrClientTestRule.getSolrClient(collectionName);
+
+ Path coreRootDir = ((EmbeddedSolrServer)
client).getCoreContainer().getCoreRootDirectory();
+
+ Path instanceDir = FilterPath.unwrap(coreRootDir.resolve(collectionName));
+
+ Path ulogDir = instanceDir.resolve("data");
+ // whether the normal default ulog dir spec `[instanceDir]/data` is
configured
+ // implicitly or explicitly, we expect to find the tlog in the same place:
+ Path resolvedTlogDir = instanceDir.resolve("data/tlog");
+ validateTlogPath(client, instanceDir, ulogDir, resolvedTlogDir);
+ }
+
+ private static void validateTlogPath(
+ SolrClient client, Path instanceDir, Path ulogDir, Path resolvedTlogDir)
throws Exception {
+ Path configSet = LuceneTestCase.createTempDir();
+ System.setProperty("enable.update.log", "true");
+ System.setProperty("solr.test.sys.prop2", "proptwo");
+ if (ulogDir != null) {
+ System.setProperty("solr.ulog.dir", ulogDir.toString()); // picked up
from `solrconfig.xml`
+ }
+ SolrTestCaseJ4.copyMinConf(configSet.toFile(), null, "solrconfig.xml");
+
+ String collectionName = instanceDir.getFileName().toString();
+
+
solrClientTestRule.newCollection(collectionName).withConfigSet(configSet.toString()).create();
+
+ // resolvedTlogDir = instanceDir.resolve("data/tlog"); // legacy impl
_always_ resulted in this
+
+ // add some docs to populate tlog
+ client.add(sdoc("id", "1"));
+ client.add(sdoc("id", "2"));
+ client.add(sdoc("id", "3"));
+ client.commit();
+
+ File[] list =
+ resolvedTlogDir.toFile().listFiles((f) -> f.isFile() &&
f.getName().startsWith("tlog."));
+
+ assertNotNull(list);
+ assertEquals(1, list.length);
+ CoreContainer cc = ((EmbeddedSolrServer) client).getCoreContainer();
+ cc.unload(collectionName, true, true, true);
+ assertFalse(resolvedTlogDir.toFile().exists());
+ }
+}
diff --git
a/solr/modules/hdfs/src/java/org/apache/solr/hdfs/HdfsDirectoryFactory.java
b/solr/modules/hdfs/src/java/org/apache/solr/hdfs/HdfsDirectoryFactory.java
index 39e4e33fa41..dade59c1b62 100644
--- a/solr/modules/hdfs/src/java/org/apache/solr/hdfs/HdfsDirectoryFactory.java
+++ b/solr/modules/hdfs/src/java/org/apache/solr/hdfs/HdfsDirectoryFactory.java
@@ -17,6 +17,7 @@
package org.apache.solr.hdfs;
import static
org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.apache.solr.cloud.ZkController.trimLeadingAndTrailingSlashes;
import com.github.benmanes.caffeine.cache.Caffeine;
import com.github.benmanes.caffeine.cache.RemovalCause;
@@ -47,7 +48,6 @@ import org.apache.lucene.store.LockFactory;
import org.apache.lucene.store.NRTCachingDirectory;
import org.apache.lucene.store.NoLockFactory;
import org.apache.lucene.store.SingleInstanceLockFactory;
-import org.apache.solr.cloud.ZkController;
import org.apache.solr.cloud.api.collections.SplitShardCmd;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
@@ -463,6 +463,14 @@ public class HdfsDirectoryFactory extends
CachingDirectoryFactory
+ " for relative dataDir paths to work");
}
+ String scopePath = scopePath(cd);
+
+ return normalize(
+ SolrPaths.normalizeDir(
+ trimLeadingAndTrailingSlashes(hdfsDataDir) + "/" + scopePath + "/"
+ cd.getDataDir()));
+ }
+
+ public static String scopePath(CoreDescriptor cd) {
// by default, we go off the instance directory
String path;
if (cd.getCloudDescriptor() != null) {
@@ -474,14 +482,7 @@ public class HdfsDirectoryFactory extends
CachingDirectoryFactory
} else {
path = cd.getName();
}
-
- return normalize(
- SolrPaths.normalizeDir(
- ZkController.trimLeadingAndTrailingSlashes(hdfsDataDir)
- + "/"
- + path
- + "/"
- + cd.getDataDir()));
+ return path;
}
/**
diff --git
a/solr/modules/hdfs/src/java/org/apache/solr/hdfs/update/HdfsUpdateLog.java
b/solr/modules/hdfs/src/java/org/apache/solr/hdfs/update/HdfsUpdateLog.java
index cca223ba850..abaf7c67eb3 100644
--- a/solr/modules/hdfs/src/java/org/apache/solr/hdfs/update/HdfsUpdateLog.java
+++ b/solr/modules/hdfs/src/java/org/apache/solr/hdfs/update/HdfsUpdateLog.java
@@ -18,31 +18,32 @@ package org.apache.solr.hdfs.update;
import java.io.FileNotFoundException;
import java.io.IOException;
+import java.io.UncheckedIOException;
import java.lang.invoke.MethodHandles;
+import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
+import java.util.Objects;
import java.util.concurrent.atomic.AtomicLong;
-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.fs.PathFilter;
import org.apache.hadoop.ipc.RemoteException;
-import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FilterDirectory;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.common.util.IOUtils;
+import org.apache.solr.core.DirectoryFactory;
import org.apache.solr.core.PluginInfo;
import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrInfoBean;
-import org.apache.solr.hdfs.util.HdfsUtil;
+import org.apache.solr.hdfs.HdfsDirectoryFactory;
+import org.apache.solr.hdfs.store.HdfsDirectory;
import org.apache.solr.update.CommitUpdateCommand;
import org.apache.solr.update.TransactionLog;
-import org.apache.solr.update.UpdateHandler;
import org.apache.solr.update.UpdateLog;
-import org.apache.solr.update.VersionInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -51,7 +52,6 @@ import org.slf4j.LoggerFactory;
*/
public class HdfsUpdateLog extends UpdateLog {
- private final Object fsLock = new Object();
private FileSystem fs;
private volatile Path hdfsTlogDir;
private final String confDir;
@@ -81,65 +81,50 @@ public class HdfsUpdateLog extends UpdateLog {
log.info("Initializing HdfsUpdateLog: tlogDfsReplication={}",
tlogDfsReplication);
}
- private Configuration getConf(Path path) {
- Configuration conf = new Configuration();
- if (confDir != null) {
- HdfsUtil.addHdfsResources(conf, confDir);
- }
-
- String fsScheme = path.toUri().getScheme();
- if (fsScheme != null) {
- conf.setBoolean("fs." + fsScheme + ".impl.disable.cache", true);
- }
- return conf;
- }
-
@Override
- public void init(UpdateHandler uhandler, SolrCore core) {
-
- // ulogDir from CoreDescriptor overrides
- String ulogDir = core.getCoreDescriptor().getUlogDir();
-
- this.uhandler = uhandler;
-
- synchronized (fsLock) {
- // just like dataDir, we do not allow
- // moving the tlog dir on reload
- if (fs == null) {
- if (ulogDir != null) {
- dataDir = ulogDir;
- }
- if (dataDir == null || dataDir.length() == 0) {
- dataDir = core.getDataDir();
- }
+ protected void initTlogDir(SolrCore core) {
+ assert fs == null;
+ URI ulog = new Path(dataDir).toUri();
+ URI coreData = new Path(core.getDataDir()).toUri();
+ if (!ulog.getScheme().equals(coreData.getScheme())
+ || !Objects.equals(ulog.getAuthority(), coreData.getAuthority())
+ ||
!java.nio.file.Path.of(ulog.getPath()).startsWith(coreData.getPath())) {
+ // ulog is hdfs, but is not scoped under the core data dir (core data
dir may
+ // be not hdfs, or is a different host/port, or an external path within
the same
+ // host/port); either way we must scope it analogous to how the data dir
is
+ // scoped. This inherits the same issue as SOLR-7187, but we're not
making
+ // anything worse than it already is.
+ String scopePath =
HdfsDirectoryFactory.scopePath(core.getCoreDescriptor());
+ hdfsTlogDir = new Path(dataDir, scopePath);
+ } else {
+ hdfsTlogDir = new Path(dataDir, TLOG_NAME);
+ }
- if (!core.getDirectoryFactory().isAbsolute(dataDir)) {
- try {
- dataDir =
core.getDirectoryFactory().getDataHome(core.getCoreDescriptor());
- } catch (IOException e) {
- throw new SolrException(ErrorCode.SERVER_ERROR, e);
- }
- }
+ maybeClearLog(core);
- try {
- Path dataDirPath = new Path(dataDir);
- fs = FileSystem.get(dataDirPath.toUri(), getConf(dataDirPath));
- } catch (IOException e) {
- throw new SolrException(ErrorCode.SERVER_ERROR, e);
- }
- } else {
- if (debug) {
- log.debug(
- "UpdateHandler init: tlogDir={}, next id={} this is a reopen or
double init ... nothing else to do.",
- hdfsTlogDir,
- id);
- }
- versionInfo.reload();
- return;
+ try {
+ // usage of tlog dir almost entirely bypasses `Directory` API; we only
need to do this so that
+ // we can remove the tlog dir via `DirectoryFactory.remove()`, which
understands how to delete
+ // on hdfs.
+ DirectoryFactory df = core.getDirectoryFactory();
+ Directory tlogDir =
+ df.get(
+ hdfsTlogDir.toUri().toString(),
+ DirectoryFactory.DirContext.DEFAULT,
+ DirectoryFactory.LOCK_TYPE_NONE);
+ try {
+ // here we assume that Hdfs update log will only be configured in
conjunction with
+ // Hdfs DirectoryFactory.
+ fs = ((HdfsDirectory) FilterDirectory.unwrap(tlogDir)).getFileSystem();
+ } catch (Throwable t) {
+ df.release(tlogDir);
+ throw t;
}
+ this.releaseTlogDir = () -> df.release(tlogDir);
+ } catch (IOException e) {
+ throw new SolrException(ErrorCode.SERVER_ERROR, e);
}
- hdfsTlogDir = new Path(dataDir, TLOG_NAME);
while (true) {
try {
if (!fs.exists(hdfsTlogDir)) {
@@ -202,10 +187,8 @@ public class HdfsUpdateLog extends UpdateLog {
}
}
- // Record first two logs (oldest first) at startup for potential tlog
- // recovery.
- // It's possible that at abnormal close both "tlog" and "prevTlog" were
- // uncapped.
+ // Record first two logs (oldest first) at startup for potential tlog
recovery.
+ // It's possible that at abnormal close both "tlog" and "prevTlog" were
uncapped.
for (TransactionLog ll : logs) {
if (newestLogsOnStartup.size() < 2) {
newestLogsOnStartup.addFirst(ll);
@@ -215,44 +198,34 @@ public class HdfsUpdateLog extends UpdateLog {
ll.closeOutput();
}
}
+ }
- try {
- versionInfo = new VersionInfo(this, numVersionBuckets);
- } catch (SolrException e) {
- log.error("Unable to use updateLog: ", e);
- throw new SolrException(
- SolrException.ErrorCode.SERVER_ERROR, "Unable to use updateLog: " +
e.getMessage(), e);
+ @Override
+ protected String resolveDataDir(SolrCore core, String path) {
+ DirectoryFactory df = core.getDirectoryFactory();
+ if (!(df instanceof HdfsDirectoryFactory)) {
+ // Given that UpdateLog prioritizes low-latency, and that network
filesystems are
+ // higher-latency, for simplicity we require that HdfsUpdateLog can only
be configured
+ // if the DirectoryFactory is also Hdfs.
+ throw new IllegalStateException(
+ HdfsUpdateLog.class.getName() + " incompatible with " +
df.getClass().getName());
}
-
- // TODO: these startingVersions assume that we successfully recover from
all
- // non-complete tlogs.
- try (RecentUpdates startingUpdates = getRecentUpdates()) {
- startingVersions = startingUpdates.getVersions(getNumRecordsToKeep());
-
- // populate recent deletes list (since we can't get that info from the
- // index)
- for (int i = startingUpdates.deleteList.size() - 1; i >= 0; i--) {
- DeleteUpdate du = startingUpdates.deleteList.get(i);
- oldDeletes.put(new BytesRef(du.id), new LogPtr(-1, du.version));
- }
-
- // 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);
- trackDeleteByQuery(q, version);
- }
+ if (df.isAbsolute(path)) {
+ return path;
+ }
+ try {
+ // For `HdfsUpdateLog` it doesn't make sense to resolve relative to
`core.instanceDir()`
+ // (which is a regular filesystem path); any relative path will be
resolved wrt the core
+ // data home.
+ String dataHome = df.getDataHome(core.getCoreDescriptor());
+ return new Path(dataHome, path).toString();
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
}
-
- // initialize metrics
-
core.getCoreMetricManager().registerMetricProducer(SolrInfoBean.Category.TLOG.toString(),
this);
}
@Override
- public String getLogDir() {
+ public String getTlogDir() {
return hdfsTlogDir.toUri().toString();
}
@@ -305,15 +278,6 @@ public class HdfsUpdateLog extends UpdateLog {
close(committed, false);
}
- @Override
- public void close(boolean committed, boolean deleteOnClose) {
- try {
- super.close(committed, deleteOnClose);
- } finally {
- IOUtils.closeQuietly(fs);
- }
- }
-
@Override
protected void ensureBufferTlog() {
if (bufferTlog != null) return;
@@ -356,16 +320,10 @@ public class HdfsUpdateLog extends UpdateLog {
}
}
- /**
- * Clears the logs on the file system. Only call before init.
- *
- * @param core the SolrCore
- * @param ulogPluginInfo the init info for the UpdateHandler
- */
+ /** Clears the logs on the file system. Only call before init. */
@Override
- public void clearLog(SolrCore core, PluginInfo ulogPluginInfo) {
- if (ulogPluginInfo == null) return;
- Path tlogDir = new Path(getTlogDir(core, ulogPluginInfo));
+ public void clearLog() {
+ Path tlogDir = hdfsTlogDir;
try {
if (fs != null && fs.exists(tlogDir)) {
String[] files = getLogList(tlogDir);
diff --git
a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/search/TestRecoveryHdfs.java
b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/search/TestRecoveryHdfs.java
index be0591ef575..3a382b13fed 100644
---
a/solr/modules/hdfs/src/test/org/apache/solr/hdfs/search/TestRecoveryHdfs.java
+++
b/solr/modules/hdfs/src/test/org/apache/solr/hdfs/search/TestRecoveryHdfs.java
@@ -132,10 +132,10 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
addAndGetVersion(sdoc("id", "REP1"), null);
assertU(commit());
- String[] logList = ulog.getLogList(new Path(ulog.getLogDir()));
+ String[] logList = ulog.getLogList(new Path(ulog.getTlogDir()));
boolean foundRep2 = false;
for (String tl : logList) {
- FileStatus status = fs.getFileStatus(new Path(ulog.getLogDir(), tl));
+ FileStatus status = fs.getFileStatus(new Path(ulog.getTlogDir(), tl));
if (status.getReplication() == 2) {
foundRep2 = true;
break;
@@ -739,7 +739,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
clearIndex();
assertU(commit());
- String logDir =
h.getCore().getUpdateHandler().getUpdateLog().getLogDir();
+ String logDir =
h.getCore().getUpdateHandler().getUpdateLog().getTlogDir();
h.close();
@@ -877,7 +877,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
- String logDir =
h.getCore().getUpdateHandler().getUpdateLog().getLogDir();
+ String logDir =
h.getCore().getUpdateHandler().getUpdateLog().getTlogDir();
clearIndex();
assertU(commit());
@@ -938,7 +938,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
try {
TestInjection.skipIndexWriterCommitOnClose = true;
- String logDir =
h.getCore().getUpdateHandler().getUpdateLog().getLogDir();
+ String logDir =
h.getCore().getUpdateHandler().getUpdateLog().getTlogDir();
clearIndex();
assertU(commit());
@@ -1016,7 +1016,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
UpdateLog.testing_logReplayFinishHook = () -> logReplayFinish.release();
- String logDir =
h.getCore().getUpdateHandler().getUpdateLog().getLogDir();
+ String logDir =
h.getCore().getUpdateHandler().getUpdateLog().getTlogDir();
clearIndex();
assertU(commit());
@@ -1105,7 +1105,7 @@ public class TestRecoveryHdfs extends SolrTestCaseJ4 {
// stops the core, removes the transaction logs, restarts the core.
void deleteLogs() throws Exception {
- String logDir = h.getCore().getUpdateHandler().getUpdateLog().getLogDir();
+ String logDir = h.getCore().getUpdateHandler().getUpdateLog().getTlogDir();
h.close();
diff --git
a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
index 987f514f05e..d31b5651fbe 100644
---
a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
+++
b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CoreAdminRequest.java
@@ -747,7 +747,7 @@ public class CoreAdminRequest extends
SolrRequest<CoreAdminResponse> {
String configFile,
String schemaFile,
String dataDir,
- String tlogDir)
+ String ulogDir)
throws SolrServerException, IOException {
CoreAdminRequest.Create req = new CoreAdminRequest.Create();
req.setCoreName(name);
@@ -755,8 +755,8 @@ public class CoreAdminRequest extends
SolrRequest<CoreAdminResponse> {
if (dataDir != null) {
req.setDataDir(dataDir);
}
- if (tlogDir != null) {
- req.setUlogDir(tlogDir);
+ if (ulogDir != null) {
+ req.setUlogDir(ulogDir);
}
if (configFile != null) {
req.setConfigName(configFile);
diff --git
a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
index df1da5fa89c..063b538b743 100644
---
a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
+++
b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestCoreAdmin.java
@@ -107,7 +107,7 @@ public class TestCoreAdmin extends
AbstractEmbeddedSolrServerTestCase {
assertFalse(core.getCoreDescriptor().isLoadOnStartup());
assertTrue(coreProveIt.getCoreDescriptor().isLoadOnStartup());
- logDir = new File(core.getUpdateHandler().getUpdateLog().getLogDir());
+ logDir = new File(core.getUpdateHandler().getUpdateLog().getTlogDir());
}
assertEquals(