You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "cshannon (via GitHub)" <gi...@apache.org> on 2023/05/14 21:23:08 UTC

[GitHub] [accumulo] cshannon opened a new pull request, #3401: Start using TabletFile instead of String for referencing data files

cshannon opened a new pull request, #3401:
URL: https://github.com/apache/accumulo/pull/3401

   This PR replaces the now closed #3385. The goal of this PR is to start to move away from using a String/Path representation everywhere for the file path reference when referring to data files in order to support easier changes in the future when adding more information besides just a path. This is to support the future changes #1327 I am making this a draft PR for nor was before this is merged I need to write new Javadocs for the new classes in the API and document changes better but before I did that I wanted to get feedback first.
   
   The current plan for supporting no chop merges is to start associating a range with a file and treating each file that is fenced off by a range as unique. This is going to lead to storing more than one metadata entry for the same file if there is multiple ranges so we will need to start handling the combination of the path and Range in the code when reading a file and not just the path like today.
   
   This PR is an incremental step by using TabletFile/StoredTabletFile and the new `AbstractTabletFile` and `UnassignedTabletFile` classes instead of String in places like `FileManager` and `FileOperations`. The idea here is that by always using those TabletFile classes instead of a Path/String it will be easier to read/write a fenced file as we can add a Range in the future to AbstractTabletFile and a getRange() method.
   
   I originally tried to just use TabletFile but ended up creating two new classes called AbstractTabletFile and UnassignedTabletFile because TabletFile does a lot of [validation](https://github.com/apache/accumulo/blob/2f27dbd188ea5cee32f486599b97181b136fa787/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java#L61) that caused several places to break in the code. There are a lot of areas where we write to RFiles or read from RFiles but the file wouldn't pass the validation checks inside TabletFile as TabletFile requires the path to contain the right structure so it can be [parsed](https://github.com/apache/accumulo/blob/2f27dbd188ea5cee32f486599b97181b136fa787/core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java#L78) into a TabletDirectory object. This is problematic in several spots: Writing files, Tests, WAL, Bulk import, etc. Essentially any place we would be doing operations on files that are not part of a Tablet/Tablet yet.
   
   I used the name `UnassignedTabletFile` but a different name can be picked of course. I chose that name as that seemed to be the best name for it is in all cases it's essentially a data file that hasn't actually been assigned to a Tablet or Tablet which is why we can't parse the TabletDirectory. When we add a Range to the API we could still fence off the new unassigned file if we wanted to (if there's a use case), there wouldn't be anything stopping us from assigning a range but I am going to assume the majority of cases (if not all) there wouldn't be an actual range, it would just be infinite.
   
   Also note that even with tracking a TabletFile we will still always need to get the metadata string representation for serialization when dealing with the metadata table. This is unavoidable of course and the format will change in the future (likely to Json instead of just a file path as it will have a serialized range) so we will need to handle that. A good example of this is ExternalCompactionMetadata. This class already tracks a StoredTabletFile but when data is serialized it calls to getMetaUpdateDelete() and that will be a new format. The format won't change in this first PR but it will later so just noting that we will need to be aware of that and make sure we can handle the change when the Range is added.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1198871377


