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/06 13:49:56 UTC

[GitHub] [accumulo] cshannon opened a new pull request, #3385: Encapsulate file metadata inside a new object

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

   The goal of this PR is to start to move away from using a String representation everywhere for the file path reference in the metadata table 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 
   
   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 file reference and Range in the code.
   
   This PR is an incremental step and introduces an object to encapsulate the file metadata instead of using a String everywhere. There is a new object that is now used as part of `TabletFile` and `StoredTabletFile` called `TabletFileMetadataEntry` which will now represent the value that is stored in the file column. Currently the only thing it has is just a file path like before but now with the addition of this class we can add a field to it (a range or other information) and it will be much easier to update since we are not using a String everywhere.
   
   This PR doesn't go too crazy yet, it introduces the new object and updates the `StoredTabletFile` and `TabletFile` objects but many places in the code still just call the [getter](https://github.com/cshannon/accumulo/blob/991fe838c059e5f2675e4e89c2d8a33b80e58e9e/core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletFileMetadataEntry.java#L55) on the new object to get the string representation. Besides a [new constructor](https://github.com/cshannon/accumulo/blob/991fe838c059e5f2675e4e89c2d8a33b80e58e9e/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java#L44) for the new object type, a string is still allowed to be passed to a [constructor](https://github.com/cshannon/accumulo/blob/991fe838c059e5f2675e4e89c2d8a33b80e58e9e/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java#L53) for StoredTabletFile and then just gets converted to the new object but we could stop doing that in the future to prevent errors. I wasn't sure how f
 ar to take this in this PR. We could go much further with more changes (at a higher risk) and push the object as far as we can and stop using Strings as much as possible. The main issue with doing that now is it would cause a lot of cascading changes to the code from everything to file handling to external compactions, basically anything using a file reference and I'm not quite sure yet how things are going to progress with the no chop merge changes.
   
   A notable example would be changes to in `FileManager`.  Instead of [tracking](https://github.com/apache/accumulo/blob/56d49f15a05db9a46dbceb845918497760601c11/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java#L484) Strings for the files when range information is added we will need to  track the new object `TabletFileMetadataEntry`. However, I think that the changes for that may be better done in my upcoming PR for adding range information because I haven't decided yet if I plan to use `TabletFileMetadataEntry` or actually just `TabletFile` itself. It might make more sense to just track `TabletFiles` (which will now contain `TabletFileMetadataEntry`). 
   
   Anyways, I think we could probably merge this into 3.0 but could also wait to 3.1. Either way I plan to make my other no-chop merge PRs based on this and continue.


-- 
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 #3385: Encapsulate file metadata inside a new object

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


##########
core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java:
##########
@@ -33,39 +35,47 @@
  * in Upgrader9to10.upgradeRelativePaths()
  */
 public class StoredTabletFile extends TabletFile {
-  private final String metadataEntry;
+  private final TabletFileMetadataEntry metadataEntry;
+
+  /**
+   * Construct a tablet file using the {@link TabletFileMetadataEntry} from the metadata. Preserve
+   * the exact string so the entry can be deleted.
+   */
+  public StoredTabletFile(final TabletFileMetadataEntry metadataEntry) {
+    super(Objects.requireNonNull(metadataEntry).getFilePath());
+    this.metadataEntry = metadataEntry;
+  }
 
   /**
    * Construct a tablet file using the string read from the metadata. Preserve the exact string so
    * the entry can be deleted.
    */
-  public StoredTabletFile(String metadataEntry) {
-    super(new Path(metadataEntry));
-    this.metadataEntry = metadataEntry;
+  public StoredTabletFile(final String metadataEntry) {
+    this(TabletFileMetadataEntry.of(metadataEntry));
   }
 
   /**
-   * Return the exact string that is stored in the metadata table. This is important for updating
-   * and deleting metadata entries. If the exact string is not used, erroneous entries can pollute
-   * the metadata table.
+   * Return the {@link TabletFileMetadataEntry} which contains the exact string that is stored in
+   * the metadata table. This is important for updating and deleting metadata entries. If the exact
+   * string is not used, erroneous entries can pollute the metadata table.
    */
-  public String getMetaUpdateDelete() {
+  public TabletFileMetadataEntry getMetaUpdateDelete() {
     return metadataEntry;
   }
 
   /**
    * Return a new Text object of {@link #getMetaUpdateDelete()}
    */
   public Text getMetaUpdateDeleteText() {
-    return new Text(getMetaUpdateDelete());
+    return getMetaUpdateDelete().getMetaText();

Review Comment:
   There is no current difference but it seems like these methods for separate insertion and deletions were set up like this intentionally based on the comments and javadoc in case it was different in the future. I can refactor things and fix this but I wasn't quite sure the history as to why they could be different or if they will always just be the same.



-- 
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 #3385: Encapsulate file metadata inside a new object

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


##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/TabletFileMetadataEntry.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.schema;
+
+import java.util.Objects;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+
+/**
+ * Represents the exact string stored in the metadata table "file" column family. Currently the
+ * metadata only contains a file reference but could be expanded in the future.
+ */
+public class TabletFileMetadataEntry implements Comparable<TabletFileMetadataEntry> {
+
+  private final Path filePath;
+  private final String filePathString;
+
+  public TabletFileMetadataEntry(final Path filePath) {
+    this.filePath = Objects.requireNonNull(filePath);
+    // Cache the string value of the filePath, so we don't have to keep converting.
+    this.filePathString = filePath.toString();
+  }
+
+  /**
+   * The file path portion of the metadata
+   *
+   * @return The file path
+   */
+  public Path getFilePath() {
+    return filePath;
+  }
+
+  /**
+   * String representation of the file path
+   *
+   * @return file path string
+   */
+  public String getFilePathString() {
+    return filePathString;
+  }
+
+  /**
+   * Exact representation of what is in the Metadata table Currently this is just a file path but
+   * may expand
+   *
+   * @return The exact metadata string
+   */
+  public String getMetaString() {
+    return filePathString;
+  }
+
+  /**
+   * Exact {@link Text} representation of the metadat Generated from {@link #getMetaString()}
+   *
+   * @return The exact metadata as Text
+   */
+  public Text getMetaText() {
+    return new Text(filePathString);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    TabletFileMetadataEntry that = (TabletFileMetadataEntry) o;
+    return Objects.equals(filePathString, that.filePathString);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(filePathString);
+  }
+
+  @Override
+  public int compareTo(TabletFileMetadataEntry o) {
+    return filePathString.compareTo(o.filePathString);
+  }
+
+  @Override
+  public String toString() {
+    return filePathString;
+  }
+
+  /**
+   * Utility to create a new TabletFileMetadataEntry from the exact String in the Metadata table
+   *
+   * @param metadataEntry Exact string for the metadataEntry
+   * @return A TabletFileMetadataEntry created from the metadata string
+   */
+  public static TabletFileMetadataEntry of(String metadataEntry) {
+    return new TabletFileMetadataEntry(new Path(Objects.requireNonNull(metadataEntry)));

Review Comment:
   Once the string passes through Path it may no longer equal the exact string stored in the metadata table because Path may normalize the string.  For the purposes of doing later metadata updates we must keep the original string that came from the metadata table without ever passing it through path.  For example the following little program
   
   ```java
       String metaEntry = "hdfs://localhost:1/a//b//c";
       System.out.println(TabletFileMetadataEntry.of(metaEntry).getMetaString());
   ``` 
   
   will print
   
   ```
   hdfs://localhost:1/a/b/c
   ```
   
   So if we have a column qualifier with `hdfs://localhost:1/a//b//c` and we later try to update the column qualifier `hdfs://localhost:1/a/b/c` then it will not update the same key in the metadata table.  This could result in duplicating a file or failing to delete a file.
   
   We currently normalize with path before writing to the metadata table, but its tricky to make any assumptions about this.  There are least two cases to consider. First, older Accumulo code may not have normalized paths in the same way before storing them.  Second, different versions of the Hadoop Path could possibly normalize in different ways.  This is why StoredTabletFile exists, it keeps that exact string that was read from the metadata table without doing any normalization on it.
   
   I am wondering if this new class is needed.  Could we not pass TabletFile and/or StoredTabletFile around in the Accumulo code instead of string?  The only code that needs StoredTabletFile is code that updating an existing metadata entry for a file, for all other code TableFile should suffice.



-- 
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 #3385: Encapsulate file metadata inside a new object

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


##########
core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java:
##########
@@ -33,39 +35,47 @@
  * in Upgrader9to10.upgradeRelativePaths()
  */
 public class StoredTabletFile extends TabletFile {
-  private final String metadataEntry;
+  private final TabletFileMetadataEntry metadataEntry;

Review Comment:
   It's not quite the same, in TabletFile the value is actually generated and created in the constructor and in StoredTabletFile is provided. It's a bit weird with how everything was set up previously and I was trying to stay to the existing pattern but it seems like there could be some refactoring work to be done here as there does seems to be some duplication in some cases.



-- 
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] DomGarguilo commented on a diff in pull request #3385: Encapsulate file metadata inside a new object

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


##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -361,7 +364,8 @@ public void deleteScanServerFileReferences(String serverAddress, UUID scanServer
 
       int pLen = ScanServerFileReferenceSection.getRowPrefix().length();
       Set<ScanServerRefTabletFile> refsToDelete = StreamSupport.stream(scanner.spliterator(), false)

Review Comment:
   ```suggestion
         Set<ScanServerRefTabletFile> refsToDelete = scanner.stream()
   ```



##########
server/base/src/main/java/org/apache/accumulo/server/metadata/ServerAmpleImpl.java:
##########
@@ -343,7 +345,8 @@ public Stream<ScanServerRefTabletFile> getScanServerFileReferences() {
       scanner.setRange(ScanServerFileReferenceSection.getRange());
       int pLen = ScanServerFileReferenceSection.getRowPrefix().length();
       return StreamSupport.stream(scanner.spliterator(), false)

Review Comment:
   ```suggestion
         return scanner.stream().onClose(scanner::close)
   ```
   This closes the scanner once the stream is closed (can't put the scanner in try-with-resources block here since it would close the scanner before the stream is used).



-- 
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 #3385: Encapsulate file metadata inside a new object

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

   > Just to note the the `equal` and `hashCode` methods in `StoredTabletFile` and `TabletFile` should be fixed. The current behavior depends on the way it works now, Something to look out for when using those classes.
   
   I will take a look at this as well as it would be nice to fix this. I can see if i can wrap that into the same PR or make a separate one as I'm working on this.


-- 
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 #3385: Encapsulate file metadata inside a new object

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


##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java:
##########
@@ -129,10 +129,11 @@ private static class GSonData {
   public String toJson() {
     GSonData jData = new GSonData();
 
-    jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.nextFiles =
-        nextFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.tmp = compactTmpName.getMetaInsert();
+    jData.inputs = jobFiles.stream()
+        .map(tabletFile -> tabletFile.getMetaUpdateDelete().getMetaString()).collect(toList());
+    jData.nextFiles = nextFiles.stream()
+        .map(tabletFile -> tabletFile.getMetaUpdateDelete().getMetaString()).collect(toList());

Review Comment:
   Same as other comment, we could update it to be the file reference but no matter what it's going to still be a String value and will need to change 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] dlmarion commented on a diff in pull request #3385: Encapsulate file metadata inside a new object

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


##########
core/src/main/java/org/apache/accumulo/core/gc/ReferenceDirectory.java:
##########
@@ -46,8 +47,8 @@ public String getTabletDir() {
    * A Tablet directory should have a metadata entry equal to the dirName.
    */
   @Override
-  public String getMetadataEntry() {
-    if (!tabletDir.equals(metadataEntry)) {
+  public TabletFileMetadataEntry getMetadataEntry() {
+    if (!tabletDir.equals(metadataEntry.getFilePathString())) {

Review Comment:
   Since the variables are `final`, I think this check should be done in the constructor.



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java:
##########
@@ -129,10 +129,11 @@ private static class GSonData {
   public String toJson() {
     GSonData jData = new GSonData();
 
-    jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.nextFiles =
-        nextFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.tmp = compactTmpName.getMetaInsert();
+    jData.inputs = jobFiles.stream()
+        .map(tabletFile -> tabletFile.getMetaUpdateDelete().getMetaString()).collect(toList());

Review Comment:
   This breaks when/if `getMetaString` changes to include the file range.



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java:
##########
@@ -129,10 +129,11 @@ private static class GSonData {
   public String toJson() {
     GSonData jData = new GSonData();
 
-    jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.nextFiles =
-        nextFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.tmp = compactTmpName.getMetaInsert();
+    jData.inputs = jobFiles.stream()
+        .map(tabletFile -> tabletFile.getMetaUpdateDelete().getMetaString()).collect(toList());
+    jData.nextFiles = nextFiles.stream()
+        .map(tabletFile -> tabletFile.getMetaUpdateDelete().getMetaString()).collect(toList());

Review Comment:
   This breaks when/if `getMetaString` changes to include the file range.



##########
core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java:
##########
@@ -33,39 +35,47 @@
  * in Upgrader9to10.upgradeRelativePaths()
  */
 public class StoredTabletFile extends TabletFile {
-  private final String metadataEntry;
+  private final TabletFileMetadataEntry metadataEntry;

Review Comment:
   Is this variable a duplicate of `TabletFile.normalizedPath`?



##########
core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java:
##########
@@ -33,39 +35,47 @@
  * in Upgrader9to10.upgradeRelativePaths()
  */
 public class StoredTabletFile extends TabletFile {
-  private final String metadataEntry;
+  private final TabletFileMetadataEntry metadataEntry;
+
+  /**
+   * Construct a tablet file using the {@link TabletFileMetadataEntry} from the metadata. Preserve
+   * the exact string so the entry can be deleted.
+   */
+  public StoredTabletFile(final TabletFileMetadataEntry metadataEntry) {
+    super(Objects.requireNonNull(metadataEntry).getFilePath());
+    this.metadataEntry = metadataEntry;
+  }
 
   /**
    * Construct a tablet file using the string read from the metadata. Preserve the exact string so
    * the entry can be deleted.
    */
-  public StoredTabletFile(String metadataEntry) {
-    super(new Path(metadataEntry));
-    this.metadataEntry = metadataEntry;
+  public StoredTabletFile(final String metadataEntry) {
+    this(TabletFileMetadataEntry.of(metadataEntry));
   }
 
   /**
-   * Return the exact string that is stored in the metadata table. This is important for updating
-   * and deleting metadata entries. If the exact string is not used, erroneous entries can pollute
-   * the metadata table.
+   * Return the {@link TabletFileMetadataEntry} which contains the exact string that is stored in
+   * the metadata table. This is important for updating and deleting metadata entries. If the exact
+   * string is not used, erroneous entries can pollute the metadata table.
    */
-  public String getMetaUpdateDelete() {
+  public TabletFileMetadataEntry getMetaUpdateDelete() {
     return metadataEntry;
   }
 
   /**
    * Return a new Text object of {@link #getMetaUpdateDelete()}
    */
   public Text getMetaUpdateDeleteText() {
-    return new Text(getMetaUpdateDelete());
+    return getMetaUpdateDelete().getMetaText();

Review Comment:
   there is no difference between this method and `TabletFile.getMetaInsert`.



##########
server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java:
##########
@@ -345,16 +345,16 @@ public CompactionStats call() throws IOException, CompactionCanceledException {
         readers.add(reader);
 
         InterruptibleIterator iter = new ProblemReportingIterator(context, extent.tableId(),
-            mapFile.getPathStr(), false, reader);
+            mapFile.getPathStr().toString(), false, reader);
 
         iter = filesToCompact.get(mapFile).wrapFileIterator(iter);
 
         iters.add(iter);
 
       } catch (Exception e) {
 
-        ProblemReports.getInstance(context).report(
-            new ProblemReport(extent.tableId(), ProblemType.FILE_READ, mapFile.getPathStr(), e));
+        ProblemReports.getInstance(context).report(new ProblemReport(extent.tableId(),
+            ProblemType.FILE_READ, mapFile.getPathStr().toString(), e));

Review Comment:
   `getPathStr` is already a String.



##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java:
##########
@@ -129,10 +129,11 @@ private static class GSonData {
   public String toJson() {
     GSonData jData = new GSonData();
 
-    jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.nextFiles =
-        nextFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.tmp = compactTmpName.getMetaInsert();
+    jData.inputs = jobFiles.stream()
+        .map(tabletFile -> tabletFile.getMetaUpdateDelete().getMetaString()).collect(toList());
+    jData.nextFiles = nextFiles.stream()
+        .map(tabletFile -> tabletFile.getMetaUpdateDelete().getMetaString()).collect(toList());
+    jData.tmp = compactTmpName.getMetaInsert().getMetaString();

Review Comment:
   This breaks when/if `getMetaString` changes to include the file range.



##########
server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java:
##########
@@ -345,16 +345,16 @@ public CompactionStats call() throws IOException, CompactionCanceledException {
         readers.add(reader);
 
         InterruptibleIterator iter = new ProblemReportingIterator(context, extent.tableId(),
-            mapFile.getPathStr(), false, reader);
+            mapFile.getPathStr().toString(), false, reader);

Review Comment:
   `getPathStr` is already a String.



-- 
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 #3385: Encapsulate file metadata inside a new object

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


##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java:
##########
@@ -129,10 +129,11 @@ private static class GSonData {
   public String toJson() {
     GSonData jData = new GSonData();
 
-    jData.inputs = jobFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.nextFiles =
-        nextFiles.stream().map(StoredTabletFile::getMetaUpdateDelete).collect(toList());
-    jData.tmp = compactTmpName.getMetaInsert();
+    jData.inputs = jobFiles.stream()
+        .map(tabletFile -> tabletFile.getMetaUpdateDelete().getMetaString()).collect(toList());

Review Comment:
   So in this case to keep the changes smaller I was simply trying to do a 1 to 1 mapping on the update in this PR and the old version was also using getMetaUpdateDelete() which used to return a String so this is the same as before.
   
   I can probably update this to just use the file reference since that's really what it needs but ultimately all of this will need to be updated regardless with the new range in a future PR so I'm not sure how worthwhile it is to make too many changes to this right now knowing a lot more will be changing in future PRs as well.



-- 
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 #3385: Encapsulate file metadata inside a new object

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

   The full IT run passed.


-- 
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 #3385: Encapsulate file metadata inside a new object

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

   Thanks for the detailed review and information @keith-turner . Based on that (and after talking to Keith more about it in detail) I'm going to close this PR as it isn't going to help the way I thought as the normalization is going to break things and this extra object isn't buying much over just using the existing `StoredTabletFile` and `TabletFile` classes.
   
   Instead, I'm going to create a new PR that just goes ahead and starts using `StoredTabletFile` and `TabletFile` in the places in the code where we are only tracking the tablet file by a Path or a String. This is necessary because when we add Range information in the future we will need to also access the Range and not just the file path when reading.
   
   The most notable example of where things will need to change is the [FileOperations](https://github.com/apache/accumulo/blob/9c6db1f3b63f4acfcd85b83a7ec3c119c7f6f0c8/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java) class (and subclasses) and [FileManager](https://github.com/apache/accumulo/blob/9c6db1f3b63f4acfcd85b83a7ec3c119c7f6f0c8/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java) but there will be others as well. Those classes need to track the TabletFile so when we add the Range in a future PR it can use it to build the readers. I have actually already prototyped this a bit in my other [PR](https://github.com/apache/accumulo/pull/3332/files#diff-a98b98051dbb7d70a19025b944c64e4b74a08d7a9aaecc9f72b2b17be55a3008) but that PR also includes the Range changes which haven't been settled yet. Instead it makes sense to just do the initial refactoring first without the Range to keep the changes smaller.
   
   Also note that even with tracking a `TabletFile` we still 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](https://github.com/apache/accumulo/blob/9c6db1f3b63f4acfcd85b83a7ec3c119c7f6f0c8/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java#L41) a `StoredTabletFile` but when data is serialized it calls to [getMetaUpdateDelete()](https://github.com/apache/accumulo/blob/9c6db1f3b63f4acfcd85b83a7ec3c119c7f6f0c8/core/src/main/java/org/apache/accumulo/core/metadata/schema/ExternalCompactionMetadata.java#L132) 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] EdColeman commented on pull request #3385: Encapsulate file metadata inside a new object

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

   Just to note the the `equal` and `hashCode` methods in `StoredTabletFile` and `TabletFile` should be fixed. The current behavior depends on the way it works now,  Something to look out for when using those classes.


-- 
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 closed pull request #3385: Encapsulate file metadata inside a new object

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon closed pull request #3385: Encapsulate file metadata inside a new object
URL: https://github.com/apache/accumulo/pull/3385


-- 
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 #3385: Encapsulate file metadata inside a new object

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

   I just kicked off a full IT run.


-- 
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