You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2020/06/25 04:19:08 UTC

[lucene-solr] branch master updated: SOLR Move "userfiles" stuff from SolrPaths to CoreContainer. (#1600)

This is an automated email from the ASF dual-hosted git repository.

dsmiley pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 3000bbd  SOLR Move "userfiles" stuff from SolrPaths to CoreContainer. (#1600)
3000bbd is described below

commit 3000bbd05aeff0d821b276a912ec481a3411f2ab
Author: David Smiley <ds...@apache.org>
AuthorDate: Thu Jun 25 00:18:58 2020 -0400

    SOLR Move "userfiles" stuff from SolrPaths to CoreContainer. (#1600)
    
    And convert String and File paths to Path API
---
 .../java/org/apache/solr/core/CoreContainer.java   | 17 ++++++
 .../src/java/org/apache/solr/core/SolrPaths.java   | 27 ---------
 .../java/org/apache/solr/handler/CatStream.java    | 70 +++++++++-------------
 .../apache/solr/servlet/SolrDispatchFilter.java    |  1 -
 .../solrj/io/stream/StreamExpressionTest.java      | 42 ++++++-------
 5 files changed, 62 insertions(+), 95 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 1ce5100..e8ab59e 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -364,6 +364,13 @@ public class CoreContainer {
         log.info("Allowing use of paths: {}", cfg.getAllowPaths());
       }
     }
+
+    Path userFilesPath = getUserFilesPath(); // TODO make configurable on cfg?
+    try {
+      Files.createDirectories(userFilesPath); // does nothing if already exists
+    } catch (Exception e) {
+      log.warn("Unable to create [{}].  Features requiring this directory may fail.", userFilesPath, e);
+    }
   }
 
   @SuppressWarnings({"unchecked"})
@@ -1987,6 +1994,16 @@ public class CoreContainer {
     return solrHome.toString();
   }
 
+  /**
+   * A path where Solr users can retrieve arbitrary files from.  Absolute.
+   * <p>
+   * This directory is generally created by each node on startup.  Files located in this directory can then be
+   * manipulated using select Solr features (e.g. streaming expressions).
+   */
+  public Path getUserFilesPath() {
+    return solrHome.resolve("userfiles");
+  }
+
   public boolean isZooKeeperAware() {
     return zkSys.getZkController() != null;
   }
diff --git a/solr/core/src/java/org/apache/solr/core/SolrPaths.java b/solr/core/src/java/org/apache/solr/core/SolrPaths.java
index b42085b..0a82cc2 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrPaths.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrPaths.java
@@ -39,13 +39,6 @@ import org.slf4j.LoggerFactory;
 public final class SolrPaths {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  /**
-   * Solr allows users to store arbitrary files in a special directory located directly under SOLR_HOME.
-   * <p>
-   * This directory is generally created by each node on startup.  Files located in this directory can then be
-   * manipulated using select Solr features (e.g. streaming expressions).
-   */
-  public static final String USER_FILES_DIRECTORY = "userfiles";
   private static final Set<String> loggedOnce = new ConcurrentSkipListSet<>();
 
   private SolrPaths() {} // don't create this
@@ -94,26 +87,6 @@ public final class SolrPaths {
     return Paths.get(home).toAbsolutePath().normalize();
   }
 
-  public static void ensureUserFilesDataDir(Path solrHome) {
-    final Path userFilesPath = getUserFilesPath(solrHome);
-    final File userFilesDirectory = new File(userFilesPath.toString());
-    if (!userFilesDirectory.exists()) {
-      try {
-        final boolean created = userFilesDirectory.mkdir();
-        if (!created) {
-          log.warn("Unable to create [{}] directory in SOLR_HOME [{}].  Features requiring this directory may fail.", USER_FILES_DIRECTORY, solrHome);
-        }
-      } catch (Exception e) {
-        log.warn("Unable to create [{}] directory in SOLR_HOME [{}].  Features requiring this directory may fail.",
-            USER_FILES_DIRECTORY, solrHome, e);
-      }
-    }
-  }
-
-  public static Path getUserFilesPath(Path solrHome) {
-    return Paths.get(solrHome.toAbsolutePath().toString(), USER_FILES_DIRECTORY).toAbsolutePath();
-  }
-
   /**
    * Ensures a directory name always ends with a '/'.
    */
diff --git a/solr/core/src/java/org/apache/solr/handler/CatStream.java b/solr/core/src/java/org/apache/solr/handler/CatStream.java
index 806c94a..0f11fdf 100644
--- a/solr/core/src/java/org/apache/solr/handler/CatStream.java
+++ b/solr/core/src/java/org/apache/solr/handler/CatStream.java
@@ -17,15 +17,14 @@
 
 package org.apache.solr.handler;
 
-import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.nio.file.Files;
-import java.nio.file.Paths;
+import java.nio.file.Path;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
+import java.util.stream.Stream;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.LineIterator;
@@ -41,7 +40,7 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.StringUtils;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrPaths;
+import org.eclipse.jetty.io.RuntimeIOException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,7 +51,7 @@ public class CatStream extends TupleStream implements Expressible {
   private final int maxLines; // -1 for no max
 
   private StreamContext context;
-  private String chroot;
+  private Path chroot;
   private Iterator<CrawlFile> allFilesToCrawl;
 
   private int linesReturned = 0;
@@ -94,9 +93,9 @@ public class CatStream extends TupleStream implements Expressible {
     }
     final SolrCore core = (SolrCore) context.get("solr-core");
 
-    this.chroot = Paths.get(core.getCoreContainer().getSolrHome(), SolrPaths.USER_FILES_DIRECTORY).toString();
-    if (! new File(this.chroot).exists()) {
-      throw new IllegalStateException(SolrPaths.USER_FILES_DIRECTORY + " directory used to load files must exist but could not be found!");
+    this.chroot = core.getCoreContainer().getUserFilesPath();
+    if (! Files.exists(chroot)) {
+      throw new IllegalStateException(chroot + " directory used to load files must exist but could not be found!");
     }
   }
 
@@ -158,28 +157,20 @@ public class CatStream extends TupleStream implements Expressible {
   }
 
   private List<CrawlFile> validateAndSetFilepathsInSandbox() {
-    final String[] relativePathRoots = commaDelimitedFilepaths.split(",");
-
     final List<CrawlFile> crawlSeeds = new ArrayList<>();
-    for (String crawlRoot : relativePathRoots) {
-      final File crawlRootFile = new File(crawlRoot);
-      if (crawlRootFile.isAbsolute()) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "file/directory to stream must be provided as an absolute path: " + crawlRoot);
-      }
-      if ( crawlRoot.contains("..")) {
+    for (String crawlRootStr : commaDelimitedFilepaths.split(",")) {
+      Path crawlRootPath = chroot.resolve(crawlRootStr).normalize();
+      if (! crawlRootPath.startsWith(chroot)) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "relative file/directory paths cannot contain '..': " + crawlRoot);
+            "file/directory to stream must be under " + chroot);
       }
 
-      final String rootAbsolutePath = getAbsolutePath(crawlRoot);
-      if (! new File(getAbsolutePath(crawlRoot)).exists()) {
-        log.warn("Unable to find abs path: {}", getAbsolutePath(crawlRoot));
+      if (! Files.exists(crawlRootPath)) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "file/directory to stream doesn't exist: " + crawlRoot);
+            "file/directory to stream doesn't exist: " + crawlRootStr);
       }
 
