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(

Reply via email to