浏览代码

HDFS-17020. RBF: mount table addAll should print failed records in std error (#5674)

Viraj Jasani 2 年之前
父节点
当前提交
3b65b5d68f
共有 11 个文件被更改,包括 58 次插入17 次删除
  1. 3 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java
  2. 3 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreMySQLImpl.java
  3. 17 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java
  5. 5 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.java
  6. 10 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntriesResponse.java
  7. 11 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntriesResponsePBImpl.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
  9. 1 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto
  10. 5 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java
  11. 1 8
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java

@@ -403,7 +403,7 @@ public abstract class StateStoreFileBaseImpl
           if (metrics != null) {
           if (metrics != null) {
             metrics.addFailure(monotonicNow() - start);
             metrics.addFailure(monotonicNow() - start);
           }
           }
-          return new StateStoreOperationResult(primaryKey);
+          return new StateStoreOperationResult(getOriginalPrimaryKey(primaryKey));
         } else {
         } else {
           LOG.debug("Not updating {}", record);
           LOG.debug("Not updating {}", record);
         }
         }
@@ -484,13 +484,13 @@ public abstract class StateStoreFileBaseImpl
     } catch (IOException e) {
     } catch (IOException e) {
       LOG.error("Cannot write {}", recordPathTemp, e);
       LOG.error("Cannot write {}", recordPathTemp, e);
       recordWrittenSuccessfully = false;
       recordWrittenSuccessfully = false;
-      failedRecordsList.add(primaryKey);
+      failedRecordsList.add(getOriginalPrimaryKey(primaryKey));
       success.set(false);
       success.set(false);
     }
     }
     // Commit
     // Commit
     if (recordWrittenSuccessfully && !rename(recordPathTemp, recordPath)) {
     if (recordWrittenSuccessfully && !rename(recordPathTemp, recordPath)) {
       LOG.error("Failed committing record into {}", recordPath);
       LOG.error("Failed committing record into {}", recordPath);
-      failedRecordsList.add(primaryKey);
+      failedRecordsList.add(getOriginalPrimaryKey(primaryKey));
       success.set(false);
       success.set(false);
     }
     }
     return null;
     return null;

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreMySQLImpl.java

@@ -187,7 +187,7 @@ public class StateStoreMySQLImpl extends StateStoreSerializableImpl {
           record.setDateModified(this.getTime());
           record.setDateModified(this.getTime());
           if (!updateRecord(tableName, primaryKey, data)) {
           if (!updateRecord(tableName, primaryKey, data)) {
             LOG.error("Cannot write {} into table {}", primaryKey, tableName);
             LOG.error("Cannot write {} into table {}", primaryKey, tableName);
-            failedRecordsKeys.add(primaryKey);
+            failedRecordsKeys.add(getOriginalPrimaryKey(primaryKey));
             success = false;
             success = false;
           }
           }
         } else {
         } else {
@@ -197,7 +197,7 @@ public class StateStoreMySQLImpl extends StateStoreSerializableImpl {
             if (metrics != null) {
             if (metrics != null) {
               metrics.addFailure(Time.monotonicNow() - start);
               metrics.addFailure(Time.monotonicNow() - start);
             }
             }
-            return new StateStoreOperationResult(primaryKey);
+            return new StateStoreOperationResult(getOriginalPrimaryKey(primaryKey));
           } else {
           } else {
             LOG.debug("Not updating {} as updates are not allowed", record);
             LOG.debug("Not updating {} as updates are not allowed", record);
           }
           }
@@ -205,7 +205,7 @@ public class StateStoreMySQLImpl extends StateStoreSerializableImpl {
       } else {
       } else {
         if (!insertRecord(tableName, primaryKey, data)) {
         if (!insertRecord(tableName, primaryKey, data)) {
           LOG.error("Cannot write {} in table {}", primaryKey, tableName);
           LOG.error("Cannot write {} in table {}", primaryKey, tableName);
-          failedRecordsKeys.add(primaryKey);
+          failedRecordsKeys.add(getOriginalPrimaryKey(primaryKey));
           success = false;
           success = false;
         }
         }
       }
       }

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.federation.store.driver.impl;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.Objects;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -105,4 +106,20 @@ public abstract class StateStoreSerializableImpl extends StateStoreBaseImpl {
     primaryKey = primaryKey.replaceAll(":", COLON_MARK);
     primaryKey = primaryKey.replaceAll(":", COLON_MARK);
     return primaryKey;
     return primaryKey;
   }
   }
