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; 027 028import org.apache.hadoop.conf.Configuration; 029import org.apache.hadoop.hbase.HBaseInterfaceAudience; 030import org.apache.hadoop.hbase.MetaTableAccessor; 031import org.apache.hadoop.hbase.ServerName; 032import org.apache.hadoop.hbase.TableName; 033import org.apache.hadoop.hbase.client.RegionInfo; 034import org.apache.hadoop.hbase.errorhandling.ForeignException; 035import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; 036import org.apache.hadoop.hbase.master.MasterCoprocessorHost; 037import org.apache.hadoop.hbase.master.MasterServices; 038import org.apache.hadoop.hbase.master.MetricsMaster; 039import org.apache.hadoop.hbase.procedure.MasterProcedureManager; 040import org.apache.hadoop.hbase.procedure.Procedure; 041import org.apache.hadoop.hbase.procedure.ProcedureCoordinator; 042import org.apache.hadoop.hbase.procedure.ProcedureCoordinatorRpcs; 043import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator; 044import org.apache.hadoop.hbase.security.User; 045import org.apache.hadoop.hbase.security.access.AccessChecker; 046import org.apache.hadoop.hbase.util.Pair; 047import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; 048import org.apache.yetus.audience.InterfaceAudience; 049import org.apache.zookeeper.KeeperException; 050import org.slf4j.Logger; 051import org.slf4j.LoggerFactory; 052import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 053import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ProcedureDescription; 054 055@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) 056public class MasterFlushTableProcedureManager extends MasterProcedureManager { 057 058 public static final String FLUSH_TABLE_PROCEDURE_SIGNATURE = "flush-table-proc"; 059 060 private static final String FLUSH_TIMEOUT_MILLIS_KEY = "hbase.flush.master.timeoutMillis"; 061 private static final int FLUSH_TIMEOUT_MILLIS_DEFAULT = 60000; 062 private static final String FLUSH_WAKE_MILLIS_KEY = "hbase.flush.master.wakeMillis"; 063 private static final int FLUSH_WAKE_MILLIS_DEFAULT = 500; 064 065 private static final String FLUSH_PROC_POOL_THREADS_KEY = 066 "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 @Override 079 public void stop(String why) { 080 LOG.info("stop: " + why); 081 this.stopped = true; 082 } 083 084 @Override 085 public boolean isStopped() { 086 return this.stopped; 087 } 088 089 @Override 090 public void initialize(MasterServices master, MetricsMaster metricsMaster) 091 throws KeeperException, IOException, UnsupportedOperationException { 092 this.master = master; 093 094 // get the configuration for the coordinator 095 Configuration conf = master.getConfiguration(); 096 long wakeFrequency = conf.getInt(FLUSH_WAKE_MILLIS_KEY, FLUSH_WAKE_MILLIS_DEFAULT); 097 long timeoutMillis = conf.getLong(FLUSH_TIMEOUT_MILLIS_KEY, FLUSH_TIMEOUT_MILLIS_DEFAULT); 098 int threads = conf.getInt(FLUSH_PROC_POOL_THREADS_KEY, FLUSH_PROC_POOL_THREADS_DEFAULT); 099 100 // setup the procedure coordinator 101 String name = master.getServerName().toString(); 102 ThreadPoolExecutor tpool = ProcedureCoordinator.defaultPool(name, threads); 103 ProcedureCoordinatorRpcs comms = new ZKProcedureCoordinator( 104 master.getZooKeeper(), getProcedureSignature(), name); 105 106 this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency); 107 } 108 109 @Override 110 public String getProcedureSignature() { 111 return FLUSH_TABLE_PROCEDURE_SIGNATURE; 112 } 113 114 @Override 115 public void execProcedure(ProcedureDescription desc) throws IOException { 116 117 TableName tableName = TableName.valueOf(desc.getInstance()); 118 119 // call pre coproc hook 120 MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); 121 if (cpHost != null) { 122 cpHost.preTableFlush(tableName); 123 } 124 125 // Get the list of region servers that host the online regions for table. 126 // We use the procedure instance name to carry the table name from the client. 127 // It is possible that regions may move after we get the region server list. 128 // Each region server will get its own online regions for the table. 129 // We may still miss regions that need to be flushed. 130 List<Pair<RegionInfo, ServerName>> regionsAndLocations; 131 132 if (TableName.META_TABLE_NAME.equals(tableName)) { 133 regionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations( 134 master.getZooKeeper()); 135 } else { 136 regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations( 137 master.getConnection(), tableName, false); 138 } 139 140 Set<String> regionServers = new HashSet<>(regionsAndLocations.size()); 141 for (Pair<RegionInfo, ServerName> region : regionsAndLocations) { 142 if (region != null && region.getFirst() != null && region.getSecond() != null) { 143 RegionInfo hri = region.getFirst(); 144 if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) continue; 145 regionServers.add(region.getSecond().toString()); 146 } 147 } 148 149 ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(desc.getInstance()); 150 151 // Kick of the global procedure from the master coordinator to the region servers. 152 // We rely on the existing Distributed Procedure framework to prevent any concurrent 153 // procedure with the same name. 154 Procedure proc = coordinator.startProcedure(monitor, desc.getInstance(), 155 new byte[0], Lists.newArrayList(regionServers)); 156 monitor.rethrowException(); 157 if (proc == null) { 158 String msg = "Failed to submit distributed procedure " + desc.getSignature() + " for '" 159 + desc.getInstance() + "'. " + "Another flush procedure is running?"; 160 LOG.error(msg); 161 throw new IOException(msg); 162 } 163 164 procMap.put(tableName, proc); 165 166 try { 167 // wait for the procedure to complete. A timer thread is kicked off that should cancel this 168 // if it takes too long. 169 proc.waitForCompleted(); 170 LOG.info("Done waiting - exec procedure " + desc.getSignature() + " for '" 171 + desc.getInstance() + "'"); 172 LOG.info("Master flush table procedure is successful!"); 173 } catch (InterruptedException e) { 174 ForeignException ee = 175 new ForeignException("Interrupted while waiting for flush table procdure to finish", e); 176 monitor.receive(ee); 177 Thread.currentThread().interrupt(); 178 } catch (ForeignException e) { 179 ForeignException ee = 180 new ForeignException("Exception while waiting for flush table procdure to finish", e); 181 monitor.receive(ee); 182 } 183 monitor.rethrowException(); 184 } 185 186 @Override 187 public void checkPermissions(ProcedureDescription desc, AccessChecker accessChecker, User user) 188 throws IOException { 189 // Done by AccessController as part of preTableFlush coprocessor hook (legacy code path). 190 // In future, when we AC is removed for good, that check should be moved here. 191 } 192 193 @Override 194 public synchronized boolean isProcedureDone(ProcedureDescription desc) throws IOException { 195 // Procedure instance name is the table name. 196 TableName tableName = TableName.valueOf(desc.getInstance()); 197 Procedure proc = procMap.get(tableName); 198 if (proc == null) { 199 // The procedure has not even been started yet. 200 // The client would request the procedure and call isProcedureDone(). 201 // The HBaseAdmin.execProcedure() wraps both request and isProcedureDone(). 202 return false; 203 } 204 // We reply on the existing Distributed Procedure framework to give us the status. 205 return proc.isCompleted(); 206 } 207 208}