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.replication.regionserver;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.List;
023import java.util.OptionalLong;
024import java.util.UUID;
025import java.util.concurrent.Executors;
026import java.util.concurrent.ScheduledExecutorService;
027import java.util.concurrent.TimeUnit;
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.fs.FileSystem;
030import org.apache.hadoop.fs.Path;
031import org.apache.hadoop.hbase.CellScanner;
032import org.apache.hadoop.hbase.HConstants;
033import org.apache.hadoop.hbase.Server;
034import org.apache.hadoop.hbase.TableName;
035import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
036import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
037import org.apache.hadoop.hbase.replication.ReplicationFactory;
038import org.apache.hadoop.hbase.replication.ReplicationPeers;
039import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
040import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
041import org.apache.hadoop.hbase.replication.ReplicationTracker;
042import org.apache.hadoop.hbase.replication.ReplicationUtils;
043import org.apache.hadoop.hbase.util.Pair;
044import org.apache.hadoop.hbase.wal.WALProvider;
045import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
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.util.concurrent.ThreadFactoryBuilder;
052
053import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
054/**
055 * Gateway to Replication. Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
056 */
057@InterfaceAudience.Private
058public class Replication implements ReplicationSourceService, ReplicationSinkService {
059  private static final Logger LOG =
060      LoggerFactory.getLogger(Replication.class);
061  private boolean isReplicationForBulkLoadDataEnabled;
062  private ReplicationSourceManager replicationManager;
063  private ReplicationQueueStorage queueStorage;
064  private ReplicationPeers replicationPeers;
065  private ReplicationTracker replicationTracker;
066  private Configuration conf;
067  private ReplicationSink replicationSink;
068  // Hosting server
069  private Server server;
070  /** Statistics thread schedule pool */
071  private ScheduledExecutorService scheduleThreadPool;
072  private int statsThreadPeriod;
073  // ReplicationLoad to access replication metrics
074  private ReplicationLoad replicationLoad;
075
076  private PeerProcedureHandler peerProcedureHandler;
077
078  /**
079   * Empty constructor
080   */
081  public Replication() {
082  }
083
084  @Override
085  public void initialize(Server server, FileSystem fs, Path logDir, Path oldLogDir,
086      WALProvider walProvider) throws IOException {
087    this.server = server;
088    this.conf = this.server.getConfiguration();
089    this.isReplicationForBulkLoadDataEnabled =
090      ReplicationUtils.isReplicationForBulkLoadDataEnabled(this.conf);
091    this.scheduleThreadPool = Executors.newScheduledThreadPool(1,
092      new ThreadFactoryBuilder()
093        .setNameFormat(server.getServerName().toShortString() + "Replication Statistics #%d")
094        .setDaemon(true)
095        .build());
096    if (this.isReplicationForBulkLoadDataEnabled) {
097      if (conf.get(HConstants.REPLICATION_CLUSTER_ID) == null
098          || conf.get(HConstants.REPLICATION_CLUSTER_ID).isEmpty()) {
099        throw new IllegalArgumentException(HConstants.REPLICATION_CLUSTER_ID
100            + " cannot be null/empty when " + HConstants.REPLICATION_BULKLOAD_ENABLE_KEY
101            + " is set to true.");
102      }
103    }
104
105    try {
106      this.queueStorage =
107          ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
108      this.replicationPeers =
109          ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf);
110      this.replicationPeers.init();
111      this.replicationTracker =
112          ReplicationFactory.getReplicationTracker(server.getZooKeeper(), this.server, this.server);
113    } catch (Exception e) {
114      throw new IOException("Failed replication handler create", e);
115    }
116    UUID clusterId = null;
117    try {
118      clusterId = ZKClusterId.getUUIDForCluster(this.server.getZooKeeper());
119    } catch (KeeperException ke) {
120      throw new IOException("Could not read cluster id", ke);
121    }
122    this.replicationManager = new ReplicationSourceManager(queueStorage, replicationPeers, replicationTracker, conf,
123      this.server, fs, logDir, oldLogDir, clusterId,
124        walProvider != null ? walProvider.getWALFileLengthProvider() : p -> OptionalLong.empty());
125    if (walProvider != null) {
126      walProvider
127        .addWALActionsListener(new ReplicationSourceWALActionListener(conf, replicationManager));
128    }
129    this.statsThreadPeriod =
130        this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
131    LOG.debug("Replication stats-in-log period={} seconds",  this.statsThreadPeriod);
132    this.replicationLoad = new ReplicationLoad();
133
134    this.peerProcedureHandler = new PeerProcedureHandlerImpl(replicationManager);
135  }
136
137  @Override
138  public PeerProcedureHandler getPeerProcedureHandler() {
139    return peerProcedureHandler;
140  }
141
142  /**
143   * Stops replication service.
144   */
145  @Override
146  public void stopReplicationService() {
147    join();
148  }
149
150  /**
151   * Join with the replication threads
152   */
153  public void join() {
154    this.replicationManager.join();
155    if (this.replicationSink != null) {
156      this.replicationSink.stopReplicationSinkServices();
157    }
158    scheduleThreadPool.shutdown();
159  }
160
161  /**
162   * Carry on the list of log entries down to the sink
163   * @param entries list of entries to replicate
164   * @param cells The data -- the cells -- that <code>entries</code> describes (the entries do not
165   *          contain the Cells we are replicating; they are passed here on the side in this
166   *          CellScanner).
167   * @param replicationClusterId Id which will uniquely identify source cluster FS client
168   *          configurations in the replication configuration directory
169   * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace
170   *          directory required for replicating hfiles
171   * @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory
172   * @throws IOException
173   */
174  @Override
175  public void replicateLogEntries(List<WALEntry> entries, CellScanner cells,
176      String replicationClusterId, String sourceBaseNamespaceDirPath,
177      String sourceHFileArchiveDirPath) throws IOException {
178    this.replicationSink.replicateEntries(entries, cells, replicationClusterId,
179      sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath);
180  }
181
182  /**
183   * If replication is enabled and this cluster is a master,
184   * it starts
185   * @throws IOException
186   */
187  @Override
188  public void startReplicationService() throws IOException {
189    this.replicationManager.init();
190    this.replicationSink = new ReplicationSink(this.conf, this.server);
191    this.scheduleThreadPool.scheduleAtFixedRate(
192      new ReplicationStatisticsTask(this.replicationSink, this.replicationManager),
193      statsThreadPeriod, statsThreadPeriod, TimeUnit.SECONDS);
194    LOG.info("{} started", this.server.toString());
195  }
196
197  /**
198   * Get the replication sources manager
199   * @return the manager if replication is enabled, else returns false
200   */
201  public ReplicationSourceManager getReplicationManager() {
202    return this.replicationManager;
203  }
204
205  void addHFileRefsToQueue(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
206      throws IOException {
207    try {
208      this.replicationManager.addHFileRefs(tableName, family, pairs);
209    } catch (IOException e) {
210      LOG.error("Failed to add hfile references in the replication queue.", e);
211      throw e;
212    }
213  }
214
215  /**
216   * Statistics task. Periodically prints the cache statistics to the log.
217   */
218  private final static class ReplicationStatisticsTask implements Runnable {
219
220    private final ReplicationSink replicationSink;
221    private final ReplicationSourceManager replicationManager;
222
223    public ReplicationStatisticsTask(ReplicationSink replicationSink,
224        ReplicationSourceManager replicationManager) {
225      this.replicationManager = replicationManager;
226      this.replicationSink = replicationSink;
227    }
228
229    @Override
230    public void run() {
231      printStats(this.replicationManager.getStats());
232      printStats(this.replicationSink.getStats());
233    }
234
235    private void printStats(String stats) {
236      if (!stats.isEmpty()) {
237        LOG.info(stats);
238      }
239    }
240  }
241
242  @Override
243  public ReplicationLoad refreshAndGetReplicationLoad() {
244    if (this.replicationLoad == null) {
245      return null;
246    }
247    // always build for latest data
248    buildReplicationLoad();
249    return this.replicationLoad;
250  }
251
252  private void buildReplicationLoad() {
253    List<ReplicationSourceInterface> allSources = new ArrayList<>();
254    allSources.addAll(this.replicationManager.getSources());
255    allSources.addAll(this.replicationManager.getOldSources());
256
257    // get sink
258    MetricsSink sinkMetrics = this.replicationSink.getSinkMetrics();
259    this.replicationLoad.buildReplicationLoad(allSources, sinkMetrics);
260  }
261}