+
+  /**
+   * Get the original primary key for the given state store record key. The returned
+   * key is readable as it is the original key.
+   *
+   * @param stateStoreRecordKey The record primary key stored by the state store implementations.
+   * @return The original primary key for the given record key.
+   */
+  protected static String getOriginalPrimaryKey(String stateStoreRecordKey) {
+    Objects.requireNonNull(stateStoreRecordKey,
+        "state store record key provided to getOriginalPrimaryKey should not be null");
+    stateStoreRecordKey = stateStoreRecordKey.replaceAll(SLASH_MARK, "/");
+    stateStoreRecordKey = stateStoreRecordKey.replaceAll(COLON_MARK, ":");
+    return stateStoreRecordKey;
+  }
+
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreZooKeeperImpl.java

@@ -255,7 +255,7 @@ public class StateStoreZooKeeperImpl extends StateStoreSerializableImpl {
               String recordZNode = getNodePath(znode, primaryKey);
               String recordZNode = getNodePath(znode, primaryKey);
               byte[] data = serialize(record);
               byte[] data = serialize(record);
               if (!writeNode(recordZNode, data, update, error)) {
               if (!writeNode(recordZNode, data, update, error)) {
-                failedRecordsKeys.add(primaryKey);
+                failedRecordsKeys.add(getOriginalPrimaryKey(primaryKey));
                 status.set(false);
                 status.set(false);
               }
               }
               return null;
               return null;

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/impl/MountTableStoreImpl.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.server.federation.router.RouterPermissionChecker;
 import org.apache.hadoop.hdfs.server.federation.router.RouterQuotaUsage;
 import org.apache.hadoop.hdfs.server.federation.router.RouterQuotaUsage;
 import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
 import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
 import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreOperationResult;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntriesResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntriesResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
@@ -138,6 +139,7 @@ public class MountTableStoreImpl extends MountTableStore {
     if (mountTables == null || mountTables.size() == 0) {
     if (mountTables == null || mountTables.size() == 0) {
       AddMountTableEntriesResponse response = AddMountTableEntriesResponse.newInstance();
       AddMountTableEntriesResponse response = AddMountTableEntriesResponse.newInstance();
       response.setStatus(false);
       response.setStatus(false);
+      response.setFailedRecordsKeys(Collections.emptyList());
       return response;
       return response;
     }
     }
     for (MountTable mountTable : mountTables) {
     for (MountTable mountTable : mountTables) {
@@ -145,9 +147,11 @@ public class MountTableStoreImpl extends MountTableStore {
       final String src = mountTable.getSourcePath();
       final String src = mountTable.getSourcePath();
       checkMountTablePermission(src);
       checkMountTablePermission(src);
     }
     }
-    boolean status = getDriver().putAll(mountTables, false, true).isOperationSuccessful();
+    StateStoreOperationResult result = getDriver().putAll(mountTables, false, true);
+    boolean status = result.isOperationSuccessful();
     AddMountTableEntriesResponse response = AddMountTableEntriesResponse.newInstance();
     AddMountTableEntriesResponse response = AddMountTableEntriesResponse.newInstance();
     response.setStatus(status);
     response.setStatus(status);
+    response.setFailedRecordsKeys(result.getFailedRecordsKeys());
     if (status) {
     if (status) {
       updateCacheAllRouters();
       updateCacheAllRouters();
     }
     }

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/AddMountTableEntriesResponse.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.federation.store.protocol;
 package org.apache.hadoop.hdfs.server.federation.store.protocol;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.List;
 
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -36,7 +37,16 @@ public abstract class AddMountTableEntriesResponse {
   @Unstable
   @Unstable
   public abstract boolean getStatus();
   public abstract boolean getStatus();
 
 
+  @Public
+  @Unstable
+  public abstract List<String> getFailedRecordsKeys();
+
   @Public
   @Public
   @Unstable
   @Unstable
   public abstract void setStatus(boolean result);
   public abstract void setStatus(boolean result);
+
+  @Public
+  @Unstable
+  public abstract void setFailedRecordsKeys(List<String> failedRecordsKeys);
+
 }
 }

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/AddMountTableEntriesResponsePBImpl.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb;
 package org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.List;
 
 
 import org.apache.hadoop.thirdparty.protobuf.Message;
 import org.apache.hadoop.thirdparty.protobuf.Message;
 
 
@@ -65,8 +66,18 @@ public class AddMountTableEntriesResponsePBImpl
     return this.translator.getProtoOrBuilder().getStatus();
     return this.translator.getProtoOrBuilder().getStatus();
   }
   }
 
 