##########
core/src/main/java/org/apache/accumulo/core/metadata/UnassignedTabletFile.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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 java.io.File;
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class UnassignedTabletFile extends AbstractTabletFile<UnassignedTabletFile> {

Review Comment:
   I like this suggestion, I think `UnreferencedTabletFile` is a better name based on the fact that the "references" is used elsewhere already.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1198061983


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java:
##########
@@ -548,22 +548,22 @@ public SortedMap<KeyExtent,Bulk.Files> computeFileToTabletMappings(FileSystem fs
         context.instanceOperations().getSystemConfiguration(), tableProps, tableId);
 
     for (FileStatus fileStatus : files) {
-      Path filePath = fileStatus.getPath();
+      UnassignedTabletFile file = UnassignedTabletFile.of(fs, fileStatus.getPath());
       CompletableFuture<Map<KeyExtent,Bulk.FileInfo>> future = CompletableFuture.supplyAsync(() -> {
         try {
           long t1 = System.currentTimeMillis();
           List<KeyExtent> extents =
-              findOverlappingTablets(context, extentCache, filePath, fs, fileLensCache, cs);
+              findOverlappingTablets(context, extentCache, file, fs, fileLensCache, cs);
           // make sure file isn't going to too many tablets
-          checkTabletCount(maxTablets, extents.size(), filePath.toString());
-          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), filePath,
+          checkTabletCount(maxTablets, extents.size(), file.toString());
+          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), file,
               fileStatus.getLen(), extents, fs, fileLensCache, cs);
           Map<KeyExtent,Bulk.FileInfo> pathLocations = new HashMap<>();
           for (KeyExtent ke : extents) {
-            pathLocations.put(ke, new Bulk.FileInfo(filePath, estSizes.getOrDefault(ke, 0L)));
+            pathLocations.put(ke, new Bulk.FileInfo(file.getPath(), estSizes.getOrDefault(ke, 0L)));

Review Comment:
   Looked into passing TabletFile to FileInfo.  FileInfo is used for serialization, so internally it needs a string.  Could possibly pass the tablet file obj have it do the conversion in the constructor. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1199122626


##########
core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java:
##########
@@ -155,4 +142,9 @@ public int hashCode() {
   public String toString() {
     return normalizedPath;
   }
+
+  public static TabletFile of(final Path path) {

Review Comment:
   > Constructors are not very descriptive which is annoying. If you have 10 different constructors they are all called the same thing (class name) with different arguments so it's not always clear the difference at first glance. 
   
   That is a good reason for using static methods instead of constructors, I like that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1198894791


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java:
##########
@@ -261,9 +262,9 @@ public MLong(long i) {
     long l;
   }
 
-  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path dataFile,
-      long fileSize, Collection<KeyExtent> extents, FileSystem ns, Cache<String,Long> fileLenCache,
-      CryptoService cs) throws IOException {
+  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf,
+      UnassignedTabletFile dataFile, long fileSize, Collection<KeyExtent> extents, FileSystem ns,
+      Cache<String,Long> fileLenCache, CryptoService cs) throws IOException {

Review Comment:
   I noticed this as well but left it alone currently as I wasn't sure if it would buy us much to switch since it's just a file length cache but it might be better to do it just for consistency and have something more safe to use than String for changes in the future.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon commented on pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#issuecomment-1555008367

   Everything passed except for 2 tests which I fixed, I will merge this once QA checks pass


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon merged pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon merged PR #3401:
URL: https://github.com/apache/accumulo/pull/3401


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1198874844


##########
core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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 java.util.Objects;
+
+import org.apache.hadoop.fs.Path;
+
+public abstract class AbstractTabletFile<T extends AbstractTabletFile<T>> implements Comparable<T> {

Review Comment:
   That's an interesting idea, I also have talked to @EdColeman a bit about #2830 and I do plan to address that problem as well in another PR. I think either having StoredTabletFile become a sibling to TabletFile or have StoredTabledFile contain a TabletFile (similar to the Location fix in #3257) instead of using inheritance could help us solve the issue but I will experiment with this a bit and see.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1198885348


##########
core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java:
##########
@@ -155,4 +142,9 @@ public int hashCode() {
   public String toString() {
     return normalizedPath;
   }
+
+  public static TabletFile of(final Path path) {

Review Comment:
   In this particular case it could be removed as it doesn't buy that much, this for me is mostly a personal preference. If we keep it we could definitely make the constructor private if we want to enforce the new method. I have grown to like using more descriptive and fluent apis ever since lambdas were introduced and the builder pattern became more of a thing. There's 2 reasons I like using these types of methods vs a constructor directly.
   
   1. I personally just find it is nice to create static initializer methods to make things more compact. It's just nicer to write `TabletFile.of(path) `vs having to write `new TabletFile(path)` . 
   2. Constructors are not very descriptive which is annoying. If you have 10 different constructors they are all called the same thing (class name) with different arguments so it's not always clear the difference at first glance. By using static methods you can name them anything you want and it makes it very clear in the code what you are doing. A good example of this being beneficial was my PR for #3257 which added a bunch of methods for Location [here](https://github.com/apache/accumulo/blob/4d7f81954be05e414f814aa116ed65536edf7ba6/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletMetadata.java#L204-L226). It makes the code nice to just be able to say `Location.last(instance)` or `Location.current(instance)` etc without having to call a constructor directly and pass in more args. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1198909020


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java:
##########
@@ -548,22 +548,22 @@ public SortedMap<KeyExtent,Bulk.Files> computeFileToTabletMappings(FileSystem fs
         context.instanceOperations().getSystemConfiguration(), tableProps, tableId);
 
     for (FileStatus fileStatus : files) {
-      Path filePath = fileStatus.getPath();
+      UnassignedTabletFile file = UnassignedTabletFile.of(fs, fileStatus.getPath());
       CompletableFuture<Map<KeyExtent,Bulk.FileInfo>> future = CompletableFuture.supplyAsync(() -> {
         try {
           long t1 = System.currentTimeMillis();
           List<KeyExtent> extents =
-              findOverlappingTablets(context, extentCache, filePath, fs, fileLensCache, cs);
+              findOverlappingTablets(context, extentCache, file, fs, fileLensCache, cs);
           // make sure file isn't going to too many tablets
-          checkTabletCount(maxTablets, extents.size(), filePath.toString());
-          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), filePath,
+          checkTabletCount(maxTablets, extents.size(), file.toString());
+          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), file,
               fileStatus.getLen(), extents, fs, fileLensCache, cs);
           Map<KeyExtent,Bulk.FileInfo> pathLocations = new HashMap<>();
           for (KeyExtent ke : extents) {
-            pathLocations.put(ke, new Bulk.FileInfo(filePath, estSizes.getOrDefault(ke, 0L)));
+            pathLocations.put(ke, new Bulk.FileInfo(file.getPath(), estSizes.getOrDefault(ke, 0L)));

Review Comment:
   I think we might want a follow on issue to fix BulkImport in general, maybe the same time we update the file length cache to use a better key than a String. Strings are used extensively in BulkImport throughout the entire API so we probably should try and make it better wherever we can and use TabletFile references while not breaking serialization. I didn't make BulkImport a priority here because BulkImport is importing new files of course and wouldn't really need to worry about fencing. 
   
   I'm wondering if I can refactor classes like [LoadPlan](https://github.com/apache/accumulo/blob/4d7f81954be05e414f814aa116ed65536edf7ba6/core/src/main/java/org/apache/accumulo/core/data/LoadPlan.java#L93) to use a TabletFile instead of a String as part of this but don't know if making breaking changes to the constructor would be ok or not for compatibility as I'm not sure if it's public or where else it's used as I haven't looked much into Bulk Import before.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1198887028


##########
server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java:
##########
@@ -94,12 +93,12 @@ public boolean equals(Object obj) {
 
     @Override
     public int hashCode() {
-      return fileName.hashCode();
+      return file.hashCode();
     }
   }
 
-  private Map<String,List<OpenReader>> openFiles;
-  private HashMap<FileSKVIterator,String> reservedReaders;
+  private Map<TabletFile,List<OpenReader>> openFiles;

Review Comment:
   Agree that this should be fine in this case (both objects resolve to the same equals anyways) but we will definitely need to make sure things don't break when #2380 is fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1198887028


##########
server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java:
##########
@@ -94,12 +93,12 @@ public boolean equals(Object obj) {
 
     @Override
     public int hashCode() {
-      return fileName.hashCode();
+      return file.hashCode();
     }
   }
 
-  private Map<String,List<OpenReader>> openFiles;
-  private HashMap<FileSKVIterator,String> reservedReaders;
+  private Map<TabletFile,List<OpenReader>> openFiles;

Review Comment:
   Agree that this should be fine in this case (both objects resolve to the same equals anyways) but we will definitely need to make sure things don't break when #2830 is fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1197958044


##########
core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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 java.util.Objects;
+
+import org.apache.hadoop.fs.Path;
+
+public abstract class AbstractTabletFile<T extends AbstractTabletFile<T>> implements Comparable<T> {

Review Comment:
   This is a half baked thought that I am posting just for discussion, not looking for any changes in this PR.
   
   Currently we can write possibly dubious code like the following because StoredTabletFile is a child type of TabletFile.
   
   ```java
   Set<TabletFile> aSet = ...
   aSet.add(new StoredTabletFile(..));
   aSet.containst(new StoredTabletFile(..));
   ```
   
   If we made StoredTabletFile a sibling type of TabletFile by making StoredTabletFile extend AbstractTabletFile, I am wondering if it would buy us anything.  It would make the code example above not compile (the add() call will not compile, the contains call would warn).  Wondering if that would help improve code correctness or just be a waste of time, not sure w/o experimenting with the change.
   
   I noticed UnassignedTabletFile was a sibling of TabletFile and that along with #2830 led me to wonder about the usefulness of making StoredTabletFile a sibling too.
   



##########
core/src/main/java/org/apache/accumulo/core/metadata/UnassignedTabletFile.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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 java.io.File;
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class UnassignedTabletFile extends AbstractTabletFile<UnassignedTabletFile> {

Review Comment:
   You mentioned you were not sure about the name.  One thing I have found that helps me with naming sometimes is to write the documentation and see what words I use in the docs.  I went through that exercise and found I was using the word referenced, so that could be a candidate for the name.  This is also aligns with the terminology used in the accumulo GC source, it calls files in tablets references.
   
   ```suggestion
   /**
    *  A file that is not intended to be added to a tablet as a reference, within the scope of the code using this class, but needs to be passed to code that processes tablet files.  These files could be temp files or files directly created by a user for bulk import.  The file may ultimately be added to a tablet later as a new file reference, but within a different scope (process, thread, code block, method, etc) that uses a different class to represent the file in that scope.
    */
   public class UnreferencedTabletFile extends AbstractTabletFile<UnreferencedTabletFile> {
   ```



##########
server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java:
##########
@@ -94,12 +93,12 @@ public boolean equals(Object obj) {
 
     @Override
     public int hashCode() {
-      return fileName.hashCode();
+      return file.hashCode();
     }
   }
 
-  private Map<String,List<OpenReader>> openFiles;
-  private HashMap<FileSKVIterator,String> reservedReaders;
+  private Map<TabletFile,List<OpenReader>> openFiles;

Review Comment:
   Anytime we use TabletFile as key we have to be wary of #2830.  I think this is probably ok for now, but future changes to StoredTabletFile equals and hashcode could break this.



##########
core/src/main/java/org/apache/accumulo/core/metadata/TabletFile.java:
##########
@@ -155,4 +142,9 @@ public int hashCode() {
   public String toString() {
     return normalizedPath;
   }
+
+  public static TabletFile of(final Path path) {

Review Comment:
   What motivated this new method? Could we eventually make the constructor private?



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java:
##########
@@ -548,22 +548,22 @@ public SortedMap<KeyExtent,Bulk.Files> computeFileToTabletMappings(FileSystem fs
         context.instanceOperations().getSystemConfiguration(), tableProps, tableId);
 
     for (FileStatus fileStatus : files) {
-      Path filePath = fileStatus.getPath();
+      UnassignedTabletFile file = UnassignedTabletFile.of(fs, fileStatus.getPath());
       CompletableFuture<Map<KeyExtent,Bulk.FileInfo>> future = CompletableFuture.supplyAsync(() -> {
         try {
           long t1 = System.currentTimeMillis();
           List<KeyExtent> extents =
-              findOverlappingTablets(context, extentCache, filePath, fs, fileLensCache, cs);
+              findOverlappingTablets(context, extentCache, file, fs, fileLensCache, cs);
           // make sure file isn't going to too many tablets
-          checkTabletCount(maxTablets, extents.size(), filePath.toString());
-          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), filePath,
+          checkTabletCount(maxTablets, extents.size(), file.toString());
+          Map<KeyExtent,Long> estSizes = estimateSizes(context.getConfiguration(), file,
               fileStatus.getLen(), extents, fs, fileLensCache, cs);
           Map<KeyExtent,Bulk.FileInfo> pathLocations = new HashMap<>();
           for (KeyExtent ke : extents) {
-            pathLocations.put(ke, new Bulk.FileInfo(filePath, estSizes.getOrDefault(ke, 0L)));
+            pathLocations.put(ke, new Bulk.FileInfo(file.getPath(), estSizes.getOrDefault(ke, 0L)));

Review Comment:
   Looked into passing TabletFile to FileInfo.  FileInfo is used for serialization, so internally it needs a string.  Could possible pass the tablet file obj have it do the conversion in the constructor. 



##########
core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java:
##########
@@ -55,7 +55,7 @@ public class RFileOperations extends FileOperations {
 
   private static RFile.Reader getReader(FileOptions options) throws IOException {
     CachableBuilder cb = new CachableBuilder()
-        .fsPath(options.getFileSystem(), new Path(options.getFilename()), options.dropCacheBehind)
+        .fsPath(options.getFileSystem(), options.getFile().getPath(), options.dropCacheBehind)

Review Comment:
   Interesting how few changes there are to this class.



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/bulk/BulkImport.java:
##########
@@ -261,9 +262,9 @@ public MLong(long i) {
     long l;
   }
 
-  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf, Path dataFile,
-      long fileSize, Collection<KeyExtent> extents, FileSystem ns, Cache<String,Long> fileLenCache,
-      CryptoService cs) throws IOException {
+  public static Map<KeyExtent,Long> estimateSizes(AccumuloConfiguration acuConf,
+      UnassignedTabletFile dataFile, long fileSize, Collection<KeyExtent> extents, FileSystem ns,
+      Cache<String,Long> fileLenCache, CryptoService cs) throws IOException {

Review Comment:
   I was wondering if we could use a better type than String for `Cache<String,Long>`.  Looked into the code a bit an I am not sure if we could use TabletFile.  If we can not use TabletFile we may be able to created a specialized CachableBlockFile.CacheId type to use in lieu of String..  This would be a follow on issue.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#discussion_r1198089585


##########
core/src/main/java/org/apache/accumulo/core/metadata/UnassignedTabletFile.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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 java.io.File;
+import java.io.IOException;
+import java.util.Objects;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class UnassignedTabletFile extends AbstractTabletFile<UnassignedTabletFile> {

Review Comment:
   Reading through your opening comments on the PR, would also be good to add something about the lack of validation to the javadoc



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] cshannon commented on pull request #3401: Start using TabletFile instead of String for referencing data files

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on PR #3401:
URL: https://github.com/apache/accumulo/pull/3401#issuecomment-1554534539

   I just started a full IT build, if everything passes this should be good to merge.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org