-      crawlSeeds.add(new CrawlFile(crawlRoot, rootAbsolutePath));
+      crawlSeeds.add(new CrawlFile(crawlRootStr, crawlRootPath));
     }
 
     return crawlSeeds;
@@ -189,7 +180,7 @@ public class CatStream extends TupleStream implements Expressible {
     while (allFilesToCrawl.hasNext()) {
       closeCurrentFileIfSet();
       currentFilePath = allFilesToCrawl.next();
-      currentFileLines = FileUtils.lineIterator(new File(currentFilePath.absolutePath), "UTF-8");
+      currentFileLines = FileUtils.lineIterator(currentFilePath.absolutePath.toFile(), "UTF-8");
       if (currentFileLines.hasNext()) return true;
     }
 
@@ -210,10 +201,6 @@ public class CatStream extends TupleStream implements Expressible {
     return currentFileLines != null && currentFileLines.hasNext();
   }
 
-  private String getAbsolutePath(String pathRelativeToChroot) {
-    return Paths.get(chroot, pathRelativeToChroot).toString();
-  }
-
   private void closeCurrentFileIfSet() throws IOException {
     if (currentFilePath != null) {
       currentFileLines.close();
@@ -224,25 +211,24 @@ public class CatStream extends TupleStream implements Expressible {
 
   private void findReadableFiles(CrawlFile seed, List<CrawlFile> foundFiles) {
 
-    final File entry = new File(seed.absolutePath);
+    final Path entry = seed.absolutePath;
 
     // Skip over paths that don't exist or that are symbolic links
-    if ((!entry.exists()) || (!entry.canRead()) || Files.isSymbolicLink(entry.toPath())) {
+    if ((!Files.exists(entry)) || (!Files.isReadable(entry)) || Files.isSymbolicLink(entry)) {
       return;
     }
 
     // We already know that the path in question exists, is readable, and is in our sandbox
-    if (entry.isFile()) {
+    if (Files.isRegularFile(entry)) {
       foundFiles.add(seed);
-    } else if (entry.isDirectory()) {
-      final String[] directoryContents = entry.list();
-      Arrays.sort(directoryContents);
-      if (directoryContents != null) {
-        for (String item : directoryContents) {
-          final String itemDisplayPath = Paths.get(seed.displayPath, item).toString();
-          final String itemAbsolutePath = Paths.get(seed.absolutePath, item).toString();
-          findReadableFiles(new CrawlFile(itemDisplayPath, itemAbsolutePath), foundFiles);
-        }
+    } else if (Files.isDirectory(entry)) {
+      try (Stream<Path> directoryContents = Files.list(entry)) {
+        directoryContents.sorted().forEach(iPath -> {
+          final String itemDisplayPath = seed.displayPath + "/" + iPath.getFileName();
+          findReadableFiles(new CrawlFile(itemDisplayPath, iPath), foundFiles);
+        });
+      } catch (IOException e) {
+        throw new RuntimeIOException(e);
       }
     }
   }
@@ -252,9 +238,9 @@ public class CatStream extends TupleStream implements Expressible {
   // - display path to avoid leaking Solr node fs details in tuples (relative to chroot)
   public class CrawlFile {
     private final String displayPath;
-    private final String absolutePath;
+    private final Path absolutePath;
 
-    public CrawlFile(String displayPath, String absolutePath) {
+    public CrawlFile(String displayPath, Path absolutePath) {
       this.displayPath = displayPath;
       this.absolutePath = absolutePath;
     }
diff --git a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
index c18c7a1..a7d8905 100644
--- a/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
+++ b/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
@@ -182,7 +182,6 @@ public class SolrDispatchFilter extends BaseSolrFilter {
       String solrHome = (String) config.getServletContext().getAttribute(SOLRHOME_ATTRIBUTE);
       final Path solrHomePath = solrHome == null ? SolrPaths.locateSolrHome() : Paths.get(solrHome);
       coresInit = createCoreContainer(solrHomePath, extraProperties);
-      SolrPaths.ensureUserFilesDataDir(solrHomePath);
       this.httpClient = coresInit.getUpdateShardHandler().getDefaultHttpClient();
       setupJvmMetrics(coresInit);
       if (log.isDebugEnabled()) {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index 99e3cfb..fa72020 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -21,7 +21,7 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
-import java.nio.file.Paths;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -57,7 +57,6 @@ import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.core.CoreDescriptor;
-import org.apache.solr.core.SolrPaths;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -99,7 +98,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     // Create a collection for use by the filestream() expression, and place some files there for it to read.
     CollectionAdminRequest.createCollection(FILESTREAM_COLLECTION, "conf", 1, 1).process(cluster.getSolrClient());
     cluster.waitForActiveCollection(FILESTREAM_COLLECTION, 1, 1);
-    final String dataDir = findUserFilesDataDir();
+    final Path dataDir = findUserFilesDataDir();
     populateFileStreamData(dataDir);
   }
 
@@ -3550,12 +3549,12 @@ public class StreamExpressionTest extends SolrCloudTestCase {
   }
 
 
-  private static String findUserFilesDataDir() {
+  private static Path findUserFilesDataDir() {
     for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       final String baseDir = cluster.getBaseDir().toAbsolutePath().toString();
       for (CoreDescriptor coreDescriptor : jetty.getCoreContainer().getCoreDescriptors()) {
         if (coreDescriptor.getCollectionName().equals(FILESTREAM_COLLECTION)) {
-          return Paths.get(jetty.getSolrHome(), SolrPaths.USER_FILES_DIRECTORY).toAbsolutePath().toString();
+          return jetty.getCoreContainer().getUserFilesPath();
         }
       }
     }
@@ -3578,29 +3577,22 @@ public class StreamExpressionTest extends SolrCloudTestCase {
    *
    * Each file contains 4 lines.  Each line looks like: "<filename> line <linenumber>"
    */
-  private static void populateFileStreamData(String dataDir) throws Exception {
-    final File baseDataDir = new File(dataDir);
-    if (! baseDataDir.exists()) baseDataDir.mkdir();
-    final File directory1 = new File(Paths.get(dataDir, "directory1").toString());
-    directory1.mkdir();
-
-    final File topLevel1 = new File(Paths.get(dataDir, "topLevel1.txt").toString());
-    final File topLevel2 = new File(Paths.get(dataDir, "topLevel2.txt").toString());
-    final File topLevelEmpty = new File(Paths.get(dataDir, "topLevel-empty.txt").toString());
-    final File secondLevel1 = new File(Paths.get(dataDir, "directory1", "secondLevel1.txt").toString());
-    final File secondLevel2 = new File(Paths.get(dataDir, "directory1", "secondLevel2.txt").toString());
-    populateFileWithData(topLevel1);
-    populateFileWithData(topLevel2);
-    topLevelEmpty.createNewFile();
-    populateFileWithData(secondLevel1);
-    populateFileWithData(secondLevel2);
+  private static void populateFileStreamData(Path dataDir) throws Exception {
+    Files.createDirectories(dataDir);
+    Files.createDirectories(dataDir.resolve("directory1"));
+
+    populateFileWithData(dataDir.resolve("topLevel1.txt"));
+    populateFileWithData(dataDir.resolve("topLevel2.txt"));
+    Files.createFile(dataDir.resolve("topLevel-empty.txt"));
+    populateFileWithData(dataDir.resolve("directory1").resolve("secondLevel1.txt"));
+    populateFileWithData(dataDir.resolve("directory1").resolve("secondLevel2.txt"));
   }
 
-  private static void populateFileWithData(File dataFile) throws Exception {
-    dataFile.createNewFile();
-    try (final BufferedWriter writer = Files.newBufferedWriter(Paths.get(dataFile.toURI()), StandardCharsets.UTF_8)) {
+  private static void populateFileWithData(Path dataFile) throws Exception {
+    Files.createFile(dataFile);
+    try (final BufferedWriter writer = Files.newBufferedWriter(dataFile, StandardCharsets.UTF_8)) {
       for (int i = 1; i <=4; i++) {
-        writer.write(dataFile.getName() + " line " + String.valueOf(i));
+        writer.write(dataFile.getFileName() + " line " + i);
         writer.newLine();
       }
     }