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/04/22 16:05:02 UTC

[GitHub] [accumulo] cshannon opened a new pull request, #3331: Reuse already read lastLocation information during location updates

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

   This change will reuse previously read lastLocation information when ManagerMetadataUtil updates last location to prevent unnecessary metadata reads. If the value is not known then a metadata read will be performed like before.
   
   This closes #3301


-- 
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] ctubbsii commented on pull request #3331: Reuse already read lastLocation information during location updates

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

   The original ticket was targeting 2.1.1, but this PR was made against the main branch instead of the 2.1 branch. I'm not sure what the desired intent is, but these should probably match, or we'll need another PR to backport it if they remain different. If you want to avoid creating a new PR, this PR can be updated by editing the base branch to 2.1, and then rebasing (cherry-picking) the commits from the HEAD of the main branch onto the head of the 2.1 branch and force pushing the branch that backs this PR.


-- 
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 #3331: Reuse already read lastLocation information during location updates

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


##########
server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java:
##########
@@ -924,7 +925,8 @@ private void getAssignmentsFromBalancer(TabletLists tLists,
                   lastLocation.getHostPort());
             }
 
-            tLists.assignments.add(new Assignment(assignment.getKey(), assignment.getValue()));
+            tLists.assignments.add(new Assignment(assignment.getKey(), assignment.getValue(),
+                lastLocation != null ? Location.last(lastLocation) : null));

Review Comment:
   I updated the PR to use the lastLocation value that is part of TabletLocationState when doing the metadata updates. Before the change the value was always coming from getServer() which is returning getLocation() and that could be current, future or last so this makes sure to always use the last value from metadata. The old value (getServer()) is still being tracked now as I created an UnassignedTablet class to track both that value and lastLocation as they are used for different things.



-- 
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 #3331: Reuse already read lastLocation information during location updates

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