+  @Override
+  public List<String> getFailedRecordsKeys() {
+    return this.translator.getProtoOrBuilder().getFailedEntriesKeysList();
+  }
+
   @Override
   @Override
   public void setStatus(boolean result) {
   public void setStatus(boolean result) {
     this.translator.getBuilder().setStatus(result);
     this.translator.getBuilder().setStatus(result);
   }
   }
+
+  @Override
+  public void setFailedRecordsKeys(List<String> failedRecordsKeys) {
+    this.translator.getBuilder().addAllFailedEntriesKeys(failedRecordsKeys);
+  }
 }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java

@@ -515,7 +515,7 @@ public class RouterAdmin extends Configured implements Tool {
         mountTable.addMountTableEntries(request);
         mountTable.addMountTableEntries(request);
     boolean added = addResponse.getStatus();
     boolean added = addResponse.getStatus();
     if (!added) {
     if (!added) {
-      System.err.println("Cannot add some or all mount points");
+      System.err.println("Cannot add mount points: " + addResponse.getFailedRecordsKeys());
     }
     }
     return added;
     return added;
   }
   }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto

@@ -168,6 +168,7 @@ message AddMountTableEntriesRequestProto {
 
 
 message AddMountTableEntriesResponseProto {
 message AddMountTableEntriesResponseProto {
   optional bool status = 1;
   optional bool status = 1;
+  repeated string failedEntriesKeys = 2;
 }
 }
 
 
 message UpdateMountTableEntryRequestProto {
 message UpdateMountTableEntryRequestProto {

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java

@@ -1882,6 +1882,8 @@ public class TestRouterAdminCLI {
 
 
   @Test
   @Test
   public void testAddMultipleMountPointsFailure() throws Exception {
   public void testAddMultipleMountPointsFailure() throws Exception {
+    System.setErr(new PrintStream(err));
+
     String[] argv =
     String[] argv =
         new String[] {"-addAll", "/testAddMultiMountPoints-01", "ns01", ",", "/dest01", ",",
         new String[] {"-addAll", "/testAddMultiMountPoints-01", "ns01", ",", "/dest01", ",",
             "/testAddMultiMountPoints-02", "ns02,ns03", "/dest02", "-order", "HASH_ALL",
             "/testAddMultiMountPoints-02", "ns02,ns03", "/dest02", "-order", "HASH_ALL",
@@ -1918,6 +1920,9 @@ public class TestRouterAdminCLI {
             "-faulttolerant"};
             "-faulttolerant"};
     // mount points were already added
     // mount points were already added
     assertNotEquals(0, ToolRunner.run(admin, argv));
     assertNotEquals(0, ToolRunner.run(admin, argv));
+
+    assertTrue("The error message should return failed entries",
+        err.toString().contains("Cannot add mount points: [/testAddMultiMountPoints-01"));
   }
   }
 
 
   private void addMountTable(String src, String nsId, String dst)
   private void addMountTable(String src, String nsId, String dst)

+ 1 - 8
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java

@@ -314,7 +314,7 @@ public class TestStateStoreDriverBase {
     StateStoreOperationResult result2 = driver.putAll(insertList.subList(0, 1), false, true);
     StateStoreOperationResult result2 = driver.putAll(insertList.subList(0, 1), false, true);
     assertFalse(result2.isOperationSuccessful());
     assertFalse(result2.isOperationSuccessful());
     assertEquals(1, result2.getFailedRecordsKeys().size());
     assertEquals(1, result2.getFailedRecordsKeys().size());
-    assertEquals(getPrimaryKey(insertList.get(0)), result2.getFailedRecordsKeys().get(0));
+    assertEquals(insertList.get(0).getPrimaryKey(), result2.getFailedRecordsKeys().get(0));
 
 
     records = driver.get(clazz);
     records = driver.get(clazz);
     assertEquals(records.getRecords().size(), 10);
     assertEquals(records.getRecords().size(), 10);
@@ -701,11 +701,4 @@ public class TestStateStoreDriverBase {
     return null;
     return null;
   }
   }
 
 
-  private static String getPrimaryKey(BaseRecord record) {
-    String primaryKey = record.getPrimaryKey();
-    primaryKey = primaryKey.replaceAll("/", "0SLASH0");
-    primaryKey = primaryKey.replaceAll(":", "_");
-    return primaryKey;
-  }
-
 }
 }