sharmaar12 commented on code in PR #7149:
URL: https://github.com/apache/hbase/pull/7149#discussion_r2244757406


##########
hbase-protocol-shaded/src/main/protobuf/server/master/Master.proto:
##########
@@ -796,6 +796,17 @@ message ModifyColumnStoreFileTrackerResponse {
   optional uint64 proc_id = 1;
 }
 
+message RefreshHFilesRequest {
+  optional TableName table_Name = 1;

Review Comment:
   Done.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java:
##########
@@ -2651,4 +2651,24 @@ List<LogEntry> getLogEntries(Set<ServerName> 
serverNames, String logType, Server
    * Get the list of cached files
    */
   List<String> getCachedFilesList(ServerName serverName) throws IOException;
+
+  /**
+   * Refresh HFiles for the table
+   * @param tableName table to refresh HFiles for

Review Comment:
   Done.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java:
##########
@@ -4543,4 +4544,61 @@ protected String getDescription() {
         }
       });
   }
+
+  public Long refreshHfiles(final TableName tableName, final long nonceGroup, 
final long nonce)
+    throws IOException {
+    // TODO Check if table exists otherwise send exception.
+    // return 121L;
+    return MasterProcedureUtil
+      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, 
nonceGroup, nonce) {
+        @Override
+        protected void run() throws IOException {
+          LOG.info("Submitting RefreshHfilesTableProcedure for a table");
+          submitProcedure(
+            new 
RefreshHFilesTableProcedure(procedureExecutor.getEnvironment(), tableName));
+        }
+
+        @Override
+        protected String getDescription() {
+          return "RefreshHfilesProcedure for a table";
+        }
+      });
+  }
+
+  public Long refreshHfiles(final String namespace, final long nonceGroup, 
final long nonce)
+    throws IOException {
+    // TODO Check if namespace exists otherwise send exception.
+    // return 122L;
+    return MasterProcedureUtil
+      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, 
nonceGroup, nonce) {
+        @Override
+        protected void run() throws IOException {
+          LOG.info("Submitting RefreshHfilesProcedure for namespace");

Review Comment:
   Done.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java:
##########
@@ -4543,4 +4544,61 @@ protected String getDescription() {
         }
       });
   }
+
+  public Long refreshHfiles(final TableName tableName, final long nonceGroup, 
final long nonce)
+    throws IOException {
+    // TODO Check if table exists otherwise send exception.
+    // return 121L;
+    return MasterProcedureUtil
+      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, 
nonceGroup, nonce) {
+        @Override
+        protected void run() throws IOException {
+          LOG.info("Submitting RefreshHfilesTableProcedure for a table");

Review Comment:
   Done.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java:
##########
@@ -4557,4 +4559,49 @@ List<String>> adminCall(controller, stub, 
request.build(),
           resp -> resp.getCachedFilesList()))
       .serverName(serverName).call();
   }
+
+  @Override
+  public CompletableFuture<Long> refreshHFiles(final TableName tableName) {
+    // Request builder
+    RefreshHFilesRequest.Builder request = RefreshHFilesRequest.newBuilder();
+    request.setTableName(ProtobufUtil.toProtoTableName(tableName));
+    // Set nonce
+    // Master Caller
+    return this.<Long> newMasterCaller()
+      .action((controller, stub) -> this.<RefreshHFilesRequest, 
RefreshHFilesResponse, Long> call(
+        controller, stub, request.build(), 
MasterService.Interface::refreshHFiles,
+        RefreshHFilesResponse::getProcId))
+      .call();
+    // return CompletableFuture.completedFuture(null);
+  }
+
+  @Override
+  public CompletableFuture<Long> refreshHFiles(final String namespace) {
+    // Request builder
+    RefreshHFilesRequest.Builder request = RefreshHFilesRequest.newBuilder();
+    request.setNamespace(namespace);
+    // Set nonce
+    // Master Caller
+    return this.<Long> newMasterCaller()
+      .action((controller, stub) -> this.<RefreshHFilesRequest, 
RefreshHFilesResponse, Long> call(
+        controller, stub, request.build(), 
MasterService.Interface::refreshHFiles,
+        RefreshHFilesResponse::getProcId))
+      .call();
+    // return CompletableFuture.completedFuture(null);
+  }
+
+  @Override
+  public CompletableFuture<Long> refreshHFiles() {
+    // Request builder
+    RefreshHFilesRequest.Builder request = RefreshHFilesRequest.newBuilder();
+    // Set nonce
+    request.setNonceGroup(ng.getNonceGroup()).setNonce(ng.newNonce());
+    // Master Caller
+    return this.<Long> newMasterCaller()
+      .action((controller, stub) -> this.<RefreshHFilesRequest, 
RefreshHFilesResponse, Long> call(
+        controller, stub, request.build(), 
MasterService.Interface::refreshHFiles,
+        RefreshHFilesResponse::getProcId))
+      .call();
+    // return CompletableFuture.completedFuture(null);

Review Comment:
   Done.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java:
##########
@@ -4557,4 +4559,49 @@ List<String>> adminCall(controller, stub, 
request.build(),
           resp -> resp.getCachedFilesList()))
       .serverName(serverName).call();
   }
+
+  @Override
+  public CompletableFuture<Long> refreshHFiles(final TableName tableName) {
+    // Request builder
+    RefreshHFilesRequest.Builder request = RefreshHFilesRequest.newBuilder();
+    request.setTableName(ProtobufUtil.toProtoTableName(tableName));
+    // Set nonce
+    // Master Caller
+    return this.<Long> newMasterCaller()
+      .action((controller, stub) -> this.<RefreshHFilesRequest, 
RefreshHFilesResponse, Long> call(
+        controller, stub, request.build(), 
MasterService.Interface::refreshHFiles,
+        RefreshHFilesResponse::getProcId))
+      .call();
+    // return CompletableFuture.completedFuture(null);
+  }
+
+  @Override
+  public CompletableFuture<Long> refreshHFiles(final String namespace) {
+    // Request builder
+    RefreshHFilesRequest.Builder request = RefreshHFilesRequest.newBuilder();
+    request.setNamespace(namespace);
+    // Set nonce
+    // Master Caller
+    return this.<Long> newMasterCaller()
+      .action((controller, stub) -> this.<RefreshHFilesRequest, 
RefreshHFilesResponse, Long> call(
+        controller, stub, request.build(), 
MasterService.Interface::refreshHFiles,
+        RefreshHFilesResponse::getProcId))
+      .call();
+    // return CompletableFuture.completedFuture(null);

Review Comment:
   Done.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesTableProcedure.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://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.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshHFilesTableProcedureState;
+
[email protected]
+public class RefreshHFilesTableProcedure
+  extends AbstractStateMachineTableProcedure<RefreshHFilesTableProcedureState> 
{
+  private static final Logger LOG = 
LoggerFactory.getLogger(RefreshHFilesTableProcedure.class);
+
+  private TableName tableName;
+  private String namespace;
+
+  public RefreshHFilesTableProcedure() {
+    super();
+  }
+
+  public RefreshHFilesTableProcedure(MasterProcedureEnv env) {
+    super(env);
+  }
+
+  public RefreshHFilesTableProcedure(MasterProcedureEnv env, TableName 
tableName) {
+    super(env);
+    this.tableName = tableName;
+  }
+
+  public RefreshHFilesTableProcedure(MasterProcedureEnv env, String namespace) 
{
+    super(env);
+    this.namespace = namespace;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.REFRESH_HFILES;
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  protected RefreshHFilesTableProcedureState getInitialState() {
+    return RefreshHFilesTableProcedureState.REFRESH_HFILES_PREPARE;
+  }
+
+  @Override
+  protected int getStateId(RefreshHFilesTableProcedureState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected RefreshHFilesTableProcedureState getState(int stateId) {
+    return RefreshHFilesTableProcedureState.forNumber(stateId);
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env,
+    RefreshHFilesTableProcedureState RefreshHFilesTableProcedureState)

Review Comment:
   Done.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RefreshHFilesCallable.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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
+ *
+ *     http://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.hadoop.hbase.regionserver;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.procedure2.BaseRSProcedureCallable;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+
[email protected]
+public class RefreshHFilesCallable extends BaseRSProcedureCallable {
+  private static final Logger LOG = 
LoggerFactory.getLogger(RefreshHFilesCallable.class);
+
+  private RegionInfo regionInfo;
+
+  @Override
+  protected void doCall() throws Exception {
+    HRegion region = rs.getRegion(regionInfo.getEncodedName());
+    LOG.debug("Starting refrehHfiles operation on region {}", region);

Review Comment:
   Done.



##########
hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java:
##########
@@ -4557,4 +4559,49 @@ List<String>> adminCall(controller, stub, 
request.build(),
           resp -> resp.getCachedFilesList()))
       .serverName(serverName).call();
   }
+
+  @Override
+  public CompletableFuture<Long> refreshHFiles(final TableName tableName) {
+    // Request builder
+    RefreshHFilesRequest.Builder request = RefreshHFilesRequest.newBuilder();
+    request.setTableName(ProtobufUtil.toProtoTableName(tableName));
+    // Set nonce
+    // Master Caller
+    return this.<Long> newMasterCaller()
+      .action((controller, stub) -> this.<RefreshHFilesRequest, 
RefreshHFilesResponse, Long> call(
+        controller, stub, request.build(), 
MasterService.Interface::refreshHFiles,
+        RefreshHFilesResponse::getProcId))
+      .call();
+    // return CompletableFuture.completedFuture(null);

Review Comment:
   Done.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java:
##########
@@ -3661,4 +3661,28 @@ public FlushTableResponse flushTable(RpcController 
controller, FlushTableRequest
       throw new ServiceException(ioe);
     }
   }
+
+  @Override
+  public MasterProtos.RefreshHFilesResponse refreshHFiles(RpcController 
controller,
+    MasterProtos.RefreshHFilesRequest request) throws ServiceException {
+    // TODO Check if table exists otherwise send exception.
+    try {
+      Long procId;
+      if (request.hasTableName()) { // if we have provided table name as 
parameter
+        // refreshHfiles for the given user tables
+        procId = 
server.refreshHfiles(ProtobufUtil.toTableName(request.getTableName()),
+          request.getNonceGroup(), request.getNonce());
+      } else if (request.hasNamespace()) { // if we have provided namespace as 
parameter
+        // refreshHfiles for all the user tables under the namespace
+        procId =
+          server.refreshHfiles(request.getNamespace(), 
request.getNonceGroup(), request.getNonce());
+      } else { // When no parameter is provided
+        // refreshHfiles for all the user tables in HBase
+        procId = server.refreshHfiles(request.getNonceGroup(), 
request.getNonce());
+      }

Review Comment:
   Done.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java:
##########
@@ -4543,4 +4544,61 @@ protected String getDescription() {
         }
       });
   }
+
+  public Long refreshHfiles(final TableName tableName, final long nonceGroup, 
final long nonce)
+    throws IOException {
+    // TODO Check if table exists otherwise send exception.
+    // return 121L;
+    return MasterProcedureUtil
+      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, 
nonceGroup, nonce) {
+        @Override
+        protected void run() throws IOException {
+          LOG.info("Submitting RefreshHfilesTableProcedure for a table");
+          submitProcedure(
+            new 
RefreshHFilesTableProcedure(procedureExecutor.getEnvironment(), tableName));
+        }
+
+        @Override
+        protected String getDescription() {
+          return "RefreshHfilesProcedure for a table";
+        }
+      });
+  }
+
+  public Long refreshHfiles(final String namespace, final long nonceGroup, 
final long nonce)
+    throws IOException {
+    // TODO Check if namespace exists otherwise send exception.
+    // return 122L;
+    return MasterProcedureUtil
+      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, 
nonceGroup, nonce) {
+        @Override
+        protected void run() throws IOException {
+          LOG.info("Submitting RefreshHfilesProcedure for namespace");
+          submitProcedure(
+            new 
RefreshHFilesTableProcedure(procedureExecutor.getEnvironment(), namespace));
+        }
+
+        @Override
+        protected String getDescription() {
+          return "RefreshHfilesProcedure for namespace";
+        }
+      });
+  }
+
+  public Long refreshHfiles(final long nonceGroup, final long nonce) throws 
IOException {
+    // return 123L;

Review Comment:
   Done.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesRegionProcedure.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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
+ *
+ *     http://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.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.procedure2.FailedRemoteDispatchException;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
+import org.apache.hadoop.hbase.regionserver.RefreshHFilesCallable;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+
[email protected]
+public class RefreshHFilesRegionProcedure extends Procedure<MasterProcedureEnv>
+  implements TableProcedureInterface,
+  RemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv, ServerName> {
+  private RegionInfo region;
+
+  public RefreshHFilesRegionProcedure() {
+  }
+
+  public RefreshHFilesRegionProcedure(RegionInfo region) {
+    this.region = region;
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) 
throws IOException {
+    MasterProcedureProtos.RefreshHFilesRegionProcedureStateData data =
+      
serializer.deserialize(MasterProcedureProtos.RefreshHFilesRegionProcedureStateData.class);
+    this.region = ProtobufUtil.toRegionInfo(data.getRegion());
+    // TODO Get the Data from region server
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) 
throws IOException {
+    MasterProcedureProtos.RefreshHFilesRegionProcedureStateData.Builder 
builder =
+      MasterProcedureProtos.RefreshHFilesRegionProcedureStateData.newBuilder();
+    builder.setRegion(ProtobufUtil.toRegionInfo(region));
+    // TODO add data that you want to pass to region server
+    serializer.serialize(builder.build());
+  }
+
+  @Override
+  protected boolean abort(MasterProcedureEnv env) {
+    return false;
+  }
+
+  @Override
+  protected void rollback(MasterProcedureEnv env) throws IOException, 
InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env)
+    throws ProcedureYieldException, ProcedureSuspendedException, 
InterruptedException {
+    RegionStates regionStates = env.getAssignmentManager().getRegionStates();
+    RegionStateNode regionNode = regionStates.getRegionStateNode(region);
+
+    ServerName targetServer = regionNode.getRegionLocation();
+
+    try {
+      env.getRemoteDispatcher().addOperationToNode(targetServer, this);
+    } catch (FailedRemoteDispatchException e) {
+      throw new ProcedureSuspendedException();
+    }
+
+    return null;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.REFRESH_HFILES;
+  }
+
+  @Override
+  public TableName getTableName() {
+    return region.getTable();
+  }
+
+  @Override
+  public void remoteOperationFailed(MasterProcedureEnv env, 
RemoteProcedureException error) {
+    // TODO redo the same thing again till retry count else send the error to 
client.
+  }
+
+  public void remoteOperationCompleted(MasterProcedureEnv env) {
+    // TODO Do nothing just LOG completed successfully as everything is 
completed successfully
+  }
+
+  @Override
+  public void remoteCallFailed(MasterProcedureEnv env, ServerName serverName, 
IOException e) {
+    // TODO redo the same thing again till retry count else send the error to 
client.
+  }
+
+  @Override
+  public Optional<RemoteProcedureDispatcher.RemoteOperation> 
remoteCallBuild(MasterProcedureEnv env,
+    ServerName serverName) {
+    MasterProcedureProtos.RefreshHFilesRegionParameter.Builder builder =
+      MasterProcedureProtos.RefreshHFilesRegionParameter.newBuilder();
+    builder.setRegion(ProtobufUtil.toRegionInfo(region));
+    // TODO Add logic on how to build remote call

Review Comment:
   Yes. Thanks for pointing it out.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesRegionProcedure.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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
+ *
+ *     http://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.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.procedure2.FailedRemoteDispatchException;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
+import org.apache.hadoop.hbase.regionserver.RefreshHFilesCallable;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+
[email protected]
+public class RefreshHFilesRegionProcedure extends Procedure<MasterProcedureEnv>
+  implements TableProcedureInterface,
+  RemoteProcedureDispatcher.RemoteProcedure<MasterProcedureEnv, ServerName> {
+  private RegionInfo region;
+
+  public RefreshHFilesRegionProcedure() {
+  }
+
+  public RefreshHFilesRegionProcedure(RegionInfo region) {
+    this.region = region;
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) 
throws IOException {
+    MasterProcedureProtos.RefreshHFilesRegionProcedureStateData data =
+      
serializer.deserialize(MasterProcedureProtos.RefreshHFilesRegionProcedureStateData.class);
+    this.region = ProtobufUtil.toRegionInfo(data.getRegion());
+    // TODO Get the Data from region server
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) 
throws IOException {
+    MasterProcedureProtos.RefreshHFilesRegionProcedureStateData.Builder 
builder =
+      MasterProcedureProtos.RefreshHFilesRegionProcedureStateData.newBuilder();
+    builder.setRegion(ProtobufUtil.toRegionInfo(region));
+    // TODO add data that you want to pass to region server
+    serializer.serialize(builder.build());
+  }
+
+  @Override
+  protected boolean abort(MasterProcedureEnv env) {
+    return false;
+  }
+
+  @Override
+  protected void rollback(MasterProcedureEnv env) throws IOException, 
InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env)
+    throws ProcedureYieldException, ProcedureSuspendedException, 
InterruptedException {
+    RegionStates regionStates = env.getAssignmentManager().getRegionStates();
+    RegionStateNode regionNode = regionStates.getRegionStateNode(region);
+
+    ServerName targetServer = regionNode.getRegionLocation();
+
+    try {
+      env.getRemoteDispatcher().addOperationToNode(targetServer, this);
+    } catch (FailedRemoteDispatchException e) {
+      throw new ProcedureSuspendedException();
+    }
+
+    return null;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.REFRESH_HFILES;
+  }
+
+  @Override
+  public TableName getTableName() {
+    return region.getTable();
+  }
+
+  @Override
+  public void remoteOperationFailed(MasterProcedureEnv env, 
RemoteProcedureException error) {
+    // TODO redo the same thing again till retry count else send the error to 
client.

Review Comment:
   Yes. We will be implementing it here. I have added the overall structure for 
only valid case so that team can validate if this is the correct approach to 
take. I will be adding the error scenarios and TODOs.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RefreshHFilesTableProcedure.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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
+ *
+ *     http://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.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshHFilesTableProcedureState;
+
[email protected]
+public class RefreshHFilesTableProcedure
+  extends AbstractStateMachineTableProcedure<RefreshHFilesTableProcedureState> 
{
+  private static final Logger LOG = 
LoggerFactory.getLogger(RefreshHFilesTableProcedure.class);
+
+  private TableName tableName;
+  private String namespace;
+
+  public RefreshHFilesTableProcedure() {
+    super();
+  }
+
+  public RefreshHFilesTableProcedure(MasterProcedureEnv env) {
+    super(env);
+  }
+
+  public RefreshHFilesTableProcedure(MasterProcedureEnv env, TableName 
tableName) {
+    super(env);
+    this.tableName = tableName;
+  }
+
+  public RefreshHFilesTableProcedure(MasterProcedureEnv env, String namespace) 
{
+    super(env);
+    this.namespace = namespace;
+  }
+
+  @Override
+  public TableOperationType getTableOperationType() {
+    return TableOperationType.REFRESH_HFILES;
+  }
+
+  @Override
+  public TableName getTableName() {
+    return tableName;
+  }
+
+  @Override
+  protected RefreshHFilesTableProcedureState getInitialState() {
+    return RefreshHFilesTableProcedureState.REFRESH_HFILES_PREPARE;
+  }
+
+  @Override
+  protected int getStateId(RefreshHFilesTableProcedureState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected RefreshHFilesTableProcedureState getState(int stateId) {
+    return RefreshHFilesTableProcedureState.forNumber(stateId);
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env,
+    RefreshHFilesTableProcedureState RefreshHFilesTableProcedureState)
+    throws IOException, InterruptedException {
+    // Refresh HFiles is idempotent operation hence rollback is not needed
+    LOG.trace("Rollback not implemented for RefreshHFilesTableProcedure state: 
{}",
+      RefreshHFilesTableProcedureState);
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env,
+    RefreshHFilesTableProcedureState RefreshHFilesTableProcedureState) {

Review Comment:
   Done.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to