##########
server/base/src/main/java/org/apache/accumulo/server/manager/state/UnassignedTablet.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.server.manager.state;
+
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+
+import com.google.common.base.Preconditions;
+
+public class UnassignedTablet {
+  private final Location location;
+  private final Location lastLocation;
+
+  public UnassignedTablet(Location location, Location lastLocation) {
+    Preconditions.checkArgument(
+        lastLocation == null || lastLocation.getType() == TabletMetadata.LocationType.LAST);
+    this.location = location;

Review Comment:
   If the following is also an expectation, could check it.
   
   ```suggestion
        Preconditions.checkArgument(location == null || location.getType() == TabletMetadata.LocationType.CURRENT || location.getType() == TabletMetadata.LocationType.FUTURE)
       this.location = location;
   ```



-- 
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 #3331: Reuse already read lastLocation information during location updates

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


##########
server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java:
##########
@@ -257,14 +257,23 @@ public static Optional<StoredTabletFile> updateTabletDataFile(ServerContext cont
    * @param tabletMutator The mutator being built
    * @param extent The tablet extent
    * @param location The new location
+   * @param prevLastLocation The previous last location, if known
    */
   public static void updateLastForAssignmentMode(ClientContext context, Ample ample,
-      Ample.TabletMutator tabletMutator, KeyExtent extent, TServerInstance location) {
+      Ample.TabletMutator tabletMutator, KeyExtent extent, TServerInstance location,
+      Location prevLastLocation) {
     // if the location mode is assignment, then preserve the current location in the last
     // location value
     if ("assignment".equals(context.getConfiguration().get(Property.TSERV_LAST_LOCATION_MODE))) {
-      TabletMetadata lastMetadata = ample.readTablet(extent, TabletMetadata.ColumnType.LAST);
-      Location lastLocation = (lastMetadata == null ? null : lastMetadata.getLast());
+      // If prevLastLocation is not provided then look it up from metadata
+      final Location lastLocation = Optional.ofNullable(prevLastLocation).orElseGet(() -> {

Review Comment:
   Could use a conditional statement here and avoid the object creation.
   
   ```
     Location lastLocation = prevLastLocation;
     if (lastLocation == null) {
           log.trace(
               "Previous last location for {} not provided to updateLastForAssignmentMode, loading from metadata.", extent);
           final TabletMetadata lastMetadata =
               ample.readTablet(extent, TabletMetadata.ColumnType.LAST);
           lastLocation = lastMetadata == null ? null : lastMetadata.getLast();
     }
   ```



-- 
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 #3331: Reuse already read lastLocation information during location updates

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


##########
server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java:
##########
@@ -253,18 +253,16 @@ public static Optional<StoredTabletFile> updateTabletDataFile(ServerContext cont
    * last location if needed and set the new last location
    *
    * @param context The server context
-   * @param ample The metadata persistence layer
    * @param tabletMutator The mutator being built
-   * @param extent The tablet extent
    * @param location The new location
+   * @param lastLocation The previous last location, which may be null
    */
-  public static void updateLastForAssignmentMode(ClientContext context, Ample ample,
-      Ample.TabletMutator tabletMutator, KeyExtent extent, TServerInstance location) {
+  public static void updateLastForAssignmentMode(ClientContext context,
+      Ample.TabletMutator tabletMutator, TServerInstance location, Location lastLocation) {
     // if the location mode is assignment, then preserve the current location in the last
     // location value
+    log.info("Loaded Last location: {}", lastLocation);

Review Comment:
   Yep, I forgot to take this out, I will remove it.



-- 
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 #3331: Reuse already read lastLocation information during location updates

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


-- 
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 #3331: Reuse already read lastLocation information during location updates

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


##########
server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java:
##########
@@ -924,7 +925,8 @@ private void getAssignmentsFromBalancer(TabletLists tLists,
                   lastLocation.getHostPort());
             }
 
-            tLists.assignments.add(new Assignment(assignment.getKey(), assignment.getValue()));
+            tLists.assignments.add(new Assignment(assignment.getKey(), assignment.getValue(),
+                lastLocation != null ? Location.last(lastLocation) : null));

Review Comment:
   Looking around in the code I am not entirely sure `lastLocation` is actually a location from the last column. Not sure what the value in the map is for ATM.  Entries are added to the unassigned map in the following method and that does not look like its adding things from the last column to the map.
   
   https://github.com/apache/accumulo/blob/9c459ce8f06d9d9e04280d020ea091fedf13749b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java#L383-L399



##########
server/base/src/main/java/org/apache/accumulo/server/manager/state/Assignment.java:
##########
@@ -20,13 +20,16 @@
 
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
 
 public class Assignment {
-  public KeyExtent tablet;
-  public TServerInstance server;
+  public final KeyExtent tablet;
+  public final TServerInstance server;
+  public final Location lastLocation;
 
-  public Assignment(KeyExtent tablet, TServerInstance server) {
+  public Assignment(KeyExtent tablet, TServerInstance server, Location lastLocation) {

Review Comment:
   ```suggestion
     public Assignment(KeyExtent tablet, TServerInstance server, Location lastLocation) {
       Preconditions.checkArgument(lastLocation == null || lastLocation.getType() == TabletMetadata.LocationType.LAST);
   ```



##########
server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java:
##########
@@ -253,18 +253,16 @@ public static Optional<StoredTabletFile> updateTabletDataFile(ServerContext cont
    * last location if needed and set the new last location
    *
    * @param context The server context
-   * @param ample The metadata persistence layer
    * @param tabletMutator The mutator being built
-   * @param extent The tablet extent
    * @param location The new location
+   * @param lastLocation The previous last location, which may be null
    */
-  public static void updateLastForAssignmentMode(ClientContext context, Ample ample,
-      Ample.TabletMutator tabletMutator, KeyExtent extent, TServerInstance location) {
+  public static void updateLastForAssignmentMode(ClientContext context,
+      Ample.TabletMutator tabletMutator, TServerInstance location, Location lastLocation) {
     // if the location mode is assignment, then preserve the current location in the last
     // location value
+    log.info("Loaded Last location: {}", lastLocation);

Review Comment:
   Was this added for your own debugging?  



##########
server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java:
##########
@@ -253,18 +253,16 @@ public static Optional<StoredTabletFile> updateTabletDataFile(ServerContext cont
    * last location if needed and set the new last location
    *
    * @param context The server context
-   * @param ample The metadata persistence layer
    * @param tabletMutator The mutator being built
-   * @param extent The tablet extent
    * @param location The new location
+   * @param lastLocation The previous last location, which may be null
    */
-  public static void updateLastForAssignmentMode(ClientContext context, Ample ample,
-      Ample.TabletMutator tabletMutator, KeyExtent extent, TServerInstance location) {
+  public static void updateLastForAssignmentMode(ClientContext context,
+      Ample.TabletMutator tabletMutator, TServerInstance location, Location lastLocation) {

Review Comment:
   ```suggestion
         Ample.TabletMutator tabletMutator, TServerInstance location, Location lastLocation) {
           Preconditions.checkArgument(lastLocation == null || lastLocation.getType() == TabletMetadata.LocationType.LAST);
   ```



-- 
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 #3331: Reuse already read lastLocation information during location updates

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

   @keith-turner - This is ready for another review, I simplified things and removed the attempt to read the metadata if the passed in lastLocation value is null and updated the Assignment class to only have one constructor that requires lastLocation now so we don't miss passing in the value. All the sunny tests passed and I kicked off another full IT build 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] keith-turner commented on a diff in pull request #3331: Reuse already read lastLocation information during location updates

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


##########
server/base/src/main/java/org/apache/accumulo/server/manager/state/UnassignedTablet.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.server.manager.state;
+
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+
+import com.google.common.base.Preconditions;
+
+public class UnassignedTablet {
+  private final Location location;
+  private final Location lastLocation;
+
+  public UnassignedTablet(Location location, Location lastLocation) {
+    Preconditions.checkArgument(
+        lastLocation == null || lastLocation.getType() == TabletMetadata.LocationType.LAST);
+    this.location = location;

Review Comment:
   If the following is also en expecation, could check it.
   
   ```suggestion
        Preconditions.checkArgument(location == null || location.getType() == TabletMetadata.LocationType.CURRENT || location.getType() == TabletMetadata.LocationType.FUTURE)
       this.location = location;
   ```



##########
server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java:
##########
@@ -240,7 +242,7 @@ public void run() {
             return mStats != null ? mStats : new MergeStats(new MergeInfo());
           });
           TabletGoalState goal = manager.getGoalState(tls, mergeStats.getMergeInfo());
-          TServerInstance location = tls.getServer();
+          Location location = tls.getLocation();

Review Comment:
   Later in the code the location is logged and the Location class does not seem to have a toString method, so with this change will need to add a toString method to the Location class or change the log stmt.



##########
server/base/src/main/java/org/apache/accumulo/server/manager/state/UnassignedTablet.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.server.manager.state;
+
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+
+import com.google.common.base.Preconditions;
+
+public class UnassignedTablet {

Review Comment:
   I like this new class it makes the existing code more clear.  I didn't see anything logging this class, but not sure its not logged. If its logged anywhere it would need a toString.



-- 
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 #3331: Reuse already read lastLocation information during location updates

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

   @keith-turner - It seemed like the best thing to do here was to add a new field to the `Assignment` class to pass the previously read last location data if known. This was nice as I didn't have to change method signatures and also works for the case where a list of assignments are passed. I added a bunch of mock tests to verify that the new behavior is correct in terms of when the update will do a metadata read or skip it. I ran through the sunny tests and they all passed but I will also kick off a full IT build.


-- 
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 #3331: Reuse already read lastLocation information during location updates

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


##########
server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java:
##########
@@ -257,14 +257,23 @@ public static Optional<StoredTabletFile> updateTabletDataFile(ServerContext cont
    * @param tabletMutator The mutator being built
    * @param extent The tablet extent
    * @param location The new location
+   * @param prevLastLocation The previous last location, if known
    */
   public static void updateLastForAssignmentMode(ClientContext context, Ample ample,
-      Ample.TabletMutator tabletMutator, KeyExtent extent, TServerInstance location) {
+      Ample.TabletMutator tabletMutator, KeyExtent extent, TServerInstance location,
+      Location prevLastLocation) {
     // if the location mode is assignment, then preserve the current location in the last
     // location value
     if ("assignment".equals(context.getConfiguration().get(Property.TSERV_LAST_LOCATION_MODE))) {
-      TabletMetadata lastMetadata = ample.readTablet(extent, TabletMetadata.ColumnType.LAST);
-      Location lastLocation = (lastMetadata == null ? null : lastMetadata.getLast());
+      // If prevLastLocation is not provided then look it up from metadata
+      final Location lastLocation = Optional.ofNullable(prevLastLocation).orElseGet(() -> {
+        log.trace(
+            "Previous last location for {} not provided to updateLastForAssignmentMode, loading from metadata.",
+            extent);
+        final TabletMetadata lastMetadata =

Review Comment:
   If the last was not present when we read earlier, then we probably do not need to read it again.  Its tricky though because null can have two meanings, it can mean it was never read previously or it was read previously and was not present.  One simplification is that if we assume that all code calling this method has already tried to read the last location, then we never need to read it here when null.



-- 
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 #3331: Reuse already read lastLocation information during location updates

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

   > The original ticket was targeting 2.1.1, but this PR was made against the main branch instead of the 2.1 branch. I'm not sure what the desired intent is, but these should probably match, or we'll need another PR to backport it if they remain different. If you want to avoid creating a new PR, this PR can be updated by editing the base branch to 2.1, and then rebasing (cherry-picking) the commits from the HEAD of the main branch onto the head of the 2.1 branch and force pushing the branch that backs this PR.
   
   I actually didn't need to do anything here but switch the target merge branch in my PR to 2.1. I actually already did base this PR off the 2.1 branch (I thought I did) but when I created the PR I just mistakenly picked the wrong target as it defaulted to main and I didn't notice.


-- 
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 #3331: Reuse already read lastLocation information during location updates

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

   I ran a full IT build against this PR and it 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 a diff in pull request #3331: Reuse already read lastLocation information during location updates

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


##########
server/base/src/main/java/org/apache/accumulo/server/manager/state/UnassignedTablet.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.server.manager.state;
+
+import org.apache.accumulo.core.metadata.TServerInstance;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location;
+
+import com.google.common.base.Preconditions;
+
+public class UnassignedTablet {

Review Comment:
   I'll add toString() and also equals/hashcode so it's there for the future if we need it.



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