001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.procedure.flush; 019 020import java.io.IOException; 021import java.util.HashMap; 022import java.util.HashSet; 023import java.util.List; 024import java.util.Map; 025import java.util.Set; 026import java.util.concurrent.ThreadPoolExecutor; 027import org.apache.hadoop.conf.Configuration; 028import org.apache.hadoop.hbase.HBaseInterfaceAudience; 029import org.apache.hadoop.hbase.HConstants; 030import org.apache.hadoop.hbase.ServerName; 031import org.apache.hadoop.hbase.TableName; 032import org.apache.hadoop.hbase.client.RegionInfo; 033import org.apache.hadoop.hbase.errorhandling.ForeignException; 034import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; 035import org.apache.hadoop.hbase.master.MasterCoprocessorHost; 036import org.apache.hadoop.hbase.master.MasterServices; 037import org.apache.hadoop.hbase.master.MetricsMaster; 038import org.apache.hadoop.hbase.procedure.MasterProcedureManager; 039import org.apache.hadoop.hbase.procedure.Procedure; 040import org.apache.hadoop.hbase.procedure.ProcedureCoordinator; 041import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; 042import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator; 043import org.apache.hadoop.hbase.security.User; 044import org.apache.hadoop.hbase.security.access.AccessChecker; 045import org.apache.hadoop.hbase.util.Pair; 046import org.apache.yetus.audience.InterfaceAudience; 047import org.apache.zookeeper.KeeperException; 048import org.slf4j.Logger; 049import org.slf4j.LoggerFactory; 050 051import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 052 053import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; 054import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; 055 056@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) 057public class MasterFlushTableProcedureManager extends MasterProcedureManager { 058 059 public static final String FLUSH_TABLE_PROCEDURE_SIGNATURE = "flush-table-proc"; 060 061 private static final String FLUSH_TIMEOUT_MILLIS_KEY = "hbase.flush.master.timeoutMillis"; 062 private static final int FLUSH_TIMEOUT_MILLIS_DEFAULT = 60000; 063 private static final String FLUSH_WAKE_MILLIS_KEY = "hbase.flush.master.wakeMillis"; 064 private static final int FLUSH_WAKE_MILLIS_DEFAULT = 500; 065 066 private static final String FLUSH_PROC_POOL_THREADS_KEY = "hbase.flush.procedure.master.threads"; 067 private static final int FLUSH_PROC_POOL_THREADS_DEFAULT = 1; 068 069 private static final Logger LOG = LoggerFactory.getLogger(MasterFlushTableProcedureManager.class); 070 071 private MasterServices master; 072 private ProcedureCoordinator coordinator; 073 private Map<TableName, Procedure> procMap = new HashMap<>(); 074 private boolean stopped; 075 076 public MasterFlushTableProcedureManager() { 077 }; 078 079 @Override 080 public void stop(String why) { 081 LOG.info("stop: " + why); 082 this.stopped = true; 083 } 084 085 @Override 086 public boolean isStopped() { 087 return this.stopped; 088 } 089 090 @Override 091 public void initialize(MasterServices master, MetricsMaster metricsMaster) 092 throws KeeperException, IOException, UnsupportedOperationException { 093 this.master = master; 094 095 // get the configuration for the coordinator 096 Configuration conf = master.getConfiguration(); 097 long wakeFrequency = conf.getInt(FLUSH_WAKE_MILLIS_KEY, FLUSH_WAKE_MILLIS_DEFAULT); 098 long timeoutMillis = conf.getLong(FLUSH_TIMEOUT_MILLIS_KEY, FLUSH_TIMEOUT_MILLIS_DEFAULT); 099 int threads = conf.getInt(FLUSH_PROC_POOL_THREADS_KEY, FLUSH_PROC_POOL_THREADS_DEFAULT); 100 101 // setup the procedure coordinator 102 String name = master.getServerName().toString(); 103 ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, threads); 104 ProcedureCoordinatorRpcs comms = 105 new ZKProcedureCoordinator(master.getZooKeeper(), getProcedureSignature(), name); 106 107 this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency); 108 } 109 110 @Override 111 public String getProcedureSignature() { 112 return FLUSH_TABLE_PROCEDURE_SIGNATURE; 113 } 114 115 @Override 116 public void execProcedure(ProcedureDescription desc) throws IOException { 117 118 TableName tableName = TableName.valueOf(desc.getInstance()); 119 120 // call pre coproc hook 121 MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); 122 if (cpHost != null) { 123 cpHost.preTableFlush(tableName); 124 } 125 126 // Get the list of region servers that host the online regions for table. 127 // We use the procedure instance name to carry the table name from the client. 128 // It is possible that regions may move after we get the region server list. 129 // Each region server will get its own online regions for the table. 130 // We may still miss regions that need to be flushed. 131 List<Pair<RegionInfo, ServerName>> regionsAndLocations = 132 master.getAssignmentManager().getTableRegionsAndLocations(tableName, false); 133 134 Set<String> regionServers = new HashSet<>(regionsAndLocations.size()); 135 for (Pair<RegionInfo, ServerName> region : regionsAndLocations) { 136 if (region != null && region.getFirst() != null && region.getSecond() != null) { 137 RegionInfo hri = region.getFirst(); 138 if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) continue; 139 regionServers.add(region.getSecond().toString()); 140 } 141 } 142 143 ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(desc.getInstance()); 144 145 HBaseProtos.NameStringPair family = null; 146 for (HBaseProtos.NameStringPair nsp : desc.getConfigurationList()) { 147 if (HConstants.FAMILY_KEY_STR.equals(nsp.getName())) { 148 family = nsp; 149 } 150 } 151 byte[] procArgs = family != null ? family.toByteArray() : new byte[0]; 152 153 // Kick of the global procedure from the master coordinator to the region servers. 154 // We rely on the existing Distributed Procedure framework to prevent any concurrent 155 // procedure with the same name. 156 Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), procArgs, 157 Lists.newArrayList(regionServers)); 158 monitor.rethrowException(); 159 if (proc == null) { 160 String msg = "Failed to submit distributed procedure " + desc.getSignature() + " for '" 161 + desc.getInstance() + "'. " + "Another flush procedure is running?"; 162 LOG.error(msg); 163 throw new IOException(msg); 164 } 165 166 procMap.put(tableName, proc); 167 168 try { 169 // wait for the procedure to complete. A timer thread is kicked off that should cancel this 170 // if it takes too long. 171 proc.waitForCompleted(); 172 LOG.info("Done waiting - exec procedure " + desc.getSignature() + " for '" 173 + desc.getInstance() + "'"); 174 LOG.info("Master flush table procedure is successful!"); 175 } catch (InterruptedException e) { 176 ForeignException ee = 177 new ForeignException("Interrupted while waiting for flush table procdure to finish", e); 178 monitor.receive(ee); 179 Thread.currentThread().interrupt(); 180 } catch (ForeignException e) { 181 ForeignException ee = 182 new ForeignException("Exception while waiting for flush table procdure to finish", e); 183 monitor.receive(ee); 184 } 185 monitor.rethrowException(); 186 } 187 188 @Override 189 public void checkPermissions(ProcedureDescription desc, AccessChecker accessChecker, User user) 190 throws IOException { 191 // Done by AccessController as part of preTableFlush coprocessor hook (legacy code path). 192 // In future, when we AC is removed for good, that check should be moved here. 193 } 194 195 @Override 196 public synchronized boolean isProcedureDone(ProcedureDescription desc) throws IOException { 197 // Procedure instance name is the table name. 198 TableName tableName = TableName.valueOf(desc.getInstance()); 199 Procedure proc = procMap.get(tableName); 200 if (proc == null) { 201 // The procedure has not even been started yet. 202 // The client would request the procedure and call isProcedureDone(). 203 // The HBaseAdmin.execProcedure() wraps both request and isProcedureDone(). 204 return false; 205 } 206 // We reply on the existing Distributed Procedure framework to give us the status. 207 return proc.isCompleted(); 208 } 209 210}