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