/*
* The Alluxio Open Foundation licenses this work under the Apache License, version 2.0
* (the "License"). You may not use this work except in compliance with the License, which is
* available at www.apache.org/licenses/LICENSE-2.0
*
* This software is distributed on an "AS IS" basis, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
* either express or implied, as more fully set forth in the License.
*
* See the NOTICE file distributed with this work for information regarding copyright ownership.
*/
package alluxio.client.file;
import alluxio.Configuration;
import alluxio.PropertyKey;
import alluxio.client.block.BlockMasterClient;
import alluxio.client.block.BlockMasterClientPool;
import alluxio.client.netty.NettyClient;
import alluxio.exception.ExceptionMessage;
import alluxio.exception.status.UnavailableException;
import alluxio.metrics.MetricsSystem;
import alluxio.network.connection.NettyChannelPool;
import alluxio.resource.CloseableResource;
import alluxio.util.network.NetworkAddressUtils;
import alluxio.util.network.NetworkAddressUtils.ServiceType;
import alluxio.wire.WorkerInfo;
import alluxio.wire.WorkerNetAddress;
import com.codahale.metrics.Gauge;
import com.google.common.base.Preconditions;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
import io.netty.util.internal.chmv8.ConcurrentHashMapV8;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.util.ArrayList;
import java.util.List;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
import javax.security.auth.Subject;
/**
* A shared context that isolates all operations within a {@link FileSystem}. Usually, one user
* only needs one instance of {@link FileSystemContext}.
*
* <p>
* NOTE: The context maintains a pool of file system master clients that is already thread-safe.
* Synchronizing {@link FileSystemContext} methods could lead to deadlock: thread A attempts to
* acquire a client when there are no clients left in the pool and blocks holding a lock on the
* {@link FileSystemContext}, when thread B attempts to release a client it owns it is unable to do
* so, because thread A holds the lock on {@link FileSystemContext}.
*/
@ThreadSafe
public final class FileSystemContext implements Closeable {
public static final FileSystemContext INSTANCE = create(null);
static {
MetricsSystem.startSinks();
Metrics.initializeGauges();
}
// Master client pools.
private volatile FileSystemMasterClientPool mFileSystemMasterClientPool;
private volatile BlockMasterClientPool mBlockMasterClientPool;
// The netty data server channel pools.
private final ConcurrentHashMapV8<SocketAddress, NettyChannelPool>
mNettyChannelPools = new ConcurrentHashMapV8<>();
/** The shared master address associated with the {@link FileSystemContext}. */
@GuardedBy("this")
private InetSocketAddress mMasterAddress;
/**
* Indicates whether the {@link #mLocalWorker} field has been lazily initialized yet.
*/
@GuardedBy("this")
private boolean mLocalWorkerInitialized;
/**
* The address of any Alluxio worker running on the local machine. This is initialized lazily.
*/
@GuardedBy("this")
private WorkerNetAddress mLocalWorker;
/** The parent user associated with the {@link FileSystemContext}. */
private final Subject mParentSubject;
/**
* Creates a new file system context.
*
* @return the context
*/
public static FileSystemContext create() {
return create(null);
}
/**
* Creates a file system context with a subject.
*
* @param subject the parent subject, set to null if not present
* @return the context
*/
public static FileSystemContext create(Subject subject) {
FileSystemContext context = new FileSystemContext(subject);
context.init();
return context;
}
/**
* Creates a file system context with a subject.
*
* @param subject the parent subject, set to null if not present
*/
private FileSystemContext(Subject subject) {
mParentSubject = subject;
}
/**
* Initializes the context. Only called in the factory methods and reset.
*/
private void init() {
mMasterAddress = NetworkAddressUtils.getConnectAddress(ServiceType.MASTER_RPC);
mFileSystemMasterClientPool = new FileSystemMasterClientPool(mParentSubject, mMasterAddress);
mBlockMasterClientPool = new BlockMasterClientPool(mParentSubject, mMasterAddress);
}
/**
* Closes all the resources associated with the context. Make sure all the resources are released
* back to this context before calling this close. After closing the context, all the resources
* that acquired from this context might fail. Only call this when you are done with using
* the {@link FileSystem} associated with this {@link FileSystemContext}.
*/
public void close() throws IOException {
mFileSystemMasterClientPool.close();
mFileSystemMasterClientPool = null;
mBlockMasterClientPool.close();
mBlockMasterClientPool = null;
for (NettyChannelPool pool : mNettyChannelPools.values()) {
pool.close();
}
mNettyChannelPools.clear();
synchronized (this) {
mMasterAddress = null;
mLocalWorkerInitialized = false;
mLocalWorker = null;
}
}
/**
* Resets the context. It is only used in {@link alluxio.hadoop.AbstractFileSystem} and
* tests to reset the default file system context.
*/
public synchronized void reset() throws IOException {
close();
init();
}
/**
* @return the parent subject
*/
public Subject getParentSubject() {
return mParentSubject;
}
/**
* @return the master address
*/
public synchronized InetSocketAddress getMasterAddress() {
return mMasterAddress;
}
/**
* Acquires a file system master client from the file system master client pool.
*
* @return the acquired file system master client
*/
public FileSystemMasterClient acquireMasterClient() {
return mFileSystemMasterClientPool.acquire();
}
/**
* Releases a file system master client into the file system master client pool.
*
* @param masterClient a file system master client to release
*/
public void releaseMasterClient(FileSystemMasterClient masterClient) {
mFileSystemMasterClientPool.release(masterClient);
}
/**
* Acquires a file system master client from the file system master client pool. The resource is
* {@code Closeable}.
*
* @return the acquired file system master client resource
*/
public CloseableResource<FileSystemMasterClient> acquireMasterClientResource() {
return new CloseableResource<FileSystemMasterClient>(mFileSystemMasterClientPool.acquire()) {
@Override
public void close() {
mFileSystemMasterClientPool.release(get());
}
};
}
/**
* Acquires a block master client resource from the block master client pool. The resource is
* {@code Closeable}.
*
* @return the acquired block master client resource
*/
public CloseableResource<BlockMasterClient> acquireBlockMasterClientResource() {
return new CloseableResource<BlockMasterClient>(mBlockMasterClientPool.acquire()) {
@Override
public void close() {
mBlockMasterClientPool.release(get());
}
};
}
/**
* Acquires a netty channel from the channel pools. If there is no available client instance
* available in the pool, it tries to create a new one. And an exception is thrown if it fails to
* create a new one.
*
* @param workerNetAddress the network address of the channel
* @return the acquired netty channel
*/
public Channel acquireNettyChannel(final WorkerNetAddress workerNetAddress) throws IOException {
SocketAddress address = NetworkAddressUtils.getDataPortSocketAddress(workerNetAddress);
if (!mNettyChannelPools.containsKey(address)) {
Bootstrap bs = NettyClient.createClientBootstrap(address);
bs.remoteAddress(address);
NettyChannelPool pool = new NettyChannelPool(bs,
Configuration.getInt(PropertyKey.USER_NETWORK_NETTY_CHANNEL_POOL_SIZE_MAX),
Configuration.getLong(PropertyKey.USER_NETWORK_NETTY_CHANNEL_POOL_GC_THRESHOLD_MS));
if (mNettyChannelPools.putIfAbsent(address, pool) != null) {
// This can happen if this function is called concurrently.
pool.close();
}
}
return mNettyChannelPools.get(address).acquire();
}
/**
* Releases a netty channel to the channel pools.
*
* @param workerNetAddress the address of the channel
* @param channel the channel to release
*/
public void releaseNettyChannel(WorkerNetAddress workerNetAddress, Channel channel) {
SocketAddress address = NetworkAddressUtils.getDataPortSocketAddress(workerNetAddress);
Preconditions.checkArgument(mNettyChannelPools.containsKey(address));
mNettyChannelPools.get(address).release(channel);
}
/**
* @return if there is a local worker running the same machine
*/
public synchronized boolean hasLocalWorker() throws IOException {
if (!mLocalWorkerInitialized) {
initializeLocalWorker();
}
return mLocalWorker != null;
}
/**
* @return a local worker running the same machine, or null if none is found
*/
public synchronized WorkerNetAddress getLocalWorker() throws IOException {
if (!mLocalWorkerInitialized) {
initializeLocalWorker();
}
return mLocalWorker;
}
private void initializeLocalWorker() throws IOException {
List<WorkerNetAddress> addresses = getWorkerAddresses();
if (!addresses.isEmpty()) {
if (addresses.get(0).getHost().equals(NetworkAddressUtils.getClientHostName())) {
mLocalWorker = addresses.get(0);
}
}
mLocalWorkerInitialized = true;
}
/**
* @return if there are any local workers, the returned list will ONLY contain the local workers,
* otherwise a list of all remote workers will be returned
*/
private List<WorkerNetAddress> getWorkerAddresses() throws IOException {
List<WorkerInfo> infos;
BlockMasterClient blockMasterClient = mBlockMasterClientPool.acquire();
try {
infos = blockMasterClient.getWorkerInfoList();
} finally {
mBlockMasterClientPool.release(blockMasterClient);
}
if (infos.isEmpty()) {
throw new UnavailableException(ExceptionMessage.NO_WORKER_AVAILABLE.getMessage());
}
// Convert the worker infos into net addresses, if there are local addresses, only keep those
List<WorkerNetAddress> workerNetAddresses = new ArrayList<>();
List<WorkerNetAddress> localWorkerNetAddresses = new ArrayList<>();
String localHostname = NetworkAddressUtils.getClientHostName();
for (WorkerInfo info : infos) {
WorkerNetAddress netAddress = info.getAddress();
if (netAddress.getHost().equals(localHostname)) {
localWorkerNetAddresses.add(netAddress);
}
workerNetAddresses.add(netAddress);
}
return localWorkerNetAddresses.isEmpty() ? workerNetAddresses : localWorkerNetAddresses;
}
/**
* Class that contains metrics about FileSystemContext.
*/
@ThreadSafe
private static final class Metrics {
private static void initializeGauges() {
MetricsSystem.registerGaugeIfAbsent(MetricsSystem.getClientMetricName("NettyConnectionsOpen"),
new Gauge<Long>() {
@Override
public Long getValue() {
long ret = 0;
for (NettyChannelPool pool : INSTANCE.mNettyChannelPools.values()) {
ret += pool.size();
}
return ret;
}
});
}
private Metrics() {} // prevent instantiation
}
}