You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2023/06/21 19:06:05 UTC

[accumulo] branch 2.1 updated: Avoid calling Path.getParent in TabletFile constructor (#3509)

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

dlmarion pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/2.1 by this push:
     new 2fbb779908 Avoid calling Path.getParent in TabletFile constructor (#3509)
2fbb779908 is described below

commit 2fbb77990871ed705854fea6be97340869929ffe
Author: Dave Marion <dl...@apache.org>
AuthorDate: Wed Jun 21 15:05:59 2023 -0400

    Avoid calling Path.getParent in TabletFile constructor (#3509)
    
    Modified TabletFile constructor such that it still performs
    validation of the file path, but without calling Path.getParent().
---
 .../accumulo/core/metadata/TabletDirectory.java    |  65 ---------
 .../apache/accumulo/core/metadata/TabletFile.java  | 155 ++++++++++++++++-----
 .../accumulo/core/metadata/ValidationUtil.java     |   1 +
 .../core/metadata/schema/TabletFileTest.java       |   6 +-
 4 files changed, 123 insertions(+), 104 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TabletDirectory.java b/core/src/main/java/org/apache/accumulo/core/metadata/TabletDirectory.java
deleted file mode 100644
index 470055b8c2..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/metadata/TabletDirectory.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://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.accumulo.core.metadata;
-
-import static org.apache.accumulo.core.Constants.HDFS_TABLES_DIR;
-
-import org.apache.accumulo.core.data.TableId;
-
-/**
- * The Tablet directory that should exist on disk. The {@link #toString()} method only returns the
- * tablet directory itself, the same as {@link #getTabletDir()}, which is just the name of the
- * directory, like "t-0003". For the full directory path, use {@link #getNormalizedPath}.
- */
-public class TabletDirectory {
-  // parts of an absolute URI, like "hdfs://1.2.3.4/accumulo/tables/2a/t-0003"
-  private final String volume; // hdfs://1.2.3.4/accumulo
-  private final TableId tableId; // 2a
-  private final String tabletDir; // t-0003
-  private final String normalizedPath;
-
-  public TabletDirectory(String volume, TableId tableId, String tabletDir) {
-    this.volume = volume;
-    this.tableId = tableId;
-    this.tabletDir = tabletDir;
-    this.normalizedPath = volume + HDFS_TABLES_DIR + "/" + tableId.canonical() + "/" + tabletDir;
-
-  }
-
-  public String getVolume() {
-    return volume;
-  }
-
-  public TableId getTableId() {
-    return tableId;
-  }
-
-  public String getTabletDir() {
-    return tabletDir;
-  }
-
-  public String getNormalizedPath() {
-    return normalizedPath;
-  }
-
-  @Override
-  public String toString() {
-    return tabletDir;
-  }
-}
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java
index de974ca687..f216e24116 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java
@@ -20,9 +20,11 @@ package org.apache.accumulo.core.metadata;
 
 import static org.apache.accumulo.core.Constants.HDFS_TABLES_DIR;
 
+import java.net.URI;
 import java.util.Objects;
 
 import org.apache.accumulo.core.data.TableId;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
@@ -41,13 +43,114 @@ import com.google.common.base.Preconditions;
  * in Upgrader9to10.upgradeRelativePaths()
  */
 public class TabletFile implements Comparable<TabletFile> {
-  // parts of an absolute URI, like "hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf"
-  private final TabletDirectory tabletDir; // hdfs://1.2.3.4/accumulo/tables/2a/t-0003
-  private final String fileName; // C0004.rf
+
+  public static class FileParts {
+
+    // parts of an absolute URI, like "hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf"
+    // volume: hdfs://1.2.3.4/accumulo
+    // tableId: 2a
+    // tabletDir: t-0003
+    // fileName: C0004.rf
+    // normalizedPath: hdfs://1.2.3.4/accumulo/tables/2a/t-0003/C0004.rf
+    private final String volume;
+    private final TableId tableId;
+    private final String tabletDir;
+    private final String fileName;
+    private final String normalizedPath;
+
+    public FileParts(String volume, TableId tableId, String tabletDir, String fileName,
+        String normalizedPath) {
+      this.volume = volume;
+      this.tableId = tableId;
+      this.tabletDir = tabletDir;
+      this.fileName = fileName;
+      this.normalizedPath = normalizedPath;
+    }
+
+    public String getVolume() {
+      return volume;
+    }
+
+    public TableId getTableId() {
+      return tableId;
+    }
+
+    public String getTabletDir() {
+      return tabletDir;
+    }
+
+    public String getFileName() {
+      return fileName;
+    }
+
+    public String getNormalizedPath() {
+      return normalizedPath;
+    }
+
+  }
+
+  private static String constructErrorMsg(Path filePath) {
+    return "Missing or invalid part of tablet file metadata entry: " + filePath;
+  }
+
+  public static FileParts parsePath(Path filePath) {
+    // File name construct: <volume>/<tablePath>/<tableId>/<tablet>/<file>
+    // Example: hdfs://namenode:9020/accumulo/tables/1/default_tablet/F00001.rf
+    final URI uri = filePath.toUri();
+
+    // validate that this is a fully qualified uri
+    Preconditions.checkArgument(uri.getScheme() != null, constructErrorMsg(filePath));
+
+    final String path = uri.getPath(); // ex: /accumulo/tables/1/default_tablet/F00001.rf
+    final String[] parts = path.split("/");
+    final int numParts = parts.length; // should contain tables, 1, default_tablet, F00001.rf
+
+    if (numParts < 4) {
+      throw new IllegalArgumentException(constructErrorMsg(filePath));
+    }
+
+    final String fileName = parts[numParts - 1];
+    final String tabletDirectory = parts[numParts - 2];
+    final TableId tableId = TableId.of(parts[numParts - 3]);
+    final String tablesPath = parts[numParts - 4];
+
+    // determine where file path starts, the rest is the volume
+    final String computedFilePath =
+        HDFS_TABLES_DIR + "/" + tableId.canonical() + "/" + tabletDirectory + "/" + fileName;
+    final String uriString = uri.toString();
+    int idx = uriString.lastIndexOf(computedFilePath);
+
+    if (idx == -1) {
+      throw new IllegalArgumentException(constructErrorMsg(filePath));
+    }
+
+    // The volume is the beginning portion of the uri up to the start
+    // of the file path.
+    final String volume = uriString.substring(0, idx);
+
+    if (StringUtils.isBlank(fileName) || StringUtils.isBlank(tabletDirectory)
+        || StringUtils.isBlank(tablesPath) || StringUtils.isBlank(volume)) {
+      throw new IllegalArgumentException(constructErrorMsg(filePath));
+    }
+    ValidationUtil.validateFileName(fileName);
+    Preconditions.checkArgument(tablesPath.equals(HDFS_TABLES_DIR_NAME),
+        "tables directory name is not " + HDFS_TABLES_DIR_NAME + ", is " + tablesPath);
+
+    final String normalizedPath = volume + computedFilePath;
+
+    if (!normalizedPath.equals(uriString)) {
+      throw new RuntimeException("Error parsing file path, " + normalizedPath + " != " + uriString);
+    }
+
+    return new FileParts(volume, tableId, tabletDirectory, fileName, normalizedPath);
+
+  }
+
   protected final Path metaPath;
-  private final String normalizedPath;
+  private final FileParts parts;
 
   private static final Logger log = LoggerFactory.getLogger(TabletFile.class);
+  private static final String HDFS_TABLES_DIR_NAME = HDFS_TABLES_DIR.substring(1);
 
   /**
    * Construct new tablet file using a Path. Used in the case where we had to use Path object to
@@ -55,44 +158,24 @@ public class TabletFile implements Comparable<TabletFile> {
    */
   public TabletFile(Path metaPath) {
     this.metaPath = Objects.requireNonNull(metaPath);
-    String errorMsg = "Missing or invalid part of tablet file metadata entry: " + metaPath;
     log.trace("Parsing TabletFile from {}", metaPath);
-
-    // use Path object to step backwards from the filename through all the parts
-    this.fileName = metaPath.getName();
-    ValidationUtil.validateFileName(fileName);
-
-    Path tabletDirPath = Objects.requireNonNull(metaPath.getParent(), errorMsg);
-
-    Path tableIdPath = Objects.requireNonNull(tabletDirPath.getParent(), errorMsg);
-    var id = tableIdPath.getName();
-
-    Path tablePath = Objects.requireNonNull(tableIdPath.getParent(), errorMsg);
-    String tpString = "/" + tablePath.getName();
-    Preconditions.checkArgument(tpString.equals(HDFS_TABLES_DIR), errorMsg);
-
-    Path volumePath = Objects.requireNonNull(tablePath.getParent(), errorMsg);
-    Preconditions.checkArgument(volumePath.toUri().getScheme() != null, errorMsg);
-    var volume = volumePath.toString();
-
-    this.tabletDir = new TabletDirectory(volume, TableId.of(id), tabletDirPath.getName());
-    this.normalizedPath = tabletDir.getNormalizedPath() + "/" + fileName;
+    parts = parsePath(metaPath);
   }
 
   public String getVolume() {
-    return tabletDir.getVolume();
+    return parts.getVolume();
   }
 
   public TableId getTableId() {
-    return tabletDir.getTableId();
+    return parts.getTableId();
   }
 
   public String getTabletDir() {
-    return tabletDir.getTabletDir();
+    return parts.getTabletDir();
   }
 
   public String getFileName() {
-    return fileName;
+    return parts.getFileName();
   }
 
   /**
@@ -100,14 +183,14 @@ public class TabletFile implements Comparable<TabletFile> {
    * metadata.
    */
   public String getPathStr() {
-    return normalizedPath;
+    return parts.getNormalizedPath();
   }
 
   /**
    * Return a string for inserting a new tablet file.
    */
   public String getMetaInsert() {
-    return normalizedPath;
+    return parts.getNormalizedPath();
   }
 
   /**
@@ -121,7 +204,7 @@ public class TabletFile implements Comparable<TabletFile> {
    * New file was written to metadata so return a StoredTabletFile
    */
   public StoredTabletFile insert() {
-    return new StoredTabletFile(normalizedPath);
+    return new StoredTabletFile(parts.getNormalizedPath());
   }
 
   public Path getPath() {
@@ -133,7 +216,7 @@ public class TabletFile implements Comparable<TabletFile> {
     if (equals(o)) {
       return 0;
     } else {
-      return normalizedPath.compareTo(o.normalizedPath);
+      return parts.getNormalizedPath().compareTo(o.parts.getNormalizedPath());
     }
   }
 
@@ -141,18 +224,18 @@ public class TabletFile implements Comparable<TabletFile> {
   public boolean equals(Object obj) {
     if (obj instanceof TabletFile) {
       TabletFile that = (TabletFile) obj;
-      return normalizedPath.equals(that.normalizedPath);
+      return parts.getNormalizedPath().equals(that.parts.getNormalizedPath());
     }
     return false;
   }
 
   @Override
   public int hashCode() {
-    return normalizedPath.hashCode();
+    return parts.getNormalizedPath().hashCode();
   }
 
   @Override
   public String toString() {
-    return normalizedPath;
+    return parts.getNormalizedPath();
   }
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java b/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java
index 7b8ba3ba01..f4bfa4358a 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ValidationUtil.java
@@ -53,6 +53,7 @@ public class ValidationUtil {
 
   public static void validateRFileName(String fileName) {
     Objects.requireNonNull(fileName);
+    // TODO: In 3.0.0 validate that filename starts with FilePrefix
     if (!fileName.endsWith(".rf") && !fileName.endsWith("_tmp")) {
       throw new IllegalArgumentException(
           "Provided filename (" + fileName + ") does not end with '.rf' or '_tmp'");
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletFileTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletFileTest.java
index d5a3ba737b..9ac3846031 100644
--- a/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/schema/TabletFileTest.java
@@ -57,9 +57,9 @@ public class TabletFileTest {
     // 2a< srv:dir
     final String message = "Failed to throw error on bad path";
 
-    assertThrows(NullPointerException.class, () -> test("C0004.rf", "", "2a", "t-0003", "C0004.rf"),
-        message);
-    assertThrows(NullPointerException.class, () -> test("dir", "", "2a", "", ""), message);
+    assertThrows(IllegalArgumentException.class,
+        () -> test("C0004.rf", "", "2a", "t-0003", "C0004.rf"), message);
+    assertThrows(IllegalArgumentException.class, () -> test("dir", "", "2a", "", ""), message);
 
     assertThrows(IllegalArgumentException.class,
         () -> test("hdfs://localhost:8020/accumulo/tablets/2a/default_tablet/F0000070.rf",