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;
019
020import java.io.IOException;
021import java.util.Collections;
022import java.util.List;
023import java.util.concurrent.TimeUnit;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.fs.FileSystem;
026import org.apache.hadoop.fs.Path;
027import org.apache.hadoop.hbase.CellScanner;
028import org.apache.hadoop.hbase.ScheduledChore;
029import org.apache.hadoop.hbase.Server;
030import org.apache.hadoop.hbase.Stoppable;
031import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
032import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
033import org.apache.hadoop.hbase.replication.regionserver.ReplicationSink;
034import org.apache.hadoop.hbase.wal.WALFactory;
035import org.apache.yetus.audience.InterfaceAudience;
036import org.slf4j.Logger;
037import org.slf4j.LoggerFactory;
038
039import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
040
041@InterfaceAudience.Private
042public class ReplicationSinkServiceImpl implements ReplicationSinkService {
043  private static final Logger LOG = LoggerFactory.getLogger(ReplicationSinkServiceImpl.class);
044
045  private Configuration conf;
046
047  private Server server;
048
049  private ReplicationSink replicationSink;
050
051  // ReplicationLoad to access replication metrics
052  private ReplicationLoad replicationLoad;
053
054  private int statsPeriodInSecond;
055
056  @Override
057  public void replicateLogEntries(List<AdminProtos.WALEntry> entries, CellScanner cells,
058    String replicationClusterId, String sourceBaseNamespaceDirPath,
059    String sourceHFileArchiveDirPath) throws IOException {
060    this.replicationSink.replicateEntries(entries, cells, replicationClusterId,
061      sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath);
062  }
063
064  @Override
065  public void initialize(Server server, FileSystem fs, Path logdir, Path oldLogDir,
066    WALFactory walFactory) throws IOException {
067    this.server = server;
068    this.conf = server.getConfiguration();
069    this.statsPeriodInSecond = this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
070    this.replicationLoad = new ReplicationLoad();
071  }
072
073  @Override
074  public void startReplicationService() throws IOException {
075    this.replicationSink = new ReplicationSink(this.conf);
076    this.server.getChoreService().scheduleChore(new ReplicationStatisticsChore(
077      "ReplicationSinkStatistics", server, (int) TimeUnit.SECONDS.toMillis(statsPeriodInSecond)));
078  }
079
080  @Override
081  public void stopReplicationService() {
082    if (this.replicationSink != null) {
083      this.replicationSink.stopReplicationSinkServices();
084    }
085  }
086
087  @Override
088  public ReplicationLoad refreshAndGetReplicationLoad() {
089    if (replicationLoad == null) {
090      return null;
091    }
092    // always build for latest data
093    replicationLoad.buildReplicationLoad(Collections.emptyList(), replicationSink.getSinkMetrics());
094    return replicationLoad;
095  }
096
097  private final class ReplicationStatisticsChore extends ScheduledChore {
098
099    ReplicationStatisticsChore(String name, Stoppable stopper, int period) {
100      super(name, stopper, period);
101    }
102
103    @Override
104    protected void chore() {
105      printStats(replicationSink.getStats());
106    }
107
108    private void printStats(String stats) {
109      if (!stats.isEmpty()) {
110        LOG.info(stats);
111      }
112    }
113  }
114}