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.lang.reflect.Constructor;
021import org.apache.hadoop.conf.Configuration;
022import org.apache.hadoop.fs.FileSystem;
023import org.apache.hadoop.hbase.util.ReflectionUtils;
024import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
025import org.apache.yetus.audience.InterfaceAudience;
026
027/**
028 * Used to create replication storage(peer, queue) classes.
029 */
030@InterfaceAudience.Private
031public final class ReplicationStorageFactory {
032
033  public static final String REPLICATION_PEER_STORAGE_IMPL = "hbase.replication.peer.storage.impl";
034
035  // must use zookeeper here, otherwise when user upgrading from an old version without changing the
036  // config file, they will loss all the replication peer data.
037  public static final ReplicationPeerStorageType DEFAULT_REPLICATION_PEER_STORAGE_IMPL =
038    ReplicationPeerStorageType.ZOOKEEPER;
039
040  private ReplicationStorageFactory() {
041  }
042
043  private static Class<? extends ReplicationPeerStorage>
044    getReplicationPeerStorageClass(Configuration conf) {
045    try {
046      ReplicationPeerStorageType type = ReplicationPeerStorageType.valueOf(
047        conf.get(REPLICATION_PEER_STORAGE_IMPL, DEFAULT_REPLICATION_PEER_STORAGE_IMPL.name())
048          .toUpperCase());
049      return type.getClazz();
050    } catch (IllegalArgumentException e) {
051      return conf.getClass(REPLICATION_PEER_STORAGE_IMPL,
052        DEFAULT_REPLICATION_PEER_STORAGE_IMPL.getClazz(), ReplicationPeerStorage.class);
053    }
054  }
055
056  /**
057   * Create a new {@link ReplicationPeerStorage}.
058   */
059  public static ReplicationPeerStorage getReplicationPeerStorage(FileSystem fs, ZKWatcher zk,
060    Configuration conf) {
061    Class<? extends ReplicationPeerStorage> clazz = getReplicationPeerStorageClass(conf);
062    for (Constructor<?> c : clazz.getConstructors()) {
063      if (c.getParameterCount() != 2) {
064        continue;
065      }
066      if (c.getParameterTypes()[0].isAssignableFrom(FileSystem.class)) {
067        return ReflectionUtils.newInstance(clazz, fs, conf);
068      } else if (c.getParameterTypes()[0].isAssignableFrom(ZKWatcher.class)) {
069        return ReflectionUtils.newInstance(clazz, zk, conf);
070      }
071    }
072    throw new IllegalArgumentException(
073      "Can not create replication peer storage with type " + clazz);
074  }
075
076  /**
077   * Create a new {@link ReplicationQueueStorage}.
078   */
079  public static ReplicationQueueStorage getReplicationQueueStorage(ZKWatcher zk,
080    Configuration conf) {
081    return new ZKReplicationQueueStorage(zk, conf);
082  }
083}