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.client; 019 020import java.io.IOException; 021import java.net.SocketAddress; 022import java.security.PrivilegedExceptionAction; 023import org.apache.hadoop.conf.Configuration; 024import org.apache.hadoop.hbase.security.User; 025import org.apache.hadoop.hbase.util.FutureUtils; 026import org.apache.hadoop.hbase.util.ReflectionUtils; 027import org.apache.yetus.audience.InterfaceAudience; 028 029/** 030 * The factory for creating {@link AsyncClusterConnection}. 031 */ 032@InterfaceAudience.Private 033public final class ClusterConnectionFactory { 034 035 public static final String HBASE_SERVER_CLUSTER_CONNECTION_IMPL = 036 "hbase.server.cluster.connection.impl"; 037 038 private ClusterConnectionFactory() { 039 } 040 041 private static AsyncClusterConnection createAsyncClusterConnection(Configuration conf, 042 ConnectionRegistry registry, SocketAddress localAddress, User user) throws IOException { 043 String clusterId = FutureUtils.get(registry.getClusterId()); 044 Class<? extends AsyncClusterConnection> clazz = 045 conf.getClass(HBASE_SERVER_CLUSTER_CONNECTION_IMPL, AsyncClusterConnectionImpl.class, 046 AsyncClusterConnection.class); 047 try { 048 return user 049 .runAs((PrivilegedExceptionAction<? extends AsyncClusterConnection>) () -> ReflectionUtils 050 .newInstance(clazz, conf, registry, clusterId, localAddress, user)); 051 } catch (Exception e) { 052 throw new IOException(e); 053 } 054 } 055 056 /** 057 * Create a new {@link AsyncClusterConnection} instance. 058 * <p/> 059 * Unlike what we have done in {@link ConnectionFactory}, here we just return an 060 * {@link AsyncClusterConnection} instead of a {@link java.util.concurrent.CompletableFuture}, 061 * which means this method could block on fetching the cluster id. This is just used to simplify 062 * the implementation, as when starting new region servers, we do not need to be event-driven. Can 063 * change later if we want a {@link java.util.concurrent.CompletableFuture} here. 064 */ 065 public static AsyncClusterConnection createAsyncClusterConnection(Configuration conf, 066 SocketAddress localAddress, User user) throws IOException { 067 return createAsyncClusterConnection(conf, ConnectionRegistryFactory.create(conf, user), 068 localAddress, user); 069 } 070 071 /** 072 * Create a new {@link AsyncClusterConnection} instance to be used at server side where we have a 073 * {@link ConnectionRegistryEndpoint}. 074 */ 075 public static AsyncClusterConnection createAsyncClusterConnection( 076 ConnectionRegistryEndpoint endpoint, Configuration conf, SocketAddress localAddress, User user) 077 throws IOException { 078 ShortCircuitConnectionRegistry registry = new ShortCircuitConnectionRegistry(endpoint); 079 return createAsyncClusterConnection(conf, registry, localAddress, user); 080 } 081}