From 68832e5ffbf4e22b896939d56e0abffabb668a13 Mon Sep 17 00:00:00 2001 From: Jiacheng Liu Date: Mon, 31 Jul 2023 16:24:20 +0800 Subject: [PATCH 01/12] Remove BlockLocationPolicy and BlockIn/OutStream ### What changes are proposed in this pull request? 1. Remove `BlockLocationPolicy` and all implementations 2. Remove `BlockInStream`, `BlockOutStream` and `UnderFileSystemFileOutStream` 3. Updates in property keys to remove relevant sections 4. Update affected commands like StressBench and LoadCommand, to avoid calling methods of the obsolete Block API ### Why are the changes needed? We are deprecating Block API and removing all relevant code ### Does this PR introduce any user facing changes? Please list the user-facing changes introduced by your change, including 1. change in user-facing APIs 2. addition or removal of property keys 6. webui pr-link: Alluxio/alluxio#17831 change-id: cid-eff1fabde08723b15e138d949ec289fb8604fa64 --- .../client/block/BlockStoreClient.java | 291 ---------- .../block/policy/BlockLocationPolicy.java | 70 --- .../policy/CapacityBaseRandomPolicy.java | 70 --- .../CapacityBasedDeterministicHashPolicy.java | 136 ----- .../block/policy/DeterministicHashPolicy.java | 122 ---- .../policy/LocalFirstAvoidEvictionPolicy.java | 131 ----- .../client/block/policy/LocalFirstPolicy.java | 110 ---- .../policy/MostAvailableFirstPolicy.java | 68 --- .../client/block/policy/RoundRobinPolicy.java | 142 ----- .../block/policy/SpecificHostPolicy.java | 107 ---- .../policy/options/GetWorkerOptions.java | 104 ---- .../client/block/stream/BlockInStream.java | 545 ------------------ .../client/block/stream/BlockOutStream.java | 289 ---------- .../stream/UnderFileSystemFileOutStream.java | 73 --- .../client/file/FileSystemContext.java | 31 - .../client/file/options/InStreamOptions.java | 37 -- .../client/file/options/OutStreamOptions.java | 29 - .../client/block/BlockStoreClientTest.java | 535 ----------------- .../policy/CapacityBaseRandomPolicyTest.java | 123 ---- ...acityBasedDeterministicHashPolicyTest.java | 316 ---------- .../policy/DeterministicHashPolicyTest.java | 121 ---- .../LocalFirstAvoidEvictionPolicyTest.java | 103 ---- .../block/policy/LocalFirstPolicyTest.java | 154 ----- .../policy/MostAvailableFirstPolicyTest.java | 61 -- .../block/policy/RoundRobinPolicyTest.java | 103 ---- .../block/policy/SpecificHostPolicyTest.java | 66 --- .../policy/options/GetWorkerOptionsTest.java | 83 --- .../block/stream/BlockInStreamTest.java | 159 ----- .../block/stream/BlockOutStreamTest.java | 42 -- .../block/stream/TestBlockInStream.java | 108 ---- .../block/stream/TestBlockOutStream.java | 73 --- .../TestUnderFileSystemFileOutStream.java | 72 --- .../file/options/OutStreamOptionsTest.java | 9 - .../main/java/alluxio/conf/PropertyKey.java | 76 --- .../main/java/alluxio/conf/RemovedKey.java | 4 - .../cli/fs/command/CopyFromLocalCommand.java | 21 +- .../alluxio/cli/fs/command/LoadCommand.java | 138 +---- .../cli/client/ClientIOWritePolicy.java | 131 ----- .../cli/client/StressClientIOBench.java | 4 - .../stress/cli/worker/StressWorkerBench.java | 12 - .../client/fs/RemoteReadIntegrationTest.java | 153 ----- .../BufferedBlockInStreamIntegrationTest.java | 3 +- .../fs/io/FileOutStreamIntegrationTest.java | 25 - .../server/ft/MultiWorkerIntegrationTest.java | 227 -------- 44 files changed, 25 insertions(+), 5252 deletions(-) delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/BlockLocationPolicy.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/CapacityBaseRandomPolicy.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/CapacityBasedDeterministicHashPolicy.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/DeterministicHashPolicy.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/LocalFirstAvoidEvictionPolicy.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/LocalFirstPolicy.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/MostAvailableFirstPolicy.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/RoundRobinPolicy.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/SpecificHostPolicy.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/policy/options/GetWorkerOptions.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/stream/BlockInStream.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/stream/BlockOutStream.java delete mode 100644 dora/core/client/fs/src/main/java/alluxio/client/block/stream/UnderFileSystemFileOutStream.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/BlockStoreClientTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/policy/CapacityBaseRandomPolicyTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/policy/CapacityBasedDeterministicHashPolicyTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/policy/DeterministicHashPolicyTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/policy/LocalFirstAvoidEvictionPolicyTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/policy/LocalFirstPolicyTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/policy/MostAvailableFirstPolicyTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/policy/RoundRobinPolicyTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/policy/SpecificHostPolicyTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/policy/options/GetWorkerOptionsTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/stream/BlockInStreamTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/stream/BlockOutStreamTest.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestBlockInStream.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestBlockOutStream.java delete mode 100644 dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestUnderFileSystemFileOutStream.java delete mode 100644 dora/stress/shell/src/main/java/alluxio/stress/cli/client/ClientIOWritePolicy.java delete mode 100644 dora/tests/src/test/java/alluxio/server/ft/MultiWorkerIntegrationTest.java diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/BlockStoreClient.java b/dora/core/client/fs/src/main/java/alluxio/client/block/BlockStoreClient.java index 65221df05e2a..7ce7067d5235 100644 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/BlockStoreClient.java +++ b/dora/core/client/fs/src/main/java/alluxio/client/block/BlockStoreClient.java @@ -11,49 +11,17 @@ package alluxio.client.block; -import static java.util.stream.Collectors.toList; -import static java.util.stream.Collectors.toSet; - -import alluxio.client.WriteType; -import alluxio.client.block.policy.BlockLocationPolicy; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.client.block.stream.BlockInStream; -import alluxio.client.block.stream.BlockInStream.BlockInStreamSource; -import alluxio.client.block.stream.BlockOutStream; -import alluxio.client.block.stream.DataWriter; -import alluxio.client.block.util.BlockLocationUtils; import alluxio.client.file.FileSystemContext; -import alluxio.client.file.URIStatus; -import alluxio.client.file.options.InStreamOptions; -import alluxio.client.file.options.OutStreamOptions; -import alluxio.collections.Pair; -import alluxio.exception.ExceptionMessage; -import alluxio.exception.PreconditionMessage; -import alluxio.exception.status.UnavailableException; import alluxio.network.TieredIdentityFactory; import alluxio.resource.CloseableResource; import alluxio.wire.BlockInfo; -import alluxio.wire.BlockLocation; import alluxio.wire.TieredIdentity; -import alluxio.wire.WorkerNetAddress; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.text.MessageFormat; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; import javax.annotation.concurrent.ThreadSafe; /** @@ -103,265 +71,6 @@ public BlockInfo getInfo(long blockId) throws IOException { } } - /** - * Gets a stream to read the data of a block. This method is primarily responsible for - * determining the data source and type of data source. The latest BlockInfo will be fetched - * from the master to ensure the locations are up-to-date. - * - * @param blockId the id of the block to read - * @param options the options associated with the read request - * @return a stream which reads from the beginning of the block - */ - public BlockInStream getInStream(long blockId, InStreamOptions options) throws IOException { - return getInStream(blockId, options, ImmutableMap.of()); - } - - /** - * Gets a stream to read the data of a block. This method is primarily responsible for - * determining the data source and type of data source. The latest BlockInfo will be fetched - * from the master to ensure the locations are up-to-date. It takes a map of failed workers and - * their most recently failed time and tries to update it when BlockInStream created failed, - * attempting to avoid reading from a recently failed worker. - * - * @param blockId the id of the block to read - * @param options the options associated with the read request - * @param failedWorkers the map of worker's addresses to most recent failure time - * @return a stream which reads from the beginning of the block - */ - public BlockInStream getInStream(long blockId, InStreamOptions options, - Map failedWorkers) throws IOException { - // Get the latest block info from master - BlockInfo info = getInfo(blockId); - return getInStream(info, options, failedWorkers); - } - - /** - * {@link #getInStream(long, InStreamOptions, Map)}. - * - * @param info the block info - * @param options the options associated with the read request - * @param failedWorkers the map of worker's addresses to most recent failure time - * @return a stream which reads from the beginning of the block - */ - public BlockInStream getInStream(BlockInfo info, InStreamOptions options, - Map failedWorkers) throws IOException { - Pair dataSourceAndType = getDataSourceAndType(info, - options.getStatus(), options.getUfsReadLocationPolicy(), failedWorkers); - WorkerNetAddress dataSource = dataSourceAndType.getFirst(); - BlockInStreamSource dataSourceType = dataSourceAndType.getSecond(); - try { - return BlockInStream.create(mContext, info, dataSource, dataSourceType, options); - } catch (UnavailableException e) { - //When BlockInStream created failed, it will update the passed-in failedWorkers - //to attempt to avoid reading from this failed worker in next try. - LOG.info("Added {} to failedWorkers for {}", dataSource, e.toString()); - failedWorkers.put(dataSource, System.currentTimeMillis()); - throw e; - } - } - - /** - * Gets the data source and type of data source of a block. This method is primarily responsible - * for determining the data source and type of data source. It takes a map of failed workers and - * their most recently failed time and tries to update it when BlockInStream created failed, - * attempting to avoid reading from a recently failed worker. - * - * @param info the info of the block to read - * @param status the URIStatus associated with the read request - * @param policy the policy determining the Alluxio worker location - * @param failedWorkers the map of worker's addresses to most recent failure time - * @return the data source and type of data source of the block - */ - public Pair getDataSourceAndType(BlockInfo info, - URIStatus status, BlockLocationPolicy policy, Map failedWorkers) - throws IOException { - List locations = info.getLocations(); - List blockWorkerInfo = Collections.emptyList(); - // Initial target workers to read the block given the block locations. - Set workerPool; - // Note that, it is possible that the blocks have been written as UFS blocks - if (status.isPersisted() - || status.getPersistenceState().equals("TO_BE_PERSISTED")) { - blockWorkerInfo = mContext.getCachedWorkers(); - if (blockWorkerInfo.isEmpty()) { - throw new UnavailableException(ExceptionMessage.NO_WORKER_AVAILABLE.getMessage()); - } - workerPool = blockWorkerInfo.stream().map(BlockWorkerInfo::getNetAddress).collect(toSet()); - } else { - if (locations.isEmpty()) { - blockWorkerInfo = mContext.getCachedWorkers(); - if (blockWorkerInfo.isEmpty()) { - throw new UnavailableException(ExceptionMessage.NO_WORKER_AVAILABLE.getMessage()); - } - throw new UnavailableException(MessageFormat - .format("Block {0} is unavailable in both Alluxio and UFS.", info.getBlockId())); - } - workerPool = locations.stream().map(BlockLocation::getWorkerAddress).collect(toSet()); - } - // Workers to read the block, after considering failed workers. - Set workers = handleFailedWorkers(workerPool, failedWorkers); - // TODO(calvin, jianjian): Consider containing these two variables in one object - BlockInStreamSource dataSourceType = null; - WorkerNetAddress dataSource = null; - locations = locations.stream() - .filter(location -> workers.contains(location.getWorkerAddress())).collect(toList()); - // First try to read data from Alluxio - if (!locations.isEmpty()) { - // TODO(calvin): Get location via a policy - List tieredLocations = - locations.stream().map(BlockLocation::getWorkerAddress) - .collect(toList()); - Collections.shuffle(tieredLocations); - Optional> nearest = - BlockLocationUtils.nearest(mTieredIdentity, tieredLocations, mContext.getClusterConf()); - if (nearest.isPresent()) { - dataSource = nearest.get().getFirst(); - dataSourceType = nearest.get().getSecond() ? mContext.hasProcessLocalWorker() - ? BlockInStreamSource.PROCESS_LOCAL : BlockInStreamSource.NODE_LOCAL - : BlockInStreamSource.REMOTE; - } - } - // Can't get data from Alluxio, get it from the UFS instead - if (dataSource == null) { - dataSourceType = BlockInStreamSource.UFS; - Preconditions.checkNotNull(policy, "The UFS read location policy is not specified"); - blockWorkerInfo = blockWorkerInfo.stream() - .filter(workerInfo -> workers.contains(workerInfo.getNetAddress())).collect(toList()); - GetWorkerOptions getWorkerOptions = GetWorkerOptions.defaults() - .setBlockInfo(new BlockInfo() - .setBlockId(info.getBlockId()) - .setLength(info.getLength()) - .setLocations(locations)) - .setBlockWorkerInfos(blockWorkerInfo); - dataSource = policy.getWorker(getWorkerOptions).orElseThrow( - () -> new UnavailableException(ExceptionMessage.NO_WORKER_AVAILABLE.getMessage()) - ); - if (mContext.hasProcessLocalWorker() - && dataSource.equals(mContext.getNodeLocalWorker())) { - dataSourceType = BlockInStreamSource.PROCESS_LOCAL; - LOG.debug("Create BlockInStream to read data from UFS through process local worker {}", - dataSource); - } else { - LOG.debug("Create BlockInStream to read data from UFS through worker {} " - + "(client embedded in local worker process: {}," - + "client co-located with worker in different processes: {}, " - + "local worker address: {})", - dataSource, mContext.hasProcessLocalWorker(), mContext.hasNodeLocalWorker(), - mContext.hasNodeLocalWorker() ? mContext.getNodeLocalWorker() : "N/A"); - } - } - return new Pair<>(dataSource, dataSourceType); - } - - private Set handleFailedWorkers(Set workers, - Map failedWorkers) { - if (workers.isEmpty()) { - return Collections.emptySet(); - } - Set nonFailed = - workers.stream().filter(worker -> !failedWorkers.containsKey(worker)).collect(toSet()); - if (nonFailed.isEmpty()) { - return Collections.singleton(workers.stream() - .min(Comparator.comparingLong(failedWorkers::get)).get()); - } - return nonFailed; - } - - /** - * Gets a stream to write data to a block. The stream can only be backed by Alluxio storage. - * - * @param blockId the block to write - * @param blockSize the standard block size to write - * @param address the address of the worker to write the block to, fails if the worker cannot - * serve the request - * @param options the output stream options - * @return an {@link BlockOutStream} which can be used to write data to the block in a streaming - * fashion - */ - public BlockOutStream getOutStream(long blockId, long blockSize, WorkerNetAddress address, - OutStreamOptions options) throws IOException { - // No specified location to write to. - Preconditions.checkNotNull(address, "address"); - LOG.debug("Create BlockOutStream for {} of block size {} at address {}, using options: {}", - blockId, blockSize, address, options); - DataWriter dataWriter = - DataWriter.Factory.create(mContext, blockId, blockSize, address, options); - return new BlockOutStream(dataWriter, blockSize, address); - } - - /** - * Gets a stream to write data to a block based on the options. The stream can only be backed by - * Alluxio storage. - * - * @param blockId the block to write - * @param blockSize the standard block size to write - * @param options the output stream option - * @return a {@link BlockOutStream} which can be used to write data to the block in a streaming - * fashion - */ - public BlockOutStream getOutStream(long blockId, long blockSize, OutStreamOptions options) - throws IOException { - WorkerNetAddress address; - BlockLocationPolicy locationPolicy = Preconditions.checkNotNull(options.getLocationPolicy(), - PreconditionMessage.BLOCK_WRITE_LOCATION_POLICY_UNSPECIFIED); - GetWorkerOptions workerOptions = GetWorkerOptions.defaults() - .setBlockInfo(new BlockInfo().setBlockId(blockId).setLength(blockSize)) - .setBlockWorkerInfos(new ArrayList<>(mContext.getCachedWorkers())); - - // The number of initial copies depends on the write type: if ASYNC_THROUGH, it is the property - // "alluxio.user.file.replication.durable" before data has been persisted; otherwise - // "alluxio.user.file.replication.min" - int initialReplicas = (options.getWriteType() == WriteType.ASYNC_THROUGH - && options.getReplicationDurable() > options.getReplicationMin()) - ? options.getReplicationDurable() : options.getReplicationMin(); - if (initialReplicas <= 1) { - address = locationPolicy.getWorker(workerOptions).orElseThrow( - () -> { - try { - if (mContext.getCachedWorkers().isEmpty()) { - return new UnavailableException(ExceptionMessage.NO_WORKER_AVAILABLE.getMessage()); - } - } catch (IOException e) { - return e; - } - return new UnavailableException( - ExceptionMessage.NO_SPACE_FOR_BLOCK_ON_WORKER.getMessage(blockSize)); - } - ); - // TODO(ggezer): Retry on another worker if this has no storage. - return getOutStream(blockId, blockSize, address, options); - } - - // Group different block workers by their hostnames - Map> blockWorkersByHost = new HashMap<>(); - for (BlockWorkerInfo blockWorker : workerOptions.getBlockWorkerInfos()) { - String hostName = blockWorker.getNetAddress().getHost(); - if (blockWorkersByHost.containsKey(hostName)) { - blockWorkersByHost.get(hostName).add(blockWorker); - } else { - blockWorkersByHost.put(hostName, com.google.common.collect.Sets.newHashSet(blockWorker)); - } - } - - // Select N workers on different hosts where N is the value of initialReplicas for this block - List workerAddressList = new ArrayList<>(); - List updatedInfos = Lists.newArrayList(workerOptions.getBlockWorkerInfos()); - for (int i = 0; i < initialReplicas; i++) { - locationPolicy.getWorker(workerOptions).ifPresent(workerAddress -> { - workerAddressList.add(workerAddress); - updatedInfos.removeAll(blockWorkersByHost.get(workerAddress.getHost())); - workerOptions.setBlockWorkerInfos(updatedInfos); - }); - } - if (workerAddressList.size() < initialReplicas) { - throw new alluxio.exception.status.ResourceExhaustedException(String.format( - "Not enough workers for replications, %d workers selected but %d required", - workerAddressList.size(), initialReplicas)); - } - return BlockOutStream - .createReplicatedBlockOutStream(mContext, blockId, blockSize, workerAddressList, options); - } - /** * Gets the total capacity of Alluxio's BlockStore. * diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/BlockLocationPolicy.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/BlockLocationPolicy.java deleted file mode 100644 index 28ab8655c2f4..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/BlockLocationPolicy.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.annotation.PublicApi; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.util.CommonUtils; -import alluxio.wire.WorkerNetAddress; - -import java.util.Optional; - -/** - *

- * Interface for determining the Alluxio worker location to serve a block write or UFS block read. - *

- * - *

- * {@link alluxio.client.file.FileInStream} uses this to determine where to read a UFS block. - *

- * - *

- * A policy must have an empty constructor to be used as default policy. - *

- */ -@PublicApi -public interface BlockLocationPolicy { - - /** - * The factory for the {@link BlockLocationPolicy}. - */ - class Factory { - private Factory() {} // prevent instantiation - - /** - * Factory for creating {@link BlockLocationPolicy}. - * - * @param conf Alluxio configuration - * @return a new instance of {@link BlockLocationPolicy} - */ - public static BlockLocationPolicy create(Class blockLocationPolicyClass, - AlluxioConfiguration conf) { - try { - Class clazz = blockLocationPolicyClass - .asSubclass(BlockLocationPolicy.class); - return CommonUtils.createNewClassInstance(clazz, new Class[] {AlluxioConfiguration.class}, - new Object[] {conf}); - } catch (ClassCastException e) { - throw new RuntimeException(e); - } - } - } - - /** - * Gets the worker's network address for serving operations requested for the block. - * - * @param options the options to get a block worker network address for a block - * @return the address of the worker to write to, or empty if no worker can be selected - */ - Optional getWorker(GetWorkerOptions options); -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/CapacityBaseRandomPolicy.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/CapacityBaseRandomPolicy.java deleted file mode 100644 index 11c1c0dc2ce9..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/CapacityBaseRandomPolicy.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.wire.WorkerNetAddress; - -import java.util.Optional; -import java.util.TreeMap; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicLong; -import javax.annotation.concurrent.ThreadSafe; - -/** - * Randomly distribute workload based on the worker capacities so bigger workers get more requests. - * The randomness is based on the capacity instead of availability because in the long run, - * all workers will be filled up and have availability close to 0. - * We do not want the policy to degenerate to all workers having the same chance. - */ -@ThreadSafe -public class CapacityBaseRandomPolicy implements BlockLocationPolicy { - - /** - * Constructs a new {@link CapacityBaseRandomPolicy} - * needed for instantiation in {@link BlockLocationPolicy.Factory}. - * - * @param ignoredConf is unused - */ - public CapacityBaseRandomPolicy(AlluxioConfiguration ignoredConf) { - } - - @Override - public Optional getWorker(GetWorkerOptions options) { - Iterable blockWorkerInfos = options.getBlockWorkerInfos(); - // All the capacities will form a ring of continuous intervals - // And we throw a die in the ring and decide which worker to pick - // For example if worker1 has capacity 10, worker2 has 20, worker3 has 40, - // the ring will look like [0, 10), [10, 30), [30, 70). - // A key in the map is the LHS of a range. - // So the map will look like {0 -> w1, 10 -> w2, 30 -> w3}. - TreeMap rangeStartMap = new TreeMap<>(); - AtomicLong totalCapacity = new AtomicLong(0L); - blockWorkerInfos.forEach(workerInfo -> { - if (workerInfo.getCapacityBytes() > 0) { - long capacityRangeStart = totalCapacity.getAndAdd(workerInfo.getCapacityBytes()); - rangeStartMap.put(capacityRangeStart, workerInfo); - } - }); - if (totalCapacity.get() == 0L) { - return Optional.empty(); - } - long randomLong = randomInCapacity(totalCapacity.get()); - return Optional.of(rangeStartMap.floorEntry(randomLong).getValue().getNetAddress()); - } - - protected long randomInCapacity(long totalCapacity) { - return ThreadLocalRandom.current().nextLong(totalCapacity); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/CapacityBasedDeterministicHashPolicy.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/CapacityBasedDeterministicHashPolicy.java deleted file mode 100644 index 936ed11770bb..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/CapacityBasedDeterministicHashPolicy.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.PropertyKey; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.Streams; -import org.apache.commons.codec.digest.MurmurHash3; - -import java.util.ArrayList; -import java.util.Comparator; -import java.util.List; -import java.util.Optional; -import java.util.TreeMap; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicLong; - -/** - * A policy that pseudo-randomly distributes blocks between workers according to their capacity, - * so that the probability a worker is chosen is equal to the ratio of its capacity over total - * capacity of all workers, provided that the blocks requested follow a uniform distribution. - * If sharding is disabled, the same block is always assigned to the same worker. If sharding - * is enabled, the block is assigned to a fixed set of workers. - * - * The target worker is determined by the following algorithm: - * 1. build a cumulative distribution function by adding up all workers and their capacities. - * workers are sorted by their host name alphabetically. - * if worker A has 90 GB, B has 10 GB and C has 900 GB, the CDF looks like - * | 0 ... 90 | 90 ... 100 | 100 ... 1000 | - * | worker A | worker B | worker C | - * 2. find a fixed starting point in [0, totalCapacity) determined by the hashed block id. - * | 0 ... 90 | 90 ... 100 | 100 ... 1000 | - * | worker A | worker B | worker C | - * ^ start = 95 - * 3. find the corresponding worker in the CDF. - * which is worker B in this example - * 4. if #shards = 1, this worker is selected. otherwise, find a set of candidates: - * 4.1 hashed_block_id(0) = block id - * 4.2 for i in [1, #shards], hashed_block_id(i) = hash(hashed_block_id(i-1)) - * 4.3 find the worker whose position corresponds to hashed_block_id(i) in the CDF, - * and add it to the candidates set - * 4.4 repeat 4.2 - 4.4 - * 5. select a random worker in the candidate set - * - * The difference between this policy and {@link CapacityBaseRandomPolicy} is that this policy - * uses the hashed block ID as the index to choose the target worker, so that the same block is - * always routed to the same set of workers. - * - * Both this policy and {@link DeterministicHashPolicy} choose workers based the hashed block ID. - * The difference is that {@link DeterministicHashPolicy} uniformly distributes the blocks among - * the configured number of shards, while this policy chooses workers based on a distribution of - * their normalized capacity. - * - * @see CapacityBaseRandomPolicy - * @see DeterministicHashPolicy - */ -public class CapacityBasedDeterministicHashPolicy implements BlockLocationPolicy { - private final int mShards; - - /** - * Constructor required by - * {@link BlockLocationPolicy.Factory#create(Class, AlluxioConfiguration)}. - * @param conf Alluxio configuration - */ - public CapacityBasedDeterministicHashPolicy(AlluxioConfiguration conf) { - int numShards = - conf.getInt(PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS); - Preconditions.checkArgument(numShards >= 1, "number of shards must be no less than 1"); - mShards = numShards; - } - - @Override - public Optional getWorker(GetWorkerOptions options) { - TreeMap capacityCdf = new TreeMap<>(); - AtomicLong totalCapacity = new AtomicLong(0); - Streams.stream(options.getBlockWorkerInfos()) - .filter(workerInfo -> workerInfo.getCapacityBytes() >= options.getBlockInfo().getLength()) - // sort by hostname to guarantee two workers with the same capacity has a defined order - .sorted(Comparator.comparing(w -> w.getNetAddress().getHost())) - .forEach(workerInfo -> { - capacityCdf.put(totalCapacity.get(), workerInfo); - totalCapacity.getAndAdd(workerInfo.getCapacityBytes()); - }); - if (totalCapacity.get() == 0 || capacityCdf.isEmpty()) { - return Optional.empty(); - } - long blockId = options.getBlockInfo().getBlockId(); - BlockWorkerInfo chosenWorker = pickWorker(capacityCdf, blockId, totalCapacity.get()); - return Optional.of(chosenWorker.getNetAddress()); - } - - private BlockWorkerInfo pickWorker(TreeMap capacityCdf, - long blockId, long totalCapacity) { - if (mShards == 1) { - // if no sharding, simply return the worker corresponding to the start point - long startPoint = Math.abs(hashBlockId(blockId)) % totalCapacity; - return capacityCdf.floorEntry(startPoint).getValue(); - } - long hashedBlockId = blockId; - List candidates = new ArrayList<>(); - for (int i = 1; i <= Math.min(mShards, capacityCdf.size()); i++) { - hashedBlockId = hashBlockId(hashedBlockId); - BlockWorkerInfo candidate = capacityCdf - .floorEntry(Math.abs(hashedBlockId) % totalCapacity) // non-null as capacities >= 0 - .getValue(); - candidates.add(candidate); - } - return getRandomCandidate(candidates); - } - - @VisibleForTesting - protected long hashBlockId(long blockId) { - return MurmurHash3.hash64(blockId); - } - - @VisibleForTesting - protected BlockWorkerInfo getRandomCandidate(List candidates) { - int randomIndex = ThreadLocalRandom.current().nextInt(candidates.size()); - return candidates.get(randomIndex); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/DeterministicHashPolicy.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/DeterministicHashPolicy.java deleted file mode 100644 index 1ec409a7f19f..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/DeterministicHashPolicy.java +++ /dev/null @@ -1,122 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.PropertyKey; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.base.MoreObjects; -import com.google.common.base.Objects; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.hash.HashFunction; -import com.google.common.hash.Hashing; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Optional; -import java.util.Random; -import javax.annotation.concurrent.NotThreadSafe; - -/** - * This policy maps the blockId to several deterministic Alluxio workers. The number of workers a - * block can be mapped to can be passed through the constructor. The default is 1. It skips the - * workers that do not have enough capacity to hold the block. - * - * This policy is useful for limiting the amount of replication that occurs when reading blocks from - * the UFS with high concurrency. With 30 workers and 100 remote clients reading the same block - * concurrently, the replication level for the block would get close to 30 as each worker reads - * and caches the block for one or more clients. If the clients use DeterministicHashPolicy with - * 3 shards, the 100 clients will split their reads between just 3 workers, so that the replication - * level for the block will be only 3 when the data is first loaded. - * - * Note that the hash function relies on the number of workers in the cluster, so if the number of - * workers changes, the workers chosen by the policy for a given block will likely change. - */ -@NotThreadSafe -public final class DeterministicHashPolicy implements BlockLocationPolicy { - /** The default number of shards to serve a block. */ - private final int mShards; - private final Random mRandom = new Random(); - private final HashFunction mHashFunc = Hashing.md5(); - - /** - * Constructs a new {@link DeterministicHashPolicy} - * needed for instantiation in {@link BlockLocationPolicy.Factory}. - * - * @param conf Alluxio configuration - */ - public DeterministicHashPolicy(AlluxioConfiguration conf) { - int numShards = - conf.getInt(PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS); - Preconditions.checkArgument(numShards >= 1); - mShards = numShards; - } - - @Override - public Optional getWorker(GetWorkerOptions options) { - List workerInfos = Lists.newArrayList(options.getBlockWorkerInfos()); - workerInfos.sort((o1, o2) -> - o1.getNetAddress().toString().compareToIgnoreCase(o2.getNetAddress().toString())); - HashMap blockWorkerInfoMap = new HashMap<>(); - for (BlockWorkerInfo workerInfo : options.getBlockWorkerInfos()) { - blockWorkerInfoMap.put(workerInfo.getNetAddress(), workerInfo); - } - - List workers = new ArrayList<>(); - // Try the next one if the worker mapped from the blockId doesn't work until all the workers - // are examined. - int hv = - Math.abs(mHashFunc.newHasher().putLong(options.getBlockInfo().getBlockId()).hash().asInt()); - int index = hv % workerInfos.size(); - for (BlockWorkerInfo ignored : workerInfos) { - WorkerNetAddress candidate = workerInfos.get(index).getNetAddress(); - BlockWorkerInfo workerInfo = blockWorkerInfoMap.get(candidate); - if (workerInfo != null - && workerInfo.getCapacityBytes() >= options.getBlockInfo().getLength()) { - workers.add(candidate); - if (workers.size() >= mShards) { - break; - } - } - index = (index + 1) % workerInfos.size(); - } - return workers.isEmpty() ? Optional.empty() : - Optional.of(workers.get(mRandom.nextInt(workers.size()))); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof DeterministicHashPolicy)) { - return false; - } - DeterministicHashPolicy that = (DeterministicHashPolicy) o; - return Objects.equal(mShards, that.mShards); - } - - @Override - public int hashCode() { - return Objects.hashCode(mShards); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("shards", mShards).toString(); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/LocalFirstAvoidEvictionPolicy.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/LocalFirstAvoidEvictionPolicy.java deleted file mode 100644 index 1abb0f5d0f6b..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/LocalFirstAvoidEvictionPolicy.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.wire.TieredIdentity; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.MoreObjects; -import com.google.common.base.Objects; -import com.google.common.collect.Lists; - -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import javax.annotation.concurrent.ThreadSafe; - -/** - * A policy that returns the local worker first, and if the local worker doesn't - * exist or doesn't have enough availability, will select the nearest worker from the active - * workers list with sufficient availability. - * - * The definition of 'nearest worker' is based on {@link alluxio.wire.TieredIdentity}. - * @see alluxio.util.TieredIdentityUtils#nearest - * - * The calculation of which worker gets selected is done for each block write. - * - * The {@link alluxio.conf.PropertyKey.USER_BLOCK_AVOID_EVICTION_POLICY_RESERVED_BYTES} - * (alluxio.user.block.avoid.eviction.policy.reserved.size.bytes) - * is used as buffer space on each worker when calculating available space - * to store each block. - */ -@ThreadSafe -public final class LocalFirstAvoidEvictionPolicy implements BlockLocationPolicy { - private final LocalFirstPolicy mPolicy; - private final long mBlockCapacityReserved; - - /** - * Constructs a new {@link LocalFirstAvoidEvictionPolicy}. - * - * @param conf Alluxio configuration - */ - public LocalFirstAvoidEvictionPolicy(AlluxioConfiguration conf) { - this(conf.getBytes(alluxio.conf.PropertyKey.USER_BLOCK_AVOID_EVICTION_POLICY_RESERVED_BYTES), - conf); - } - - LocalFirstAvoidEvictionPolicy(long blockCapacityReserved, - AlluxioConfiguration conf) { - mPolicy = LocalFirstPolicy.create(conf); - mBlockCapacityReserved = blockCapacityReserved; - } - - @VisibleForTesting - LocalFirstAvoidEvictionPolicy(long blockCapacityReserved, TieredIdentity identity, - AlluxioConfiguration conf) { - mPolicy = new LocalFirstPolicy(identity, conf); - mBlockCapacityReserved = blockCapacityReserved; - } - - @Override - public Optional getWorker(GetWorkerOptions options) { - List allWorkers = Lists.newArrayList(options.getBlockWorkerInfos()); - // Prefer workers with enough availability. - List workers = allWorkers.stream() - .filter(worker -> getAvailableBytes(worker) >= options.getBlockInfo().getLength()) - .collect(Collectors.toList()); - if (workers.isEmpty()) { - workers = allWorkers; - } - GetWorkerOptions filteredWorkers = GetWorkerOptions.defaults() - .setBlockInfo(options.getBlockInfo()) - .setBlockWorkerInfos(workers); - return mPolicy.getWorker(filteredWorkers); - } - - /** - * Calculate the available bytes for a worker with the added buffer of - * {@link alluxio.conf.PropertyKey.USER_BLOCK_AVOID_EVICTION_POLICY_RESERVED_BYTES} - * (alluxio.user.block.avoid.eviction.policy.reserved.size.bytes) - * - * Since the information of BlockWorkerInfo is updated after a file - * completes a write, mCapacityBytes minus mUsedBytes may not be the true available bytes. - * - * @param workerInfo BlockWorkerInfo of the worker - * @return the available bytes of the worker - */ - private long getAvailableBytes(BlockWorkerInfo workerInfo) { - long capacityBytes = workerInfo.getCapacityBytes(); - long usedBytes = workerInfo.getUsedBytes(); - return capacityBytes - usedBytes - mBlockCapacityReserved; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof LocalFirstAvoidEvictionPolicy)) { - return false; - } - LocalFirstAvoidEvictionPolicy that = (LocalFirstAvoidEvictionPolicy) o; - return Objects.equal(mPolicy, that.mPolicy) - && Objects.equal(mBlockCapacityReserved, that.mBlockCapacityReserved); - } - - @Override - public int hashCode() { - return Objects.hashCode(mPolicy); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("policy", mPolicy) - .add("blockCapacityReservered", mBlockCapacityReserved) - .toString(); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/LocalFirstPolicy.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/LocalFirstPolicy.java deleted file mode 100644 index 10d0aadfae2c..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/LocalFirstPolicy.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.client.block.util.BlockLocationUtils; -import alluxio.collections.Pair; -import alluxio.conf.AlluxioConfiguration; -import alluxio.network.TieredIdentityFactory; -import alluxio.wire.TieredIdentity; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; - -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.stream.Collectors; -import javax.annotation.concurrent.ThreadSafe; - -/** - * A policy that returns the local worker first, and if the local worker doesn't - * exist or doesn't have enough capacity, will select the nearest worker from the active - * workers list with sufficient capacity. - * - * The definition of 'nearest worker' is based on {@link alluxio.wire.TieredIdentity}. - * @see alluxio.util.TieredIdentityUtils#nearest - * - * The calculation of which worker gets selected is done for each block write. - */ -@ThreadSafe -public final class LocalFirstPolicy implements BlockLocationPolicy { - private final TieredIdentity mTieredIdentity; - private final AlluxioConfiguration mConf; - - /** - * Constructs a new {@link LocalFirstPolicy}. - * - * @param conf Alluxio configuration - */ - public LocalFirstPolicy(AlluxioConfiguration conf) { - mTieredIdentity = TieredIdentityFactory.localIdentity(conf); - mConf = conf; - } - - static LocalFirstPolicy create(AlluxioConfiguration conf) { - return new LocalFirstPolicy(conf); - } - - @VisibleForTesting - LocalFirstPolicy(TieredIdentity identity, AlluxioConfiguration conf) { - mTieredIdentity = identity; - mConf = conf; - } - - @Override - public Optional getWorker(GetWorkerOptions options) { - List shuffledWorkers = Lists.newArrayList(options.getBlockWorkerInfos()); - Collections.shuffle(shuffledWorkers); - // Workers must have enough capacity to hold the block. - List candidateWorkers = shuffledWorkers.stream() - .filter(worker -> worker.getCapacityBytes() >= options.getBlockInfo().getLength()) - .collect(Collectors.toList()); - - // Try finding the nearest worker. - List addresses = candidateWorkers.stream() - .map(BlockWorkerInfo::getNetAddress) - .filter(Objects::nonNull) - .collect(Collectors.toList()); - Optional> nearest = - BlockLocationUtils.nearest(mTieredIdentity, addresses, mConf); - return nearest.map(Pair::getFirst); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof LocalFirstPolicy)) { - return false; - } - LocalFirstPolicy that = (LocalFirstPolicy) o; - return Objects.equals(mTieredIdentity, that.mTieredIdentity); - } - - @Override - public int hashCode() { - return Objects.hash(mTieredIdentity); - } - - @Override - public String toString() { - return com.google.common.base.MoreObjects.toStringHelper(this) - .add("tieredIdentity", mTieredIdentity) - .toString(); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/MostAvailableFirstPolicy.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/MostAvailableFirstPolicy.java deleted file mode 100644 index e0a604884ebf..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/MostAvailableFirstPolicy.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.base.MoreObjects; - -import java.util.Optional; -import javax.annotation.concurrent.ThreadSafe; - -/** - * A policy that returns the worker with the most available bytes. - */ -@ThreadSafe -public final class MostAvailableFirstPolicy implements BlockLocationPolicy { - - /** - * Constructs a new {@link MostAvailableFirstPolicy} - * needed for instantiation in {@link BlockLocationPolicy.Factory}. - * - * @param ignoredConf is unused - */ - public MostAvailableFirstPolicy(AlluxioConfiguration ignoredConf) {} - - /** - * The policy returns null if no worker is qualified. - */ - @Override - public Optional getWorker(GetWorkerOptions options) { - long mostAvailableBytes = -1; - WorkerNetAddress result = null; - for (BlockWorkerInfo workerInfo : options.getBlockWorkerInfos()) { - if (workerInfo.getCapacityBytes() - workerInfo.getUsedBytes() > mostAvailableBytes) { - mostAvailableBytes = workerInfo.getCapacityBytes() - workerInfo.getUsedBytes(); - result = workerInfo.getNetAddress(); - } - } - return Optional.ofNullable(result); - } - - @Override - public boolean equals(Object o) { - return o instanceof MostAvailableFirstPolicy; - } - - @Override - public int hashCode() { - return 0; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).toString(); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/RoundRobinPolicy.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/RoundRobinPolicy.java deleted file mode 100644 index c7f8935f28e8..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/RoundRobinPolicy.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.base.MoreObjects; -import com.google.common.base.Objects; -import com.google.common.collect.Lists; - -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import javax.annotation.concurrent.NotThreadSafe; - -/** - * A policy that chooses the worker for the next block in a round-robin manner and skips workers - * that do not have enough space. - */ -@NotThreadSafe -public final class RoundRobinPolicy implements BlockLocationPolicy { - private List mWorkerInfoList; - private int mIndex; - private boolean mInitialized = false; - /** This caches the {@link WorkerNetAddress} for the block IDs.*/ - private final HashMap mBlockLocationCache = new HashMap<>(); - - /** - * Constructs a new {@link RoundRobinPolicy}. - * - * @param ignoredConf unused, but needed for instantiation in {@link BlockLocationPolicy.Factory} - */ - public RoundRobinPolicy(AlluxioConfiguration ignoredConf) {} - - /** - * The policy uses the first fetch of worker info list as the base, and visits each of them in a - * round-robin manner in the subsequent calls. The policy doesn't assume the list of worker info - * in the subsequent calls has the same order from the first, and it will skip the workers that - * are no longer active. - * - * Returns null if no worker can be found. - * - * @param options options - * @return the address of the worker to write to - */ - @Override - public Optional getWorker(GetWorkerOptions options) { - Set eligibleAddresses = new HashSet<>(); - for (BlockWorkerInfo info : options.getBlockWorkerInfos()) { - eligibleAddresses.add(info.getNetAddress()); - } - - WorkerNetAddress address = mBlockLocationCache.get(options.getBlockInfo().getBlockId()); - if (address != null && eligibleAddresses.contains(address)) { - return Optional.of(address); - } else { - address = null; - } - - if (!mInitialized) { - mWorkerInfoList = Lists.newArrayList(options.getBlockWorkerInfos()); - Collections.shuffle(mWorkerInfoList); - mIndex = 0; - mInitialized = true; - } - - // at most try all the workers - for (int i = 0; i < mWorkerInfoList.size(); i++) { - WorkerNetAddress candidate = mWorkerInfoList.get(mIndex).getNetAddress(); - BlockWorkerInfo workerInfo = findBlockWorkerInfo(options.getBlockWorkerInfos(), candidate); - mIndex = (mIndex + 1) % mWorkerInfoList.size(); - if (workerInfo != null - && workerInfo.getCapacityBytes() >= options.getBlockInfo().getLength() - && eligibleAddresses.contains(candidate)) { - address = candidate; - break; - } - } - mBlockLocationCache.put(options.getBlockInfo().getBlockId(), address); - return Optional.ofNullable(address); - } - - /** - * @param workerInfoList the list of worker info - * @param address the address to look for - * @return the worker info in the list that matches the host name, null if not found - */ - private BlockWorkerInfo findBlockWorkerInfo(Iterable workerInfoList, - WorkerNetAddress address) { - for (BlockWorkerInfo info : workerInfoList) { - if (info.getNetAddress().equals(address)) { - return info; - } - } - return null; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof RoundRobinPolicy)) { - return false; - } - RoundRobinPolicy that = (RoundRobinPolicy) o; - return Objects.equal(mWorkerInfoList, that.mWorkerInfoList) - && Objects.equal(mIndex, that.mIndex) - && Objects.equal(mInitialized, that.mInitialized) - && Objects.equal(mBlockLocationCache, that.mBlockLocationCache); - } - - @Override - public int hashCode() { - return Objects.hashCode(mWorkerInfoList, mIndex, mInitialized, mBlockLocationCache); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("workerInfoList", mWorkerInfoList) - .add("index", mIndex) - .add("initialized", mInitialized) - .add("blockLocationCache", mBlockLocationCache) - .toString(); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/SpecificHostPolicy.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/SpecificHostPolicy.java deleted file mode 100644 index 30224aaf1c1e..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/SpecificHostPolicy.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.PropertyKey; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.base.MoreObjects; -import com.google.common.base.Objects; -import com.google.common.base.Preconditions; - -import java.util.Optional; -import javax.annotation.Nullable; -import javax.annotation.concurrent.ThreadSafe; - -/** - * Always returns a worker with the hostname specified by - * {@link PropertyKey#WORKER_HOSTNAME} (alluxio.worker.hostname). - */ -@ThreadSafe -public final class SpecificHostPolicy implements BlockLocationPolicy { - private final String mHostname; - @Nullable - private final Integer mRpcPort; - - /** - * Constructs a new {@link SpecificHostPolicy} - * needed for instantiation in {@link BlockLocationPolicy.Factory}. - * - * @param conf Alluxio configuration - */ - public SpecificHostPolicy(AlluxioConfiguration conf) { - this(conf.getString(PropertyKey.WORKER_HOSTNAME), conf.getInt(PropertyKey.WORKER_RPC_PORT)); - } - - /** - * Constructs the policy with the hostname. - * - * @param hostname the name of the host - */ - public SpecificHostPolicy(String hostname) { - this(hostname, null); - } - - /** - * Constructs the policy with the hostname and port. - * - * @param hostname the name of the host - * @param rpcPort the rpc port - */ - public SpecificHostPolicy(String hostname, @Nullable Integer rpcPort) { - mHostname = Preconditions.checkNotNull(hostname, "hostname"); - mRpcPort = rpcPort; - } - - /** - * Returns null if no active worker matches the hostname - * provided in WORKER_HOSTNAME (alluxio.worker.hostname). - */ - @Override - public Optional getWorker(GetWorkerOptions options) { - // find the first worker matching the host name - for (BlockWorkerInfo info : options.getBlockWorkerInfos()) { - if (info.getNetAddress().getHost().equals(mHostname) - && (mRpcPort == null || info.getNetAddress().getRpcPort() == mRpcPort)) { - return Optional.of(info.getNetAddress()); - } - } - return Optional.empty(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof SpecificHostPolicy)) { - return false; - } - SpecificHostPolicy that = (SpecificHostPolicy) o; - return Objects.equal(mHostname, that.mHostname); - } - - @Override - public int hashCode() { - return Objects.hashCode(mHostname); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("hostname", mHostname) - .toString(); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/options/GetWorkerOptions.java b/dora/core/client/fs/src/main/java/alluxio/client/block/policy/options/GetWorkerOptions.java deleted file mode 100644 index 6f73e3093b43..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/policy/options/GetWorkerOptions.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.block.policy.options; - -import alluxio.annotation.PublicApi; -import alluxio.client.block.BlockWorkerInfo; -import alluxio.wire.BlockInfo; - -import com.google.common.base.MoreObjects; -import com.google.common.base.Objects; - -import java.util.List; - -/** - * Method options for - * {@link alluxio.client.block.policy.BlockLocationPolicy#getWorker(GetWorkerOptions)}. - */ -@PublicApi -public final class GetWorkerOptions { - private List mBlockWorkerInfos; - private BlockInfo mBlockInfo; - - /** - * @return the default {@link GetWorkerOptions} - */ - public static GetWorkerOptions defaults() { - return new GetWorkerOptions(); - } - - /** - * Creates a new instance with defaults. - */ - private GetWorkerOptions() { - mBlockInfo = new BlockInfo(); - } - - /** - * @return the list of block worker infos - */ - public BlockInfo getBlockInfo() { - return mBlockInfo; - } - - /** - * @return the list of block worker infos - */ - public Iterable getBlockWorkerInfos() { - return mBlockWorkerInfos; - } - - /** - * @param blockInfo the block information - * @return the updated options - */ - public GetWorkerOptions setBlockInfo(BlockInfo blockInfo) { - mBlockInfo = blockInfo; - return this; - } - - /** - * @param blockWorkerInfos the block worker infos - * @return the updated options - */ - public GetWorkerOptions setBlockWorkerInfos( - List blockWorkerInfos) { - mBlockWorkerInfos = blockWorkerInfos; - return this; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof GetWorkerOptions)) { - return false; - } - GetWorkerOptions that = (GetWorkerOptions) o; - return Objects.equal(mBlockWorkerInfos, that.mBlockWorkerInfos) - && Objects.equal(mBlockInfo, that.mBlockInfo); - } - - @Override - public int hashCode() { - return Objects.hashCode(mBlockWorkerInfos, mBlockInfo); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("blockInfo", mBlockInfo) - .add("blockWorkerInfos", mBlockWorkerInfos) - .toString(); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/stream/BlockInStream.java b/dora/core/client/fs/src/main/java/alluxio/client/block/stream/BlockInStream.java deleted file mode 100644 index 145f39a35809..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/stream/BlockInStream.java +++ /dev/null @@ -1,545 +0,0 @@ -/* - * 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.block.stream; - -import alluxio.Seekable; -import alluxio.client.BoundedStream; -import alluxio.client.CanUnbuffer; -import alluxio.client.PositionedReadable; -import alluxio.client.ReadType; -import alluxio.client.file.FileSystemContext; -import alluxio.client.file.options.InStreamOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.PropertyKey; -import alluxio.exception.PreconditionMessage; -import alluxio.exception.status.OutOfRangeException; -import alluxio.grpc.ReadRequest; -import alluxio.network.protocol.databuffer.DataBuffer; -import alluxio.proto.dataserver.Protocol; -import alluxio.util.LogUtils; -import alluxio.util.io.BufferUtils; -import alluxio.util.network.NettyUtils; -import alluxio.util.network.NetworkAddressUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.InputStream; -import java.nio.ByteBuffer; -import java.util.Objects; -import javax.annotation.concurrent.NotThreadSafe; - -/** - * Provides an {@link InputStream} implementation that is based on {@link DataReader}s to - * stream data chunk by chunk. - */ -@NotThreadSafe -public class BlockInStream extends InputStream implements BoundedStream, Seekable, - PositionedReadable, CanUnbuffer { - private static final Logger LOG = LoggerFactory.getLogger(BlockInStream.class); - - /** the source tracking where the block is from. */ - public enum BlockInStreamSource { - PROCESS_LOCAL, // The block is from a worker in the same process - NODE_LOCAL, // The block is from a separate worker process on the same node - REMOTE, // The block is from a remote worker - UFS // The block is in UFS - } - - private final WorkerNetAddress mAddress; - private final BlockInStreamSource mInStreamSource; - /** The id of the block or UFS file to which this InStream provides access. */ - private final long mId; - /** The size in bytes of the block. */ - private final long mLength; - private final byte[] mSingleByte = new byte[1]; - - /** Current position of the stream, relative to the start of the block. */ - private long mPos = 0; - /** The current data chunk. */ - protected DataBuffer mCurrentChunk; - - protected DataReader mDataReader; - private final DataReader.Factory mDataReaderFactory; - - private boolean mClosed = false; - private boolean mEOF = false; - - /** - * Creates a {@link BlockInStream}. - * - * One of several read behaviors: - * - * - Domain socket - if the data source is the local worker and the local worker has a domain - * socket server - * - Local Loopback Read - if the data source is the local worker - * - Read from remote worker - if the data source is a remote worker - * - UFS Read from worker - if the data source is UFS, read from the UFS policy's designated - * worker (ufs -> local or remote worker -> client) - * - * @param context the file system context - * @param info the block info - * @param dataSource the Alluxio worker which should read the data - * @param dataSourceType the source location of the block - * @param options the InStream options - * @return the {@link BlockInStream} object - */ - public static BlockInStream create(FileSystemContext context, BlockInfo info, - WorkerNetAddress dataSource, BlockInStreamSource dataSourceType, InStreamOptions options) - throws IOException { - long blockId = info.getBlockId(); - long blockSize = info.getLength(); - - if (dataSourceType == BlockInStreamSource.PROCESS_LOCAL) { - // Interaction between the current client and the worker it embedded to should - // go through worker internal communication directly without RPC involves - LOG.debug("Creating worker process local input stream for block {} @ {}", - blockId, dataSource); - return createProcessLocalBlockInStream(context, dataSource, blockId, blockSize, options); - } - - AlluxioConfiguration alluxioConf = context.getClusterConf(); - boolean sourceSupportsDomainSocket = NettyUtils.isDomainSocketSupported(dataSource); - boolean sourceIsLocal = dataSourceType == BlockInStreamSource.NODE_LOCAL; - boolean nettyTransEnabled = - alluxioConf.getBoolean(PropertyKey.USER_NETTY_DATA_TRANSMISSION_ENABLED); - - // use Netty to transfer data - if (nettyTransEnabled) { - // TODO(JiamingMai): implement this logic - LOG.debug("Creating Netty input stream for block {} @ {} from client {} reading through {} (" - + "data locates in the local worker {}, sourceSupportDomainSocket {})", - blockId, dataSource, NetworkAddressUtils.getClientHostName(alluxioConf), dataSource, - sourceIsLocal, sourceSupportsDomainSocket); - return createNettyBlockInStream(context, dataSource, dataSourceType, blockId, - blockSize, options); - } - - // gRPC - LOG.debug("Creating gRPC input stream for block {} @ {} from client {} reading through {} (" - + "data locates in the local worker {}, sourceSupportDomainSocket {})", - blockId, dataSource, NetworkAddressUtils.getClientHostName(alluxioConf), dataSource, - sourceIsLocal, sourceSupportsDomainSocket); - return createGrpcBlockInStream(context, dataSource, dataSourceType, blockId, - blockSize, options); - } - - /** - * Creates a {@link BlockInStream} to read from the worker process-local to this client - * directly without RPC involves, if the block does not exist in this worker, will read from - * the UFS storage via this worker. - * - * @param context the file system context - * @param address the network address of the gRPC data server to read from - * @param blockId the block ID - * @param length the block length - * @param options the in stream options - * @return the {@link BlockInStream} created - */ - private static BlockInStream createProcessLocalBlockInStream(FileSystemContext context, - WorkerNetAddress address, long blockId, long length, InStreamOptions options) { - AlluxioConfiguration conf = context.getClusterConf(); - long chunkSize = conf.getBytes( - PropertyKey.USER_LOCAL_READER_CHUNK_SIZE_BYTES); - return new BlockInStream(new BlockWorkerDataReader.Factory( - context.getProcessLocalWorker().orElseThrow(NullPointerException::new), - blockId, chunkSize, options), - address, BlockInStreamSource.PROCESS_LOCAL, blockId, length); - } - - /** - * Creates a {@link BlockInStream} to read from a Netty data server. - * - * @param context the file system context - * @param address the address of the gRPC data server - * @param blockSource the source location of the block - * @param blockSize the block size - * @param blockId the block id - * @return the {@link BlockInStream} created - */ - private static BlockInStream createNettyBlockInStream(FileSystemContext context, - WorkerNetAddress address, BlockInStreamSource blockSource, - long blockId, long blockSize, InStreamOptions options) { - AlluxioConfiguration conf = context.getClusterConf(); - long chunkSize = conf.getBytes( - PropertyKey.USER_STREAMING_READER_CHUNK_SIZE_BYTES); - // Construct the partial read request - Protocol.ReadRequest.Builder builder = Protocol.ReadRequest.newBuilder() - .setBlockId(blockId) - .setPromote(ReadType.fromProto(options.getOptions().getReadType()).isPromote()) - .setChunkSize(chunkSize); - DataReader.Factory factory = new NettyDataReader.Factory(context, address, builder); - return new BlockInStream(factory, address, blockSource, blockId, blockSize); - } - - /** - * Creates a {@link BlockInStream} to read from a gRPC data server. - * - * @param context the file system context - * @param address the address of the gRPC data server - * @param blockSource the source location of the block - * @param blockSize the block size - * @param blockId the block id - * @return the {@link BlockInStream} created - */ - private static BlockInStream createGrpcBlockInStream(FileSystemContext context, - WorkerNetAddress address, BlockInStreamSource blockSource, - long blockId, long blockSize, InStreamOptions options) { - AlluxioConfiguration conf = context.getClusterConf(); - long chunkSize = conf.getBytes( - PropertyKey.USER_STREAMING_READER_CHUNK_SIZE_BYTES); - // Construct the partial read request - ReadRequest.Builder builder = ReadRequest.newBuilder() - .setBlockId(blockId) - .setPromote(ReadType.fromProto(options.getOptions().getReadType()).isPromote()) - .setOpenUfsBlockOptions(options.getOpenUfsBlockOptions(blockId)) // Add UFS fallback options - .setPositionShort(options.getPositionShort()) - .setChunkSize(chunkSize); - DataReader.Factory factory; - if (context.getClusterConf().getBoolean(PropertyKey.FUSE_SHARED_CACHING_READER_ENABLED) - && blockSize > chunkSize * 4) { - // Heuristic to resolve issues/12146, guarded by alluxio.fuse.shared.caching.reader.enabled - // GrpcDataReader instances are shared across FileInStreams to mitigate seek cost - factory = new SharedGrpcDataReader.Factory(context, address, builder, blockSize); - } else { - factory = new GrpcDataReader.Factory(context, address, builder); - } - return new BlockInStream(factory, address, blockSource, blockId, blockSize); - } - - /** - * Creates a {@link BlockInStream} to read from a specific remote server. Should only be used - * in cases where the data source and method of reading is known, i.e. worker - worker - * communication. - * - * @param context the file system context - * @param blockId the block id - * @param address the address of the gRPC data server - * @param blockSource the source location of the block - * @param blockSize the size of the block - * @param ufsOptions the ufs read options - * @return the {@link BlockInStream} created - */ - public static BlockInStream createRemoteBlockInStream(FileSystemContext context, long blockId, - WorkerNetAddress address, BlockInStreamSource blockSource, long blockSize, - Protocol.OpenUfsBlockOptions ufsOptions) { - AlluxioConfiguration conf = context.getClusterConf(); - long chunkSize = conf.getBytes( - PropertyKey.USER_STREAMING_READER_CHUNK_SIZE_BYTES); - ReadRequest readRequest = ReadRequest.newBuilder().setBlockId(blockId) - .setOpenUfsBlockOptions(ufsOptions).setChunkSize(chunkSize).buildPartial(); - DataReader.Factory factory = new GrpcDataReader.Factory(context, address, - readRequest.toBuilder()); - return new BlockInStream(factory, address, blockSource, blockId, blockSize); - } - - /** - * Creates an instance of {@link BlockInStream}. - * - * @param dataReaderFactory the data reader factory - * @param address the address of the gRPC data server - * @param blockSource the source location of the block - * @param id the ID (either block ID or UFS file ID) - * @param length the length - */ - @VisibleForTesting - protected BlockInStream(DataReader.Factory dataReaderFactory, - WorkerNetAddress address, BlockInStreamSource blockSource, long id, long length) { - mDataReaderFactory = dataReaderFactory; - mAddress = address; - mInStreamSource = blockSource; - mId = id; - mLength = length; - } - - @Override - public long getPos() { - return mPos; - } - - @Override - public int read() throws IOException { - int bytesRead = read(mSingleByte); - if (bytesRead == -1) { - return -1; - } - Preconditions.checkState(bytesRead == 1); - return BufferUtils.byteToInt(mSingleByte[0]); - } - - @Override - public int read(byte[] b) throws IOException { - return read(b, 0, b.length); - } - - @Override - public int read(byte[] b, int off, int len) throws IOException { - Objects.requireNonNull(b, "Read buffer cannot be null"); - return read(ByteBuffer.wrap(b), off, len); - } - - /** - * Reads up to len bytes of data from the input stream into the byte buffer. - * - * @param byteBuffer the buffer into which the data is read - * @param off the start offset in the buffer at which the data is written - * @param len the maximum number of bytes to read - * @return the total number of bytes read into the buffer, or -1 if there is no more data because - * the end of the stream has been reached - */ - public int read(ByteBuffer byteBuffer, int off, int len) throws IOException { - Preconditions.checkArgument(off >= 0 && len >= 0 && len + off <= byteBuffer.capacity(), - PreconditionMessage.ERR_BUFFER_STATE.toString(), byteBuffer.capacity(), off, len); - checkIfClosed(); - if (len == 0) { - - return 0; - } - if (mPos == mLength) { - return -1; - } - readChunk(); - if (mCurrentChunk == null) { - mEOF = true; - } - if (mEOF) { - closeDataReader(); - if (mPos < mLength) { - throw new OutOfRangeException(String.format("Block %s is expected to be %s bytes, " - + "but only %s bytes are available in the UFS. " - + "Please retry the read and on the next access, " - + "Alluxio will sync with the UFS and fetch the updated file content.", - mId, mLength, mPos)); - } - return -1; - } - int toRead = Math.min(len, mCurrentChunk.readableBytes()); - byteBuffer.position(off).limit(off + toRead); - mCurrentChunk.readBytes(byteBuffer); - mPos += toRead; - if (mPos == mLength) { - // a performance improvement introduced by https://github.com/Alluxio/alluxio/issues/14020 - closeDataReader(); - } - return toRead; - } - - @Override - public int positionedRead(long pos, byte[] b, int off, int len) throws IOException { - if (len == 0) { - return 0; - } - if (pos < 0 || pos >= mLength) { - return -1; - } - - int lenCopy = len; - try (DataReader reader = mDataReaderFactory.create(pos, len)) { - // We try to read len bytes instead of returning after reading one chunk because - // it is not free to create/close a DataReader. - while (len > 0) { - DataBuffer dataBuffer = null; - try { - dataBuffer = reader.readChunk(); - if (dataBuffer == null) { - break; - } - Preconditions.checkState(dataBuffer.readableBytes() <= len); - int toRead = dataBuffer.readableBytes(); - dataBuffer.readBytes(b, off, toRead); - len -= toRead; - off += toRead; - } finally { - if (dataBuffer != null) { - dataBuffer.release(); - } - } - } - } - if (lenCopy == len) { - return -1; - } - return lenCopy - len; - } - - @Override - public long remaining() { - return mEOF ? 0 : mLength - mPos; - } - - @Override - public void seek(long pos) throws IOException { - checkIfClosed(); - Preconditions.checkArgument(pos >= 0, PreconditionMessage.ERR_SEEK_NEGATIVE.toString(), pos); - Preconditions.checkArgument(pos <= mLength, - "Seek position past the end of the read region (block or file).", mId); - if (pos == mPos) { - return; - } - // When alluxio.fuse.shared.caching.reader.enabled is on (to resolve issues/12146), - // use the heuristic to improve seek performance with fewer data reader close. - if (mDataReader instanceof SharedGrpcDataReader) { - seekForSharedGrpcDataReader(pos); - return; - } - if (pos < mPos) { - mEOF = false; - } - closeDataReader(); - mPos = pos; - } - - private void seekForSharedGrpcDataReader(long pos) throws IOException { - if (pos < mPos) { - mEOF = false; - // because the reader is shared, let's not close it but simply seek - ((SharedGrpcDataReader) mDataReader).seek(pos); - if (mCurrentChunk != null) { - mCurrentChunk.release(); - mCurrentChunk = null; - } - } else { - // TODO(lu) combine the original seek logic and the following general improvements - // that are helpful in both fuse and non-fuse scenarios - // Try to read data already received but haven't processed - long curPos = mPos; - while (mCurrentChunk != null && curPos < pos) { - long nextPos = curPos + mCurrentChunk.readableBytes(); - if (nextPos <= pos) { - curPos = nextPos; - mCurrentChunk.release(); - mCurrentChunk = mDataReader.readChunk(); - } else { - // TODO(chaowang) introduce seek in DataBuffer - int toRead = (int) (pos - curPos); - final byte[] b = new byte[toRead]; - mCurrentChunk.readBytes(b, 0, toRead); - curPos = pos; - } - } - - if (curPos < pos) { - // Not enough data in queue, close the data reader - closeDataReader(); - } - } - mPos = pos; - } - - @Override - public long skip(long n) throws IOException { - checkIfClosed(); - if (n <= 0) { - return 0; - } - - long toSkip = Math.min(remaining(), n); - mPos += toSkip; - - closeDataReader(); - return toSkip; - } - - @Override - public void close() throws IOException { - if (mClosed) { - return; - } - try { - closeDataReader(); - } finally { - mDataReaderFactory.close(); - } - mClosed = true; - } - - /** - * @return the underlying data reader factory - */ - @VisibleForTesting - public DataReader.Factory getDataReaderFactory() { - return mDataReaderFactory; - } - - /** - * Reads a new chunk from the channel if all of the current chunk is read. - */ - private void readChunk() throws IOException { - if (mDataReader == null) { - mDataReader = mDataReaderFactory.create(mPos, mLength - mPos); - } - - if (mCurrentChunk != null && mCurrentChunk.readableBytes() == 0) { - mCurrentChunk.release(); - mCurrentChunk = null; - } - if (mCurrentChunk == null) { - mCurrentChunk = mDataReader.readChunk(); - } - } - - /** - * Close the current data reader. - */ - private void closeDataReader() throws IOException { - if (mCurrentChunk != null) { - mCurrentChunk.release(); - mCurrentChunk = null; - } - if (mDataReader != null) { - mDataReader.close(); - } - mDataReader = null; - } - - @Override - public void unbuffer() { - try { - closeDataReader(); - } catch (IOException e) { - LogUtils.warnWithException(LOG, "failed to unbuffer the block stream", e); - } - } - - /** - * Convenience method to ensure the stream is not closed. - */ - private void checkIfClosed() { - Preconditions.checkState(!mClosed, "Cannot do operations on a closed BlockInStream"); - } - - /** - * @return the address of the data server - */ - public WorkerNetAddress getAddress() { - return mAddress; - } - - /** - * @return the source of the block location - */ - public BlockInStreamSource getSource() { - return mInStreamSource; - } - - /** - * @return the block ID - */ - public long getId() { - return mId; - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/stream/BlockOutStream.java b/dora/core/client/fs/src/main/java/alluxio/client/block/stream/BlockOutStream.java deleted file mode 100644 index ed07ea9cde90..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/stream/BlockOutStream.java +++ /dev/null @@ -1,289 +0,0 @@ -/* - * 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.block.stream; - -import alluxio.client.BoundedStream; -import alluxio.client.Cancelable; -import alluxio.client.file.FileSystemContext; -import alluxio.client.file.options.OutStreamOptions; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.base.Preconditions; -import com.google.common.io.Closer; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.PooledByteBufAllocator; -import io.netty.buffer.Unpooled; - -import java.io.IOException; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.Objects; -import javax.annotation.concurrent.NotThreadSafe; - -/** - * Provides an {@link OutputStream} implementation that is based on {@link DataWriter} which - * streams data chunk by chunk. - */ -@NotThreadSafe -public class BlockOutStream extends OutputStream implements BoundedStream, Cancelable { - - private final Closer mCloser; - /** Length of the stream. If unknown, set to Long.MAX_VALUE. */ - private final long mLength; - private final WorkerNetAddress mAddress; - private ByteBuf mCurrentChunk = null; - - private final List mDataWriters; - private boolean mClosed; - - /** - * Constructs a new {@link BlockOutStream} with only one {@link DataWriter}. - * - * @param dataWriter the data writer - * @param length the length of the stream - * @param address the Alluxio worker address - */ - public BlockOutStream(DataWriter dataWriter, long length, WorkerNetAddress address) { - mCloser = Closer.create(); - mLength = length; - Objects.requireNonNull(address); - mAddress = address; - mDataWriters = new ArrayList<>(1); - Objects.requireNonNull(dataWriter); - mDataWriters.add(dataWriter); - mCloser.register(dataWriter); - mClosed = false; - } - - /** - * @return the remaining size of the block - */ - @Override - public long remaining() { - long pos = Long.MAX_VALUE; - for (DataWriter dataWriter : mDataWriters) { - pos = Math.min(pos, dataWriter.pos()); - } - return mLength - pos - (mCurrentChunk != null ? mCurrentChunk.readableBytes() : 0); - } - - /** - * Creates a new remote block output stream. - * - * @param context the file system context - * @param blockId the block id - * @param blockSize the block size - * @param workerNetAddresses the worker network addresses - * @param options the options - * @return the {@link BlockOutStream} instance created - */ - public static BlockOutStream createReplicatedBlockOutStream(FileSystemContext context, - long blockId, long blockSize, java.util.List workerNetAddresses, - OutStreamOptions options) throws IOException { - List dataWriters = new ArrayList<>(); - for (WorkerNetAddress address: workerNetAddresses) { - DataWriter dataWriter = - DataWriter.Factory.create(context, blockId, blockSize, address, options); - dataWriters.add(dataWriter); - } - return new BlockOutStream(dataWriters, blockSize, workerNetAddresses); - } - - /** - * Constructs a new {@link BlockOutStream} with only one {@link DataWriter}. - * - * @param dataWriters the data writer - * @param length the length of the stream - * @param workerNetAddresses the worker network addresses - */ - protected BlockOutStream(List dataWriters, long length, - java.util.List workerNetAddresses) { - mCloser = Closer.create(); - mLength = length; - mAddress = workerNetAddresses.get(0); - mDataWriters = dataWriters; - for (DataWriter dataWriter : dataWriters) { - mCloser.register(dataWriter); - } - mClosed = false; - } - - @Override - public void write(int b) throws IOException { - Preconditions.checkState(remaining() > 0, "Cannot write past end of block"); - updateCurrentChunk(false); - mCurrentChunk.writeByte(b); - } - - @Override - public void write(byte[] b) throws IOException { - write(b, 0, b.length); - } - - @Override - public void write(byte[] b, int off, int len) throws IOException { - writeInternal(Unpooled.wrappedBuffer(b), off, len); - } - - /** - * Writes the data in the specified byte buf to this output stream. - * - * @param buf the buffer - * @throws IOException exception - */ - public void write(io.netty.buffer.ByteBuf buf) throws IOException { - write(buf, 0, buf.readableBytes()); - } - - /** - * Writes len bytes from the specified byte buf starting at offset off to this output stream. - * - * @param buf the buffer - * @param off the offset - * @param len the length - */ - public void write(io.netty.buffer.ByteBuf buf, int off, int len) throws IOException { - writeInternal(buf, off, len); - } - - private void writeInternal(ByteBuf b, int off, int len) throws IOException { - if (len == 0) { - return; - } - - while (len > 0) { - updateCurrentChunk(false); - int toWrite = Math.min(len, mCurrentChunk.writableBytes()); - mCurrentChunk.writeBytes(b, off, toWrite); - off += toWrite; - len -= toWrite; - } - updateCurrentChunk(false); - } - - @Override - public void flush() throws IOException { - if (mClosed) { - return; - } - updateCurrentChunk(true); - for (DataWriter dataWriter : mDataWriters) { - dataWriter.flush(); - } - } - - @Override - public void cancel() throws IOException { - if (mClosed) { - return; - } - releaseCurrentChunk(); - - List exceptions = new LinkedList<>(); - for (DataWriter dataWriter : mDataWriters) { - try { - dataWriter.cancel(); - } catch (IOException e) { - exceptions.add(e); - } - } - if (exceptions.size() > 0) { - IOException ex = new IOException("Failed to cancel all block write attempts"); - exceptions.forEach(ex::addSuppressed); - throw ex; - } - - close(); - } - - @Override - public void close() throws IOException { - if (mClosed) { - return; - } - try { - updateCurrentChunk(true); - } catch (Throwable t) { - throw mCloser.rethrow(t); - } finally { - mClosed = true; - mCloser.close(); - } - } - - /** - * @return the worker address for this stream - */ - public WorkerNetAddress getAddress() { - return mAddress; - } - - /** - * Updates the current chunk. - * - * @param lastChunk if the current packet is the last packet - */ - private void updateCurrentChunk(boolean lastChunk) throws IOException { - // Early return for the most common case. - if (mCurrentChunk != null && mCurrentChunk.writableBytes() > 0 && !lastChunk) { - return; - } - - if (mCurrentChunk == null) { - if (!lastChunk) { - mCurrentChunk = allocateBuffer(); - } - return; - } - - if (mCurrentChunk.writableBytes() == 0 || lastChunk) { - try { - if (mCurrentChunk.readableBytes() > 0) { - for (DataWriter dataWriter : mDataWriters) { - mCurrentChunk.retain(); - dataWriter.writeChunk(mCurrentChunk.duplicate()); - } - } else { - Preconditions.checkState(lastChunk); - } - } finally { - // If the packet has bytes to read, we increment its refcount explicitly for every packet - // writer. So we need to release here. If the packet has no bytes to read, then it has - // to be the last packet. It needs to be released as well. - mCurrentChunk.release(); - mCurrentChunk = null; - } - } - if (!lastChunk) { - mCurrentChunk = allocateBuffer(); - } - } - - /** - * Releases the current packet. - */ - private void releaseCurrentChunk() { - if (mCurrentChunk != null) { - mCurrentChunk.release(); - mCurrentChunk = null; - } - } - - /** - * @return a newly allocated byte buffer of the user defined default size - */ - private ByteBuf allocateBuffer() { - return PooledByteBufAllocator.DEFAULT.buffer(mDataWriters.get(0).chunkSize()); - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/block/stream/UnderFileSystemFileOutStream.java b/dora/core/client/fs/src/main/java/alluxio/client/block/stream/UnderFileSystemFileOutStream.java deleted file mode 100644 index 40df073e81d2..000000000000 --- a/dora/core/client/fs/src/main/java/alluxio/client/block/stream/UnderFileSystemFileOutStream.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.block.stream; - -import alluxio.client.file.FileSystemContext; -import alluxio.client.file.options.OutStreamOptions; -import alluxio.grpc.RequestType; -import alluxio.metrics.MetricKey; -import alluxio.metrics.MetricsSystem; -import alluxio.wire.WorkerNetAddress; - -import com.codahale.metrics.Timer; - -import java.io.IOException; -import javax.annotation.concurrent.NotThreadSafe; - -/** - * Provides a streaming API to write to a file in the under file system through an Alluxio - * worker's data server. - */ -@NotThreadSafe -public class UnderFileSystemFileOutStream extends BlockOutStream { - private static final int ID_UNUSED = -1; - private final DataWriter mDataWriter; - - /** - * Creates an instance of {@link UnderFileSystemFileOutStream} that writes to a UFS file. - * - * @param context the file system context - * @param address the data server address - * @param options the out stream options - * @return the under file system output stream - */ - public static UnderFileSystemFileOutStream create(FileSystemContext context, - WorkerNetAddress address, OutStreamOptions options) throws IOException { - return new UnderFileSystemFileOutStream(GrpcDataWriter.create(context, address, - ID_UNUSED, Long.MAX_VALUE, RequestType.UFS_FILE, options), address); - } - - /** - * Constructs a new {@link UnderFileSystemFileOutStream} with only one {@link DataWriter}. - * - * @param dataWriter the data writer - */ - protected UnderFileSystemFileOutStream(DataWriter dataWriter, WorkerNetAddress address) { - super(dataWriter, Long.MAX_VALUE, address); - mDataWriter = dataWriter; - } - - /** - * @return the data writer for the stream - */ - public DataWriter getDataWriter() { - return mDataWriter; - } - - @Override - public void close() throws IOException { - try (Timer.Context ctx = MetricsSystem - .uniformTimer(MetricKey.CLOSE_UFS_OUTSTREAM_LATENCY.getName()).time()) { - super.close(); - } - } -} diff --git a/dora/core/client/fs/src/main/java/alluxio/client/file/FileSystemContext.java b/dora/core/client/fs/src/main/java/alluxio/client/file/FileSystemContext.java index 1d86ceb358d3..1783a0b86c09 100644 --- a/dora/core/client/fs/src/main/java/alluxio/client/file/FileSystemContext.java +++ b/dora/core/client/fs/src/main/java/alluxio/client/file/FileSystemContext.java @@ -19,7 +19,6 @@ import alluxio.client.block.BlockMasterClient; import alluxio.client.block.BlockMasterClientPool; import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.BlockLocationPolicy; import alluxio.client.block.stream.BlockWorkerClient; import alluxio.client.block.stream.BlockWorkerClientPool; import alluxio.client.file.FileSystemContextReinitializer.ReinitBlockerResource; @@ -66,7 +65,6 @@ import java.net.SocketAddress; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -187,8 +185,6 @@ public class FileSystemContext implements Closeable { private final List mMasterAddresses; - private final Map mBlockLocationPolicyMap; - /** * FileSystemContextFactory, it can be extended. */ @@ -411,7 +407,6 @@ protected FileSystemContext(AlluxioConfiguration conf, @Nullable BlockWorker blo new TimeoutRefresh(conf.getMs(PropertyKey.USER_WORKER_LIST_REFRESH_INTERVAL)); LOG.debug("Created context with id: {}, with local block worker: {}", mId, mBlockWorker != null); - mBlockLocationPolicyMap = new ConcurrentHashMap(); } /** @@ -921,32 +916,6 @@ private List getWorkerAddresses() throws IOException { return localWorkerNetAddresses.isEmpty() ? workerNetAddresses : localWorkerNetAddresses; } - /** - * Gets the readBlockLocationPolicy. - * - * @param alluxioConf Alluxio configuration - * - * @return the readBlockLocationPolicy - */ - public BlockLocationPolicy getReadBlockLocationPolicy(AlluxioConfiguration alluxioConf) { - return mBlockLocationPolicyMap.computeIfAbsent( - alluxioConf.getClass(PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY), - pc -> BlockLocationPolicy.Factory.create(pc, alluxioConf)); - } - - /** - * Gets the writeBlockLocationPolicy. - * - * @param alluxioConf Alluxio configuration - * - * @return the writeBlockLocationPolicy - */ - public BlockLocationPolicy getWriteBlockLocationPolicy(AlluxioConfiguration alluxioConf) { - return mBlockLocationPolicyMap.computeIfAbsent( - alluxioConf.getClass(PropertyKey.USER_BLOCK_WRITE_LOCATION_POLICY), - pc -> BlockLocationPolicy.Factory.create(pc, alluxioConf)); - } - /** * Key for block worker client pools. This requires both the worker address and the username, so * that block workers are created for different users. diff --git a/dora/core/client/fs/src/main/java/alluxio/client/file/options/InStreamOptions.java b/dora/core/client/fs/src/main/java/alluxio/client/file/options/InStreamOptions.java index 0ffe5f141f30..a42029084313 100644 --- a/dora/core/client/fs/src/main/java/alluxio/client/file/options/InStreamOptions.java +++ b/dora/core/client/fs/src/main/java/alluxio/client/file/options/InStreamOptions.java @@ -12,8 +12,6 @@ package alluxio.client.file.options; import alluxio.client.ReadType; -import alluxio.client.block.policy.BlockLocationPolicy; -import alluxio.client.block.policy.SpecificHostPolicy; import alluxio.client.file.FileSystemContext; import alluxio.client.file.URIStatus; import alluxio.conf.AlluxioConfiguration; @@ -25,7 +23,6 @@ import alluxio.wire.BlockInfo; import alluxio.wire.FileBlockInfo; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; import com.google.common.base.Preconditions; @@ -44,7 +41,6 @@ public final class InStreamOptions { private final URIStatus mStatus; private final OpenFilePOptions mProtoOptions; - private BlockLocationPolicy mUfsReadLocationPolicy; private boolean mPositionShort; /** @@ -91,13 +87,6 @@ public InStreamOptions(URIStatus status, @Nonnull OpenFilePOptions options, mStatus = status; mProtoOptions = openOptions; - if (options.hasUfsReadWorkerLocation()) { - int port = options.getUfsReadWorkerLocation().getRpcPort(); - mUfsReadLocationPolicy = new SpecificHostPolicy( - options.getUfsReadWorkerLocation().getHost(), port == 0 ? null : port); - } else { - mUfsReadLocationPolicy = context.getReadBlockLocationPolicy(alluxioConf); - } mPositionShort = false; } @@ -108,32 +97,6 @@ public OpenFilePOptions getOptions() { return mProtoOptions; } - /** - * Sets block read location policy. - * - * @param ufsReadLocationPolicy block location policy implementation - */ - @VisibleForTesting - public void setUfsReadLocationPolicy(BlockLocationPolicy ufsReadLocationPolicy) { - mUfsReadLocationPolicy = Preconditions.checkNotNull(ufsReadLocationPolicy); - } - - /** - * Sets whether the operation is positioned read to a small buffer. - * - * @param positionShort whether the operation is positioned read to a small buffer - */ - public void setPositionShort(boolean positionShort) { - mPositionShort = positionShort; - } - - /** - * @return the {@link BlockLocationPolicy} associated with the instream - */ - public BlockLocationPolicy getUfsReadLocationPolicy() { - return mUfsReadLocationPolicy; - } - /** * @return the {@link URIStatus} associated with the instream */ diff --git a/dora/core/client/fs/src/main/java/alluxio/client/file/options/OutStreamOptions.java b/dora/core/client/fs/src/main/java/alluxio/client/file/options/OutStreamOptions.java index 9967e33a2cec..d721b0d45e01 100644 --- a/dora/core/client/fs/src/main/java/alluxio/client/file/options/OutStreamOptions.java +++ b/dora/core/client/fs/src/main/java/alluxio/client/file/options/OutStreamOptions.java @@ -15,8 +15,6 @@ import alluxio.client.AlluxioStorageType; import alluxio.client.UnderStorageType; import alluxio.client.WriteType; -import alluxio.client.block.policy.BlockLocationPolicy; -import alluxio.client.block.policy.SpecificHostPolicy; import alluxio.client.file.FileSystemContext; import alluxio.conf.AlluxioConfiguration; import alluxio.conf.PropertyKey; @@ -43,7 +41,6 @@ public class OutStreamOptions { private FileSystemMasterCommonPOptions mCommonOptions; private long mBlockSizeBytes; - private BlockLocationPolicy mLocationPolicy; private int mWriteTier; private WriteType mWriteType; private String mOwner; @@ -82,7 +79,6 @@ public static OutStreamOptions defaults(FileSystemContext context) { * @param options CreateFile options * @param context Alluxio client context * @param alluxioConf the Alluxio configuration - * @throws Exception if {@link BlockLocationPolicy} can't be loaded */ public OutStreamOptions(CreateFilePOptions options, FileSystemContext context, AlluxioConfiguration alluxioConf) { @@ -114,17 +110,11 @@ public OutStreamOptions(CreateFilePOptions options, FileSystemContext context, if (options.hasWriteType()) { mWriteType = WriteType.fromProto(options.getWriteType()); } - if (options.hasWorkerLocation()) { - int port = options.getWorkerLocation().getRpcPort(); - mLocationPolicy = new SpecificHostPolicy( - options.getWorkerLocation().getHost(), port == 0 ? null : port); - } } protected OutStreamOptions(FileSystemContext context, AlluxioConfiguration alluxioConf) { mCommonOptions = FileSystemOptionsUtils.commonDefaults(alluxioConf); mBlockSizeBytes = alluxioConf.getBytes(PropertyKey.USER_BLOCK_SIZE_BYTES_DEFAULT); - mLocationPolicy = context.getWriteBlockLocationPolicy(alluxioConf); mWriteTier = alluxioConf.getInt(PropertyKey.USER_FILE_WRITE_TIER_DEFAULT); mWriteType = alluxioConf.getEnum(PropertyKey.USER_FILE_WRITE_TYPE_DEFAULT, WriteType.class); try { @@ -168,13 +158,6 @@ public long getBlockSizeBytes() { return mBlockSizeBytes; } - /** - * @return the file write location policy - */ - public BlockLocationPolicy getLocationPolicy() { - return mLocationPolicy; - } - /** * @return the Alluxio storage type */ @@ -306,15 +289,6 @@ public OutStreamOptions setBlockSizeBytes(long blockSizeBytes) { return this; } - /** - * @param locationPolicy the file write location policy - * @return the updated options object - */ - public OutStreamOptions setLocationPolicy(BlockLocationPolicy locationPolicy) { - mLocationPolicy = locationPolicy; - return this; - } - /** * @param commonOptions the FileSystem Master Common POptions(only for copy) * @return the updated options object @@ -441,7 +415,6 @@ public boolean equals(Object o) { && Objects.equal(mBlockSizeBytes, that.mBlockSizeBytes) && Objects.equal(mCommonOptions, that.mCommonOptions) && Objects.equal(mGroup, that.mGroup) - && Objects.equal(mLocationPolicy, that.mLocationPolicy) && Objects.equal(mMediumType, that.mMediumType) && Objects.equal(mMode, that.mMode) && Objects.equal(mMountId, that.mMountId) @@ -462,7 +435,6 @@ public int hashCode() { mBlockSizeBytes, mCommonOptions, mGroup, - mLocationPolicy, mMediumType, mMode, mMountId, @@ -484,7 +456,6 @@ public String toString() { .add("blockSizeBytes", mBlockSizeBytes) .add("commonOptions", mCommonOptions) .add("group", mGroup) - .add("locationPolicy", mLocationPolicy) .add("mediumType", mMediumType) .add("mode", mMode) .add("mountId", mMountId) diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/BlockStoreClientTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/BlockStoreClientTest.java deleted file mode 100644 index dd24de6f9e9a..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/BlockStoreClientTest.java +++ /dev/null @@ -1,535 +0,0 @@ -/* - * 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.block; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import alluxio.ClientContext; -import alluxio.ConfigurationRule; -import alluxio.annotation.dora.DoraTestTodoItem; -import alluxio.client.WriteType; -import alluxio.client.block.policy.BlockLocationPolicy; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.client.block.stream.BlockInStream; -import alluxio.client.block.stream.BlockOutStream; -import alluxio.client.block.stream.BlockWorkerClient; -import alluxio.client.block.stream.BlockWorkerDataReader; -import alluxio.client.block.stream.GrpcDataReader; -import alluxio.client.block.stream.NoopClosableResource; -import alluxio.client.file.FileSystemContext; -import alluxio.client.file.URIStatus; -import alluxio.client.file.options.InStreamOptions; -import alluxio.client.file.options.OutStreamOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.Configuration; -import alluxio.conf.InstancedConfiguration; -import alluxio.conf.PropertyKey; -import alluxio.exception.ExceptionMessage; -import alluxio.exception.PreconditionMessage; -import alluxio.exception.status.UnavailableException; -import alluxio.grpc.OpenFilePOptions; -import alluxio.network.TieredIdentityFactory; -import alluxio.resource.DummyCloseableResource; -import alluxio.util.FileSystemOptionsUtils; -import alluxio.util.network.NetworkAddressUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.BlockLocation; -import alluxio.wire.FileBlockInfo; -import alluxio.wire.FileInfo; -import alluxio.wire.WorkerNetAddress; -import alluxio.worker.block.BlockWorker; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import io.grpc.stub.ClientCallStreamObserver; -import io.grpc.stub.StreamObserver; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mockito; -import org.mockito.stubbing.Answer; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; - -import java.io.Closeable; -import java.util.AbstractMap; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import javax.annotation.concurrent.ThreadSafe; - -/** - * Tests for {@link BlockStoreClient}. - */ -@RunWith(PowerMockRunner.class) -@PrepareForTest({FileSystemContext.class}) -@DoraTestTodoItem(action = DoraTestTodoItem.Action.FIX, owner = "bowen", - comment = "recycle the applicable test cases after we remove BlockStore") -public final class BlockStoreClientTest { - - private static final InstancedConfiguration S_CONF = Configuration.copyGlobal(); - - private static final long BLOCK_ID = 3L; - private static final long BLOCK_LENGTH = 100L; - private static final String WORKER_HOSTNAME_LOCAL = - NetworkAddressUtils.getLocalHostName((int) S_CONF - .getMs(PropertyKey.NETWORK_HOST_RESOLUTION_TIMEOUT_MS)); - private static final String WORKER_HOSTNAME_REMOTE = "remote"; - private static final WorkerNetAddress WORKER_NET_ADDRESS_LOCAL = new WorkerNetAddress() - .setHost(WORKER_HOSTNAME_LOCAL); - private static final WorkerNetAddress WORKER_NET_ADDRESS_REMOTE = new WorkerNetAddress() - .setHost(WORKER_HOSTNAME_REMOTE); - private ClientCallStreamObserver mStreamObserver; - - /** - * A mock class used to return controlled result when selecting workers. - */ - @ThreadSafe - public static class MockBlockLocationPolicyTest implements BlockLocationPolicy { - private List mWorkerNetAddresses; - private int mIndex; - - /** - * Cosntructs this mock location policy with empty host list, - * needed for instantiation in {@link BlockLocationPolicy.Factory}. - * - * @param ignoredConf is unused - */ - public MockBlockLocationPolicyTest(AlluxioConfiguration ignoredConf) { - mIndex = 0; - mWorkerNetAddresses = Collections.emptyList(); - } - - /** - * Constructs this mock policy that returns the given result, once a time, in the input order. - * - * @param addresses list of addresses this mock policy will return - */ - public MockBlockLocationPolicyTest(List addresses) { - mWorkerNetAddresses = Lists.newArrayList(addresses); - mIndex = 0; - } - - public void setHosts(List addresses) { - mWorkerNetAddresses = Lists.newArrayList(addresses); - mIndex = 0; - } - - @Override - public Optional getWorker(GetWorkerOptions options) { - if (mWorkerNetAddresses.isEmpty()) { - return Optional.empty(); - } - return Optional.of(mWorkerNetAddresses.get(mIndex++)); - } - } - - private BlockMasterClient mMasterClient; - private BlockWorkerClient mWorkerClient; - private BlockStoreClient mBlockStore; - private FileSystemContext mContext; - private ClientContext mClientContext; - - @Before - public void before() throws Exception { - mMasterClient = PowerMockito.mock(BlockMasterClient.class); - mWorkerClient = PowerMockito.mock(BlockWorkerClient.class); - - mClientContext = ClientContext.create(S_CONF); - - mContext = PowerMockito.mock(FileSystemContext.class); - when(mContext.acquireBlockMasterClientResource()) - .thenReturn(new DummyCloseableResource<>(mMasterClient)); - when(mContext.getClientContext()).thenReturn(mClientContext); - when(mContext.getClusterConf()).thenReturn(S_CONF); - when(mContext.getReadBlockLocationPolicy(any(AlluxioConfiguration.class))) - .thenAnswer((Answer) invocation -> { - AlluxioConfiguration conf = - invocation.getArgument(0, AlluxioConfiguration.class); - return BlockLocationPolicy.Factory.create( - conf.getClass(PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY), conf); - }); - - mBlockStore = new BlockStoreClient(mContext, - TieredIdentityFactory.fromString("node=" + WORKER_HOSTNAME_LOCAL, S_CONF)); - - when(mContext.acquireBlockWorkerClient(any(WorkerNetAddress.class))) - .thenReturn(new NoopClosableResource<>(mWorkerClient)); - mStreamObserver = PowerMockito.mock(ClientCallStreamObserver.class); - when(mWorkerClient.writeBlock(any(StreamObserver.class))) - .thenReturn(mStreamObserver); - when(mStreamObserver.isReady()).thenReturn(true); - when(mContext.getCachedWorkers()).thenReturn(Lists.newArrayList( - new BlockWorkerInfo(new WorkerNetAddress(), -1, -1))); - } - - @Test - @Ignore - public void getOutStreamUsingLocationPolicy() { - OutStreamOptions options = - OutStreamOptions.defaults(mContext).setWriteType(WriteType.MUST_CACHE) - .setLocationPolicy((workerOptions) -> { - throw new RuntimeException("policy threw exception"); - }); - assertThrows(Exception.class, () -> mBlockStore.getOutStream(BLOCK_ID, BLOCK_LENGTH, options)); - } - - @Test - @Ignore - public void getOutStreamMissingLocationPolicy() { - OutStreamOptions options = - OutStreamOptions.defaults(mContext).setBlockSizeBytes(BLOCK_LENGTH) - .setWriteType(WriteType.MUST_CACHE).setLocationPolicy(null); - Exception e = assertThrows(NullPointerException.class, () -> - mBlockStore.getOutStream(BLOCK_ID, BLOCK_LENGTH, options)); - assertTrue(e.getMessage() - .contains(PreconditionMessage.BLOCK_WRITE_LOCATION_POLICY_UNSPECIFIED.toString())); - } - - @Test - @Ignore - public void getOutStreamNoWorker() { - OutStreamOptions options = - OutStreamOptions - .defaults(mContext) - .setBlockSizeBytes(BLOCK_LENGTH) - .setWriteType(WriteType.MUST_CACHE) - .setLocationPolicy( - new MockBlockLocationPolicyTest(Lists.newArrayList())); - Exception e = assertThrows(UnavailableException.class, () -> - mBlockStore.getOutStream(BLOCK_ID, BLOCK_LENGTH, options)); - assertTrue(e.getMessage() - .contains(ExceptionMessage.NO_SPACE_FOR_BLOCK_ON_WORKER.getMessage(BLOCK_LENGTH))); - } - - @Test - @Ignore - public void getOutStreamRemote() throws Exception { - WorkerNetAddress worker1 = new WorkerNetAddress().setHost("worker1"); - WorkerNetAddress worker2 = new WorkerNetAddress().setHost("worker2"); - OutStreamOptions options = - OutStreamOptions.defaults(mContext).setBlockSizeBytes(BLOCK_LENGTH) - .setLocationPolicy(new MockBlockLocationPolicyTest(Arrays.asList(worker1, worker2))) - .setWriteType(WriteType.MUST_CACHE); - BlockOutStream stream1 = mBlockStore.getOutStream(BLOCK_ID, BLOCK_LENGTH, options); - assertEquals(worker1, stream1.getAddress()); - BlockOutStream stream2 = mBlockStore.getOutStream(BLOCK_ID, BLOCK_LENGTH, options); - assertEquals(worker2, stream2.getAddress()); - } - - @Test - @Ignore - public void getInStreamUfsMockLocaltion() throws Exception { - try (Closeable ignored = new ConfigurationRule(PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY, - MockBlockLocationPolicyTest.class.getTypeName(), S_CONF).toResource()) { - WorkerNetAddress worker1 = new WorkerNetAddress().setHost("worker1"); - WorkerNetAddress worker2 = new WorkerNetAddress().setHost("worker2"); - BlockInfo info = new BlockInfo().setBlockId(0); - URIStatus dummyStatus = new URIStatus(new FileInfo().setPersisted(true) - .setBlockIds(Collections.singletonList(0L)) - .setFileBlockInfos(Collections.singletonList(new FileBlockInfo().setBlockInfo(info)))); - OpenFilePOptions readOptions = OpenFilePOptions.newBuilder().build(); - InStreamOptions options = new InStreamOptions(dummyStatus, readOptions, S_CONF, mContext); - ((MockBlockLocationPolicyTest) options.getUfsReadLocationPolicy()) - .setHosts(Arrays.asList(worker1, worker2)); - when(mMasterClient.getBlockInfo(BLOCK_ID)).thenReturn(new BlockInfo()); - when(mContext.getCachedWorkers()).thenReturn( - Lists.newArrayList(new BlockWorkerInfo(worker1, -1, -1), - new BlockWorkerInfo(worker2, -1, -1))); - - // Location policy chooses worker1 first. - assertEquals(worker1, mBlockStore.getInStream(BLOCK_ID, options).getAddress()); - // Location policy chooses worker2 second. - assertEquals(worker2, mBlockStore.getInStream(BLOCK_ID, options).getAddress()); - } - } - - @Test - @Ignore - public void getInStreamUfsLocalFirst() throws Exception { - WorkerNetAddress remote = new WorkerNetAddress().setHost("remote"); - WorkerNetAddress local = new WorkerNetAddress().setHost(WORKER_HOSTNAME_LOCAL); - BlockInfo info = new BlockInfo().setBlockId(0); - URIStatus dummyStatus = - new URIStatus(new FileInfo().setPersisted(true).setBlockIds(Collections.singletonList(0L)) - .setFileBlockInfos(Collections.singletonList(new FileBlockInfo().setBlockInfo(info)))); - OpenFilePOptions readOptions = OpenFilePOptions.newBuilder().build(); - InStreamOptions options = new InStreamOptions(dummyStatus, readOptions, S_CONF, mContext); - when(mMasterClient.getBlockInfo(BLOCK_ID)).thenReturn(new BlockInfo()); - when(mContext.getCachedWorkers()).thenReturn( - Lists.newArrayList(new BlockWorkerInfo(remote, 100, 0), - new BlockWorkerInfo(local, 100, 0))); - - BlockInStream stream = mBlockStore.getInStream(BLOCK_ID, options); - assertEquals(local, stream.getAddress()); - assertEquals(GrpcDataReader.Factory.class.getName(), - stream.getDataReaderFactory().getClass().getName()); - } - - @Test - @Ignore - public void getInStreamNoWorkers() throws Exception { - URIStatus dummyStatus = - new URIStatus(new FileInfo().setPersisted(true).setBlockIds(Collections.singletonList(0L))); - InStreamOptions options = - new InStreamOptions(dummyStatus, FileSystemOptionsUtils.openFileDefaults(S_CONF), - S_CONF, mContext); - when(mMasterClient.getBlockInfo(BLOCK_ID)).thenReturn(new BlockInfo()); - when(mContext.getCachedWorkers()).thenReturn(Collections.emptyList()); - Exception e = assertThrows(UnavailableException.class, () -> - mBlockStore.getInStream(BLOCK_ID, options).getAddress()); - assertTrue(e.getMessage().contains(ExceptionMessage.NO_WORKER_AVAILABLE.getMessage())); - } - - @Test - @Ignore - public void getInStreamMissingBlock() throws Exception { - URIStatus dummyStatus = new URIStatus( - new FileInfo().setPersisted(false).setBlockIds(Collections.singletonList(0L))); - InStreamOptions options = - new InStreamOptions(dummyStatus, FileSystemOptionsUtils.openFileDefaults(S_CONF), - S_CONF, mContext); - when(mMasterClient.getBlockInfo(BLOCK_ID)).thenReturn(new BlockInfo()); - Exception e = assertThrows(UnavailableException.class, () -> - mBlockStore.getInStream(BLOCK_ID, options).getAddress()); - assertTrue(e.getMessage().contains("unavailable in both Alluxio and UFS")); - } - - @Test - @Ignore - public void getInStreamRemote() throws Exception { - WorkerNetAddress remote1 = new WorkerNetAddress().setHost("remote1"); - WorkerNetAddress remote2 = new WorkerNetAddress().setHost("remote2"); - - BlockInfo info = new BlockInfo().setBlockId(BLOCK_ID).setLocations(Arrays - .asList(new BlockLocation().setWorkerAddress(remote1), - new BlockLocation().setWorkerAddress(remote2))); - - when(mMasterClient.getBlockInfo(BLOCK_ID)).thenReturn(info); - // We should sometimes get remote1 and sometimes get remote2. - Set results = new HashSet<>(); - for (int i = 0; i < 40; i++) { - results.add(mBlockStore.getInStream(BLOCK_ID, new InStreamOptions( - new URIStatus(new FileInfo().setBlockIds(Lists.newArrayList(BLOCK_ID))), - S_CONF, mContext)) - .getAddress()); - } - assertEquals(Sets.newHashSet(remote1, remote2), results); - } - - @Test - @Ignore - public void getInStreamProcessLocal() throws Exception { - WorkerNetAddress remote = new WorkerNetAddress().setHost("remote"); - WorkerNetAddress local = new WorkerNetAddress().setHost(WORKER_HOSTNAME_LOCAL); - BlockInfo info = new BlockInfo().setBlockId(BLOCK_ID).setLocations(Arrays - .asList(new BlockLocation().setWorkerAddress(remote), - new BlockLocation().setWorkerAddress(local))); - when(mMasterClient.getBlockInfo(BLOCK_ID)).thenReturn(info); - - when(mContext.hasProcessLocalWorker()).thenReturn(true); - Optional blockWorker = Optional.of(Mockito.mock(BlockWorker.class)); - when(mContext.getProcessLocalWorker()).thenReturn(blockWorker); - - BlockInStream stream = mBlockStore.getInStream(BLOCK_ID, new InStreamOptions( - new URIStatus(new FileInfo().setBlockIds(Lists.newArrayList(BLOCK_ID))), - S_CONF, mContext)); - assertEquals(local, stream.getAddress()); - assertEquals(BlockWorkerDataReader.Factory.class.getName(), - stream.getDataReaderFactory().getClass().getName()); - } - - @Test - @Ignore - public void getInStreamUfsProcessLocal() throws Exception { - WorkerNetAddress remote = new WorkerNetAddress().setHost("remote"); - WorkerNetAddress local = new WorkerNetAddress().setHost(WORKER_HOSTNAME_LOCAL); - BlockInfo info = new BlockInfo().setBlockId(0); - URIStatus dummyStatus = - new URIStatus(new FileInfo().setPersisted(true).setBlockIds(Collections.singletonList(0L)) - .setFileBlockInfos(Collections.singletonList(new FileBlockInfo().setBlockInfo(info)))); - OpenFilePOptions readOptions = OpenFilePOptions.newBuilder().build(); - InStreamOptions options = new InStreamOptions(dummyStatus, readOptions, S_CONF, mContext); - when(mMasterClient.getBlockInfo(BLOCK_ID)).thenReturn(new BlockInfo()); - when(mContext.getCachedWorkers()).thenReturn( - Lists.newArrayList(new BlockWorkerInfo(remote, 100, 0), - new BlockWorkerInfo(local, 100, 0))); - - when(mContext.getNodeLocalWorker()).thenReturn(local); - when(mContext.hasProcessLocalWorker()).thenReturn(true); - Optional blockWorker = Optional.of(Mockito.mock(BlockWorker.class)); - when(mContext.getProcessLocalWorker()).thenReturn(blockWorker); - - BlockInStream stream = mBlockStore.getInStream(BLOCK_ID, options); - assertEquals(local, stream.getAddress()); - assertEquals(BlockWorkerDataReader.Factory.class.getName(), - stream.getDataReaderFactory().getClass().getName()); - } - - @Test - @Ignore - public void getInStreamInAlluxioOnlyFallbackToAvailableWorker() throws Exception { - int workerCount = 4; - boolean persisted = false; - int[] blockLocations = new int[]{2, 3}; - Map failedWorkers = ImmutableMap.of( - 0, 3L, - 1, 1L, - 3, 2L); - int expectedWorker = 2; - testGetInStreamFallback(workerCount, persisted, blockLocations, failedWorkers, expectedWorker); - } - - @Test - @Ignore - public void getInStreamPersistedAndInAlluxioFallbackToUFS() throws Exception { - int workerCount = 3; - boolean persisted = true; - int[] blockLocations = new int[]{0, 2}; - Map failedWorkers = ImmutableMap.of( - 0, 5L, - 2, 2L); - int expectedWorker = 1; - testGetInStreamFallback(workerCount, persisted, blockLocations, failedWorkers, expectedWorker); - } - - @Test - @Ignore - public void getInStreamPersistedFallbackToLeastRecentlyFailed() throws Exception { - int workerCount = 3; - boolean persisted = true; - int[] blockLocations = new int[0]; - Map failedWorkers = ImmutableMap.of( - 0, 5L, - 1, 1L, - 2, 2L); - int expectedWorker = 1; - testGetInStreamFallback(workerCount, persisted, blockLocations, failedWorkers, expectedWorker); - } - - @Test - @Ignore - public void getInStreamInAlluxioOnlyFallbackToLeastRecentlyFailed() throws Exception { - int workerCount = 5; - boolean persisted = false; - int[] blockLocations = new int[]{1, 2, 3}; - Map failedWorkers = ImmutableMap.of( - 0, 5L, - 1, 3L, - 2, 2L, - 3, 4L, - 4, 1L); - int expectedWorker = 2; - testGetInStreamFallback(workerCount, persisted, blockLocations, failedWorkers, expectedWorker); - } - - @Test - @Ignore - public void getInStreamInAlluxioWhenCreateStreamIsFailed() throws Exception { - int workerCount = 5; - boolean persisted = false; - int[] blockLocations = new int[]{2, 3, 4}; - Map failedWorkers = ImmutableMap.of( - 0, 3L, - 1, 1L, - 3, 2L); - int expectedWorker = 2; - WorkerNetAddress[] workers = new WorkerNetAddress[workerCount]; - for (int i = 0; i < workers.length - 1; i++) { - workers[i] = new WorkerNetAddress().setHost(String.format("worker-%d", i)); - } - workers[workers.length - 1] = new WorkerNetAddress().setHost(WORKER_HOSTNAME_LOCAL); - when(mContext.acquireBlockWorkerClient(WORKER_NET_ADDRESS_LOCAL)) - .thenThrow(new UnavailableException("failed to connect to " - + WORKER_NET_ADDRESS_LOCAL.getHost())); - BlockInfo info = new BlockInfo().setBlockId(BLOCK_ID) - .setLocations(Arrays.stream(blockLocations).mapToObj(x -> - new BlockLocation().setWorkerAddress(workers[x])).collect(Collectors.toList())); - URIStatus dummyStatus = - new URIStatus(new FileInfo().setPersisted(persisted) - .setBlockIds(Collections.singletonList(BLOCK_ID)) - .setFileBlockInfos(Collections.singletonList(new FileBlockInfo().setBlockInfo(info)))); - BlockLocationPolicy mockPolicy = mock(BlockLocationPolicy.class); - when(mockPolicy.getWorker(any())).thenAnswer(arg -> arg - .getArgument(0, GetWorkerOptions.class).getBlockWorkerInfos().iterator().next() - .getNetAddress()); - InStreamOptions options = - new InStreamOptions(dummyStatus, FileSystemOptionsUtils.openFileDefaults(S_CONF), - S_CONF, mContext); - options.setUfsReadLocationPolicy(mockPolicy); - when(mMasterClient.getBlockInfo(BLOCK_ID)).thenReturn(info); - when(mContext.getCachedWorkers()).thenReturn( - Arrays.stream(workers) - .map(x -> new BlockWorkerInfo(x, -1, -1)).collect((Collectors.toList()))); - Map failedWorkerAddresses = failedWorkers.entrySet().stream() - .map(x -> new AbstractMap.SimpleImmutableEntry<>(workers[x.getKey()], x.getValue())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - BlockInStream inStream = null; - int i = 2; - while (i-- > 0) { - try { - inStream = mBlockStore.getInStream(BLOCK_ID, options, - failedWorkerAddresses); - } catch (Exception e) { - //do nothing - } - } - assertNotNull(inStream); - assertEquals(workers[expectedWorker], inStream.getAddress()); - } - - private void testGetInStreamFallback(int workerCount, boolean isPersisted, int[] blockLocations, - Map failedWorkers, int expectedWorker) throws Exception { - WorkerNetAddress[] workers = new WorkerNetAddress[workerCount]; - Arrays.setAll(workers, i -> new WorkerNetAddress().setHost(String.format("worker-%d", i))); - BlockInfo info = new BlockInfo().setBlockId(BLOCK_ID) - .setLocations(Arrays.stream(blockLocations).mapToObj(x -> - new BlockLocation().setWorkerAddress(workers[x])).collect(Collectors.toList())); - URIStatus dummyStatus = - new URIStatus(new FileInfo().setPersisted(isPersisted) - .setBlockIds(Collections.singletonList(BLOCK_ID)) - .setFileBlockInfos(Collections.singletonList(new FileBlockInfo().setBlockInfo(info)))); - BlockLocationPolicy mockPolicy = mock(BlockLocationPolicy.class); - when(mockPolicy.getWorker(any())).thenAnswer(arg -> Optional.ofNullable(arg - .getArgument(0, GetWorkerOptions.class).getBlockWorkerInfos().iterator().next() - .getNetAddress())); - InStreamOptions options = - new InStreamOptions(dummyStatus, FileSystemOptionsUtils.openFileDefaults(S_CONF), - S_CONF, mContext); - options.setUfsReadLocationPolicy(mockPolicy); - when(mMasterClient.getBlockInfo(BLOCK_ID)).thenReturn(info); - when(mContext.getCachedWorkers()).thenReturn(Arrays.stream(workers) - .map(x -> new BlockWorkerInfo(x, -1, -1)).collect((Collectors.toList()))); - Map failedWorkerAddresses = failedWorkers.entrySet().stream() - .map(x -> new AbstractMap.SimpleImmutableEntry<>(workers[x.getKey()], x.getValue())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - BlockInStream inStream = mBlockStore.getInStream(BLOCK_ID, options, failedWorkerAddresses); - - assertEquals(workers[expectedWorker], inStream.getAddress()); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/CapacityBaseRandomPolicyTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/policy/CapacityBaseRandomPolicyTest.java deleted file mode 100644 index 7748e42119ba..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/CapacityBaseRandomPolicyTest.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.Configuration; -import alluxio.wire.WorkerNetAddress; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Optional; - -public class CapacityBaseRandomPolicyTest { - - @Test - public void getWorkerDifferentCapacity() { - GetWorkerOptions getWorkerOptions = GetWorkerOptions.defaults(); - ArrayList blockWorkerInfos = new ArrayList<>(); - WorkerNetAddress netAddress1 = new WorkerNetAddress().setHost("1"); - WorkerNetAddress netAddress2 = new WorkerNetAddress().setHost("2"); - WorkerNetAddress netAddress3 = new WorkerNetAddress().setHost("3"); - WorkerNetAddress netAddress4 = new WorkerNetAddress().setHost("4"); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress1, 10, 0)); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress2, 100, 0)); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress3, 0, 0)); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress4, 1000, 0)); - getWorkerOptions.setBlockWorkerInfos(blockWorkerInfos); - Assert.assertEquals(Optional.of(netAddress1), - buildPolicyWithTarget(0).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress1), - buildPolicyWithTarget(7).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress1), - buildPolicyWithTarget(9).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress2), - buildPolicyWithTarget(10).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress2), - buildPolicyWithTarget(70).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress2), - buildPolicyWithTarget(109).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress4), - buildPolicyWithTarget(110).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress4), - buildPolicyWithTarget(700).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress4), - buildPolicyWithTarget(1109).getWorker(getWorkerOptions)); - Optional address = buildPolicyWithTarget(1109).getWorker(getWorkerOptions); - Assert.assertTrue(address.isPresent()); - Assert.assertNotEquals(netAddress1, address.get()); - } - - @Test - public void getWorkerSameCapacity() { - GetWorkerOptions getWorkerOptions = GetWorkerOptions.defaults(); - ArrayList blockWorkerInfos = new ArrayList<>(); - WorkerNetAddress netAddress1 = new WorkerNetAddress().setHost("1"); - WorkerNetAddress netAddress2 = new WorkerNetAddress().setHost("2"); - WorkerNetAddress netAddress3 = new WorkerNetAddress().setHost("3"); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress1, 100, 0)); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress2, 100, 0)); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress3, 100, 0)); - getWorkerOptions.setBlockWorkerInfos(blockWorkerInfos); - Assert.assertEquals(Optional.of(netAddress1), - buildPolicyWithTarget(0).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress1), - buildPolicyWithTarget(7).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress1), - buildPolicyWithTarget(99).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress2), - buildPolicyWithTarget(100).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress2), - buildPolicyWithTarget(156).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress2), - buildPolicyWithTarget(199).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress3), - buildPolicyWithTarget(200).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress3), - buildPolicyWithTarget(211).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.of(netAddress3), - buildPolicyWithTarget(299).getWorker(getWorkerOptions)); - Optional address = buildPolicyWithTarget(299).getWorker(getWorkerOptions); - Assert.assertTrue(address.isPresent()); - Assert.assertNotEquals(netAddress1, address.get()); - } - - @Test - public void testNoMatchWorker() { - GetWorkerOptions getWorkerOptions = GetWorkerOptions.defaults(); - ArrayList blockWorkerInfos = new ArrayList<>(); - WorkerNetAddress netAddress1 = new WorkerNetAddress(); - WorkerNetAddress netAddress2 = new WorkerNetAddress(); - WorkerNetAddress netAddress3 = new WorkerNetAddress(); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress1, 0, 0)); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress2, 0, 0)); - blockWorkerInfos.add(new BlockWorkerInfo(netAddress3, 0, 0)); - getWorkerOptions.setBlockWorkerInfos(blockWorkerInfos); - Assert.assertEquals(Optional.empty(), buildPolicyWithTarget(0).getWorker(getWorkerOptions)); - Assert.assertEquals(Optional.empty(), buildPolicyWithTarget(1009).getWorker(getWorkerOptions)); - } - - /** - * @param targetValue must be in [0,totalCapacity) - */ - private CapacityBaseRandomPolicy buildPolicyWithTarget(final int targetValue) { - return new CapacityBaseRandomPolicy(Configuration.global()) { - @Override - protected long randomInCapacity(long totalCapacity) { - return targetValue; - } - }; - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/CapacityBasedDeterministicHashPolicyTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/policy/CapacityBasedDeterministicHashPolicyTest.java deleted file mode 100644 index cb422716a831..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/CapacityBasedDeterministicHashPolicyTest.java +++ /dev/null @@ -1,316 +0,0 @@ -/* - * 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.block.policy; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.Configuration; -import alluxio.conf.InstancedConfiguration; -import alluxio.conf.PropertyKey; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.ThreadLocalRandom; -import java.util.stream.Collectors; -import java.util.stream.LongStream; - -public class CapacityBasedDeterministicHashPolicyTest { - - private static final CapacityBasedDeterministicHashPolicy NO_SHARDING_POLICY; - private static final CapacityBasedDeterministicHashPolicy THREE_SHARDS_POLICY; - private static final AlluxioConfiguration NO_SHARDING_CONF; - private static final AlluxioConfiguration THREE_SHARDS_CONF; - - static { - InstancedConfiguration noShardingConf = Configuration.copyGlobal(); - noShardingConf.set( - PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS, 1); - NO_SHARDING_CONF = noShardingConf; - InstancedConfiguration threeShardsConf = Configuration.copyGlobal(); - threeShardsConf.set( - PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS, 3); - THREE_SHARDS_CONF = threeShardsConf; - NO_SHARDING_POLICY = new CapacityBasedDeterministicHashPolicy(NO_SHARDING_CONF); - THREE_SHARDS_POLICY = new CapacityBasedDeterministicHashPolicy(THREE_SHARDS_CONF); - } - - @Test - public void basic() { - class TestPolicy extends CapacityBasedDeterministicHashPolicy { - public TestPolicy(AlluxioConfiguration conf) { - super(conf); - } - - @Override - protected long hashBlockId(long blockId) { - return blockId; - } - - @Override - protected BlockWorkerInfo getRandomCandidate(List candidates) { - // always pick the last candidate - Preconditions.checkArgument(candidates.size() >= 1); - return candidates.get(candidates.size() - 1); - } - } - - TestPolicy policy = new TestPolicy(NO_SHARDING_CONF); - - // total capacity: 100 - List blockWorkerInfos = ImmutableList.of( - new BlockWorkerInfo(new WorkerNetAddress().setHost("0"), 10, 0), - new BlockWorkerInfo(new WorkerNetAddress().setHost("1"), 20, 0), - new BlockWorkerInfo(new WorkerNetAddress().setHost("2"), 20, 0), - new BlockWorkerInfo(new WorkerNetAddress().setHost("3"), 0, 0), - new BlockWorkerInfo(new WorkerNetAddress().setHost("4"), 50, 0) - ); - BlockInfo blockInfo = new BlockInfo(); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(blockWorkerInfos) - .setBlockInfo(blockInfo); - - blockInfo.setBlockId(1); - assertEquals("0", policy.getWorker(options).get().getHost()); - blockInfo.setBlockId(5); - assertEquals("0", policy.getWorker(options).get().getHost()); - blockInfo.setBlockId(10); - assertEquals("1", policy.getWorker(options).get().getHost()); - blockInfo.setBlockId(30); - assertEquals("2", policy.getWorker(options).get().getHost()); - blockInfo.setBlockId(50); - assertEquals("4", policy.getWorker(options).get().getHost()); - } - - @Test - public void sharding() { - class TestPolicy extends CapacityBasedDeterministicHashPolicy { - private final long mTotalCapacity; - - public TestPolicy(AlluxioConfiguration conf, long totalCapacity) { - super(conf); - mTotalCapacity = totalCapacity; - } - - @Override - protected long hashBlockId(long blockId) { - // this simulates a hash function that generates a hash value that is either - // the block id itself, or its complement against total capacity - return mTotalCapacity - blockId; - } - - @Override - protected BlockWorkerInfo getRandomCandidate(List candidates) { - // always pick the last candidate - Preconditions.checkArgument(candidates.size() >= 1); - return candidates.get(candidates.size() - 1); - } - } - - // total capacity: 100 - List blockWorkerInfos = ImmutableList.of( - new BlockWorkerInfo(new WorkerNetAddress().setHost("0"), 10, 0), - new BlockWorkerInfo(new WorkerNetAddress().setHost("1"), 20, 0), - new BlockWorkerInfo(new WorkerNetAddress().setHost("2"), 20, 0), - new BlockWorkerInfo(new WorkerNetAddress().setHost("3"), 0, 0), - new BlockWorkerInfo(new WorkerNetAddress().setHost("4"), 50, 0) - ); - BlockInfo blockInfo = new BlockInfo(); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(blockWorkerInfos) - .setBlockInfo(blockInfo); - - InstancedConfiguration shard4Conf = Configuration.copyGlobal(); - shard4Conf - .set(PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS, 4); - TestPolicy policyShard4 = new TestPolicy(shard4Conf, 100); - TestPolicy policyShard3 = new TestPolicy(THREE_SHARDS_CONF, 100); - - // for 3 shards policy, the block ids are hashed 3 times, - // therefore the effective hash value is the block id's complement - // for 4 shards policy, the hash value is the same as the block id - blockInfo.setBlockId(1); - assertEquals("4", policyShard3.getWorker(options).get().getHost()); - assertEquals("0", policyShard4.getWorker(options).get().getHost()); - blockInfo.setBlockId(5); - assertEquals("4", policyShard3.getWorker(options).get().getHost()); - assertEquals("0", policyShard4.getWorker(options).get().getHost()); - blockInfo.setBlockId(10); - assertEquals("4", policyShard3.getWorker(options).get().getHost()); - assertEquals("1", policyShard4.getWorker(options).get().getHost()); - blockInfo.setBlockId(60); - assertEquals("2", policyShard3.getWorker(options).get().getHost()); - assertEquals("4", policyShard4.getWorker(options).get().getHost()); - blockInfo.setBlockId(90); - assertEquals("1", policyShard3.getWorker(options).get().getHost()); - assertEquals("4", policyShard4.getWorker(options).get().getHost()); - } - - /** - * Tests that the probability a worker is chosen is linear to its normalized capacity, - * provided uniform block requests distribution. - */ - @Test - public void linearDistribution() { - final long capacityUpperBound = 1000; - final int numWorkers = 100; - final int numTrials = 100000; - final List capacities = - LongStream.generate(() -> ThreadLocalRandom.current().nextLong(capacityUpperBound)) - .limit(numWorkers).boxed().collect(Collectors.toList()); - final long totalCapacity = capacities.stream().reduce(0L, Long::sum); - - ImmutableMap.Builder workersBuilder = ImmutableMap.builder(); - for (int i = 0; i < numWorkers; i++) { - // used bytes shouldn't matter in case of CapacityBasedDeterministicHashPolicy; - // random number does not affect the outcome of the policy - long randomUsedBytes = ThreadLocalRandom.current().nextLong(); - WorkerNetAddress addr = new WorkerNetAddress().setHost(String.valueOf(i)); - BlockWorkerInfo workerInfo = new BlockWorkerInfo(addr, capacities.get(i), randomUsedBytes); - workersBuilder.put(addr, workerInfo); - } - Map workers = workersBuilder.build(); - - BlockInfo blockInfo = new BlockInfo(); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockInfo(blockInfo) - .setBlockWorkerInfos(ImmutableList.copyOf(workers.values())); - // worker to number of hits map - Map hits = new HashMap<>(); - for (int i = 0; i < numTrials; i++) { - // assume uniform block distribution - blockInfo.setBlockId(ThreadLocalRandom.current().nextLong()); - Optional chosen = THREE_SHARDS_POLICY.getWorker(options); - assertTrue(chosen.isPresent()); - hits.computeIfPresent(chosen.get(), (k, v) -> v + 1); - hits.putIfAbsent(chosen.get(), 1L); - } - // the chance that workers of a particular capacity are chosen converges to - // the ratio of their capacity over total capacity, as the number of trials increases - final double confidence = 0.01; - for (Map.Entry entry : hits.entrySet()) { - long capacity = workers.get(entry.getKey()).getCapacityBytes(); - double normalizedCapacity = capacity * 1.0 / totalCapacity; - double normalizedHits = entry.getValue() * 1.0 / numTrials; - assertTrue(Math.abs(normalizedCapacity - normalizedHits) < confidence); - } - } - - /** - * Tests that the outcome of the policy is deterministic if sharding is turned off. - */ - @Test - public void deterministicChoice() { - List workerInfos = generateBlockWorkerInfos(100, 1); - BlockInfo blockInfo = new BlockInfo().setBlockId(1); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockInfo(blockInfo) - .setBlockWorkerInfos(workerInfos); - WorkerNetAddress chosen = NO_SHARDING_POLICY.getWorker(options).get(); - for (int i = 0; i < 10000; i++) { - Optional workerInfo = NO_SHARDING_POLICY.getWorker(options); - assertTrue(workerInfo.isPresent()); - assertEquals(chosen, workerInfo.get()); - } - } - - /** - * Tests that when sharding is enabled (shards >1), the upper bound of the number of all - * possibly selected workers is the configured shards value. - * - * Note: the lower bound is 1. - */ - @Test - public void numShardsDoesNotExceedConfiguredValue() { - List workerInfos = generateBlockWorkerInfos(100, 1); - BlockInfo blockInfo = new BlockInfo().setBlockId(1); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockInfo(blockInfo) - .setBlockWorkerInfos(workerInfos); - for (int numShards = 1; numShards < 20; numShards++) { - InstancedConfiguration conf = Configuration.copyGlobal(); - conf.set(PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS, - numShards); - CapacityBasedDeterministicHashPolicy policy = new CapacityBasedDeterministicHashPolicy(conf); - Set seenWorkers = new HashSet<>(); - for (int i = 0; i < 1000; i++) { - Optional workerInfo = policy.getWorker(options); - assertTrue(workerInfo.isPresent()); - seenWorkers.add(workerInfo.get()); - } - assertTrue(seenWorkers.size() <= numShards); - } - } - - @Test - public void zeroCapacityWorker() { - List workerInfos = generateBlockWorkerInfos(10, 0); - BlockInfo blockInfo = new BlockInfo().setBlockId(1); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockInfo(blockInfo) - .setBlockWorkerInfos(workerInfos); - assertFalse(NO_SHARDING_POLICY.getWorker(options).isPresent()); - } - - /** - * Tests that two workers with the same capacity has a well-defined order, independent of the - * order they are present in the worker list. - */ - @Test - public void stability() { - List workerInfos = new ArrayList<>(generateBlockWorkerInfos(10, 100)); - BlockInfo blockInfo = new BlockInfo().setBlockId(1); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockInfo(blockInfo) - .setBlockWorkerInfos(workerInfos); - assertTrue(NO_SHARDING_POLICY.getWorker(options).isPresent()); - WorkerNetAddress chosen = NO_SHARDING_POLICY.getWorker(options).get(); - for (int i = 0; i < 100; i++) { - Collections.shuffle(workerInfos); - assertTrue(NO_SHARDING_POLICY.getWorker(options).isPresent()); - assertEquals(chosen, NO_SHARDING_POLICY.getWorker(options).get()); - } - } - - /** - * Generates a list of workers with the same capacity, and with the index as its hostname. - */ - private List generateBlockWorkerInfos(int numWorkers, int capacity) { - ImmutableList.Builder workerInfoBuilder = ImmutableList.builder(); - for (int i = 0; i < numWorkers; i++) { - // used bytes shouldn't matter in case of CapacityBasedDeterministicHashPolicy; - // random number does not affect the outcome of the policy - long randomUsedBytes = ThreadLocalRandom.current().nextLong(); - WorkerNetAddress addr = new WorkerNetAddress().setHost(String.valueOf(i)); - BlockWorkerInfo workerInfo = new BlockWorkerInfo(addr, capacity, randomUsedBytes); - workerInfoBuilder.add(workerInfo); - } - return workerInfoBuilder.build(); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/DeterministicHashPolicyTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/policy/DeterministicHashPolicyTest.java deleted file mode 100644 index 93b1fd6824df..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/DeterministicHashPolicyTest.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * 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.block.policy; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; - -import alluxio.Constants; -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.Configuration; -import alluxio.conf.InstancedConfiguration; -import alluxio.conf.PropertyKey; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; - -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -/** - * Tests {@link DeterministicHashPolicy}. - */ -public final class DeterministicHashPolicyTest { - private static final int PORT = 1; - - private final List mWorkerInfos = new ArrayList<>(); - private static InstancedConfiguration sConf = Configuration.copyGlobal(); - - @Before - public void before() { - mWorkerInfos.clear(); - mWorkerInfos.add(new BlockWorkerInfo( - new WorkerNetAddress().setHost("worker1").setRpcPort(PORT).setDataPort(PORT) - .setWebPort(PORT), Constants.GB, 0)); - mWorkerInfos.add(new BlockWorkerInfo( - new WorkerNetAddress().setHost("worker2").setRpcPort(PORT).setDataPort(PORT) - .setWebPort(PORT), 2 * (long) Constants.GB, 0)); - mWorkerInfos.add(new BlockWorkerInfo( - new WorkerNetAddress().setHost("worker3").setRpcPort(PORT).setDataPort(PORT) - .setWebPort(PORT), 3 * (long) Constants.GB, 0)); - mWorkerInfos.add(new BlockWorkerInfo( - new WorkerNetAddress().setHost("worker4").setRpcPort(PORT).setDataPort(PORT) - .setWebPort(PORT), 3 * (long) Constants.GB, 0)); - } - - @Test - public void getWorkerDeterministically() { - DeterministicHashPolicy policy = (DeterministicHashPolicy) BlockLocationPolicy.Factory.create( - DeterministicHashPolicy.class, sConf); - String host = policy.getWorker(GetWorkerOptions.defaults().setBlockWorkerInfos(mWorkerInfos) - .setBlockInfo(new BlockInfo().setBlockId(1).setLength(2 * (long) Constants.GB))) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost(); - for (int i = 0; i < 10; i++) { - DeterministicHashPolicy p = (DeterministicHashPolicy) BlockLocationPolicy.Factory.create( - DeterministicHashPolicy.class, - sConf); - // For the same block, always return the same worker. - assertEquals(host, p.getWorker( - GetWorkerOptions.defaults().setBlockWorkerInfos(mWorkerInfos) - .setBlockInfo(new BlockInfo().setBlockId(1).setLength(2 * (long) Constants.GB))) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - assertEquals(host, p.getWorker( - GetWorkerOptions.defaults().setBlockWorkerInfos(mWorkerInfos) - .setBlockInfo(new BlockInfo().setBlockId(1).setLength(2 * (long) Constants.GB))) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - } - } - - @Test - public void getWorkerEnoughCapacity() { - DeterministicHashPolicy policy = (DeterministicHashPolicy) BlockLocationPolicy.Factory.create( - DeterministicHashPolicy.class, sConf); - for (long blockId = 0; blockId < 100; blockId++) { - // worker1 does not have enough capacity. It should never be picked. - assertNotEquals("worker1", policy.getWorker( - GetWorkerOptions.defaults().setBlockWorkerInfos(mWorkerInfos) - .setBlockInfo(new BlockInfo().setBlockId(blockId) - .setLength(2 * (long) Constants.GB))) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - } - } - - @Test - public void getWorkerMultipleShards() { - sConf.set(PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS, 2); - DeterministicHashPolicy policy2 = (DeterministicHashPolicy) BlockLocationPolicy.Factory.create( - DeterministicHashPolicy.class, sConf); - Set addresses1 = new HashSet<>(); - Set addresses2 = new HashSet<>(); - for (int i = 0; i < 100; i++) { - addresses1.add(policy2.getWorker( - GetWorkerOptions.defaults().setBlockWorkerInfos(mWorkerInfos) - .setBlockInfo(new BlockInfo().setBlockId(1) - .setLength(2 * (long) Constants.GB))) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - addresses2.add(policy2.getWorker( - GetWorkerOptions.defaults().setBlockWorkerInfos(mWorkerInfos) - .setBlockInfo(new BlockInfo().setBlockId(1) - .setLength(2 * (long) Constants.GB))) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - } - // With sufficient traffic, 2 (= #shards) workers should be picked to serve the block. - assertEquals(2, addresses1.size()); - assertEquals(2, addresses2.size()); - assertEquals(addresses1, addresses2); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/LocalFirstAvoidEvictionPolicyTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/policy/LocalFirstAvoidEvictionPolicyTest.java deleted file mode 100644 index 0b6176c377cf..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/LocalFirstAvoidEvictionPolicyTest.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.block.policy; - -import static alluxio.client.util.ClientTestUtils.worker; -import static org.junit.Assert.assertEquals; - -import alluxio.Constants; -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.Configuration; -import alluxio.conf.PropertyKey; -import alluxio.network.TieredIdentityFactory; -import alluxio.util.network.NetworkAddressUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.testing.EqualsTester; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -/** - * Tests {@link LocalFirstAvoidEvictionPolicy}. The class delegates to {@link LocalFirstPolicy}, so - * most of its functionality is tested in {@link LocalFirstPolicyTest}. - */ -public class LocalFirstAvoidEvictionPolicyTest { - - private final AlluxioConfiguration mConf = Configuration.global(); - - @Test - public void chooseClosestTierAvoidEviction() throws Exception { - List workers = new ArrayList<>(); - workers.add(worker(Constants.GB, Constants.MB, "node2", "rack3")); - workers.add(worker(Constants.GB, 0, "node3", "rack2")); - workers.add(worker(Constants.GB, 0, "node4", "rack3")); - BlockLocationPolicy policy; - WorkerNetAddress chosen; - // local rack with enough availability - policy = new LocalFirstAvoidEvictionPolicy( - mConf.getBytes(PropertyKey.USER_BLOCK_AVOID_EVICTION_POLICY_RESERVED_BYTES), - TieredIdentityFactory.fromString("node=node2,rack=rack3", mConf), mConf); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workers).setBlockInfo(new BlockInfo().setLength(Constants.GB)); - chosen = policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")); - assertEquals("node4", chosen.getTieredIdentity().getTier(0).getValue()); - } - - /** - * Tests that another worker is picked in case the local host does not have enough availability. - */ - @Test - public void getOthersWhenNotEnoughAvailabilityOnLocal() { - String localhostName = NetworkAddressUtils.getLocalHostName(1000); - BlockLocationPolicy policy = new LocalFirstAvoidEvictionPolicy(mConf); - List workers = new ArrayList<>(); - workers.add(worker(Constants.GB, 0, "worker1", "")); - workers.add(worker(Constants.MB, Constants.MB, localhostName, "")); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workers).setBlockInfo(new BlockInfo().setLength(Constants.MB)); - assertEquals("worker1", policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")) - .getHost()); - } - - /** - * Tests that local host is picked if none of the workers has enough availability. - */ - @Test - public void getLocalWhenNoneHasAvailability() { - String localhostName = NetworkAddressUtils.getLocalHostName(1000); - BlockLocationPolicy policy = new LocalFirstAvoidEvictionPolicy(mConf); - List workers = new ArrayList<>(); - workers.add(worker(Constants.GB, Constants.MB, "worker1", "")); - workers.add(worker(Constants.GB, Constants.MB, localhostName, "")); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workers).setBlockInfo(new BlockInfo().setLength(Constants.GB)); - assertEquals(localhostName, policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")) - .getHost()); - } - - @Test - public void equalsTest() { - new EqualsTester() - .addEqualityGroup( - new LocalFirstAvoidEvictionPolicy(mConf), - new LocalFirstAvoidEvictionPolicy(mConf)) - .testEquals(); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/LocalFirstPolicyTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/policy/LocalFirstPolicyTest.java deleted file mode 100644 index 759f4bc6f389..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/LocalFirstPolicyTest.java +++ /dev/null @@ -1,154 +0,0 @@ -/* - * 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.block.policy; - -import static alluxio.client.util.ClientTestUtils.worker; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import alluxio.Constants; -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.Configuration; -import alluxio.conf.PropertyKey; -import alluxio.network.TieredIdentityFactory; -import alluxio.util.network.NetworkAddressUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.testing.EqualsTester; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -/** - * Tests {@link LocalFirstPolicy}. - */ -public final class LocalFirstPolicyTest { - - private static final AlluxioConfiguration S_CONF = Configuration.global(); - private static final int S_RESOLUTION_TIMEOUT = - (int) Configuration.getMs(PropertyKey.NETWORK_HOST_RESOLUTION_TIMEOUT_MS); - - /** - * Tests that the local host is returned first. - */ - @Test - public void getLocalFirst() { - String localhostName = - NetworkAddressUtils.getLocalHostName(S_RESOLUTION_TIMEOUT); - LocalFirstPolicy policy = new LocalFirstPolicy(S_CONF); - List workers = new ArrayList<>(); - workers.add(worker(Constants.GB, "worker1", "")); - workers.add(worker(Constants.GB, localhostName, "")); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workers).setBlockInfo(new BlockInfo().setLength(Constants.MB)); - assertEquals(localhostName, policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - } - - /** - * Tests that another worker is picked in case the local host does not have enough capacity. - */ - @Test - public void getOthersWhenNotEnoughCapacityOnLocal() { - String localhostName = NetworkAddressUtils.getLocalHostName(S_RESOLUTION_TIMEOUT); - LocalFirstPolicy policy = new LocalFirstPolicy(S_CONF); - List workers = new ArrayList<>(); - workers.add(worker(Constants.GB, "worker1", "")); - workers.add(worker(Constants.MB, localhostName, "")); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workers).setBlockInfo(new BlockInfo().setLength(Constants.GB)); - assertEquals("worker1", policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - } - - /** - * Tests that non-local workers are randomly selected. - */ - @Test - public void getOthersRandomly() { - LocalFirstPolicy policy = new LocalFirstPolicy(S_CONF); - List workers = new ArrayList<>(); - workers.add(worker(Constants.GB, "worker1", "")); - workers.add(worker(Constants.GB, "worker2", "")); - - boolean success = false; - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workers).setBlockInfo(new BlockInfo().setLength(Constants.MB)); - for (int i = 0; i < 100; i++) { - String host = policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost(); - if (!host.equals(policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost())) { - success = true; - break; - } - } - assertTrue(success); - } - - @Test - public void chooseClosestTier() throws Exception { - List workers = new ArrayList<>(); - workers.add(worker(Constants.GB, "node2", "rack3")); - workers.add(worker(Constants.GB, "node3", "rack2")); - workers.add(worker(Constants.GB, "node4", "rack3")); - LocalFirstPolicy policy; - WorkerNetAddress chosen; - // local rack - policy = new LocalFirstPolicy(TieredIdentityFactory.fromString("node=node1,rack=rack2", - S_CONF), S_CONF); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workers).setBlockInfo(new BlockInfo().setLength(Constants.GB)); - chosen = policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")); - assertEquals("rack2", chosen.getTieredIdentity().getTier(1).getValue()); - - // local node - policy = new LocalFirstPolicy(TieredIdentityFactory.fromString("node=node4,rack=rack3", - S_CONF), - S_CONF); - chosen = policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")); - assertEquals("node4", chosen.getTieredIdentity().getTier(0).getValue()); - } - - @Test - public void tieredLocalityEnoughSpace() throws Exception { - List workers = new ArrayList<>(); - // Local node doesn't have enough space - workers.add(worker(Constants.MB, "node2", "rack3")); - workers.add(worker(Constants.GB, "node3", "rack2")); - // Local rack has enough space - workers.add(worker(Constants.GB, "node4", "rack3")); - LocalFirstPolicy policy = new LocalFirstPolicy(TieredIdentityFactory - .fromString("node=node2,rack=rack3", S_CONF), S_CONF); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workers).setBlockInfo(new BlockInfo().setLength(Constants.GB)); - WorkerNetAddress chosen = policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")); - assertEquals(workers.get(2).getNetAddress(), chosen); - } - - @Test - public void equalsTest() throws Exception { - new EqualsTester() - .addEqualityGroup(new LocalFirstPolicy(TieredIdentityFactory.fromString("node=x,rack=y", - S_CONF), S_CONF)) - .addEqualityGroup(new LocalFirstPolicy(TieredIdentityFactory.fromString("node=x,rack=z", - S_CONF), S_CONF)) - .testEquals(); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/MostAvailableFirstPolicyTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/policy/MostAvailableFirstPolicyTest.java deleted file mode 100644 index e4bb60e97814..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/MostAvailableFirstPolicyTest.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.Constants; -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.Configuration; -import alluxio.test.util.CommonUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -/** - * Tests {@link MostAvailableFirstPolicy}. - */ -public final class MostAvailableFirstPolicyTest { - private static final int PORT = 1; - - /** - * Tests that the worker with the most available space is chosen. - */ - @Test - public void getMostAvailableWorker() { - List workerInfoList = new ArrayList<>(); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker1") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), Constants.GB, 0)); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker2") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), 2 * (long) Constants.GB, 0)); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker3") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), 3 * (long) Constants.GB, 0)); - MostAvailableFirstPolicy policy = new MostAvailableFirstPolicy(null); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workerInfoList).setBlockInfo(new BlockInfo().setLength(Constants.MB)); - Assert.assertEquals("worker3", - policy.getWorker(options).orElseThrow( - () -> new IllegalStateException("Expected worker3")).getHost()); - } - - @Test - public void equalsTest() { - CommonUtils.testEquals(MostAvailableFirstPolicy.class, - new Class[]{AlluxioConfiguration.class}, - new Object[]{Configuration.global()}); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/RoundRobinPolicyTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/policy/RoundRobinPolicyTest.java deleted file mode 100644 index 0840355b2e67..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/RoundRobinPolicyTest.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.block.policy; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; - -import alluxio.Constants; -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.Configuration; -import alluxio.test.util.CommonUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; - -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -/** - * Tests {@link RoundRobinPolicy}. - */ -public final class RoundRobinPolicyTest { - private static final int PORT = 1; - - /** - * Tests that the correct workers are chosen when round-robin is used. - */ - @Test - public void getWorker() { - List workerInfoList = new ArrayList<>(); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker1") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), Constants.GB, 0)); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker2") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), 2 * (long) Constants.GB, 0)); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker3") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), 3 * (long) Constants.GB, 0)); - RoundRobinPolicy policy = new RoundRobinPolicy(Configuration.global()); - - GetWorkerOptions options = GetWorkerOptions.defaults().setBlockWorkerInfos(workerInfoList) - .setBlockInfo(new BlockInfo().setLength(2 * (long) Constants.GB)); - assertNotEquals( - policy.getWorker(options).orElseThrow( - () -> new IllegalStateException("Expected worker")).getHost(), - policy.getWorker(options.setBlockInfo(options.getBlockInfo().setBlockId(123))) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - - assertEquals( - policy.getWorker(options.setBlockInfo(options.getBlockInfo().setBlockId(555))) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost(), - policy.getWorker(options.setBlockInfo(options.getBlockInfo().setBlockId(555))) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - } - - /** - * Tests that no workers are returned when there are no eligible workers. - */ - @Test - public void getWorkerNoneEligible() { - RoundRobinPolicy policy = new RoundRobinPolicy(Configuration.global()); - GetWorkerOptions options = GetWorkerOptions.defaults().setBlockWorkerInfos(new ArrayList<>()) - .setBlockInfo(new BlockInfo().setLength(2 * (long) Constants.GB)); - assertFalse(policy.getWorker(options).isPresent()); - } - - /** - * Tests that no workers are returned when subsequent calls to the policy have no eligible - * workers. - */ - @Test - public void getWorkerNoneEligibleAfterCache() { - List workerInfoList = new ArrayList<>(); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker1") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), Constants.GB, 0)); - - RoundRobinPolicy policy = new RoundRobinPolicy(Configuration.global()); - GetWorkerOptions options = GetWorkerOptions.defaults().setBlockWorkerInfos(workerInfoList) - .setBlockInfo(new BlockInfo().setLength(Constants.MB)); - assertTrue(policy.getWorker(options).isPresent()); - options.setBlockWorkerInfos(new ArrayList<>()); - assertFalse(policy.getWorker(options).isPresent()); - } - - @Test - public void equalsTest() { - AlluxioConfiguration conf = Configuration.global(); - CommonUtils.testEquals(RoundRobinPolicy.class, new Class[]{AlluxioConfiguration.class}, - new Object[]{conf}); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/SpecificHostPolicyTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/policy/SpecificHostPolicyTest.java deleted file mode 100644 index d63c4aa99fa4..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/SpecificHostPolicyTest.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.block.policy; - -import alluxio.Constants; -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; - -/** - * Tests {@link SpecificHostPolicy}. - */ -public final class SpecificHostPolicyTest { - private static final int PORT = 1; - - /** - * Tests that the correct worker is returned when using the policy. - */ - @Test - public void policy() { - SpecificHostPolicy policy = new SpecificHostPolicy("worker2"); - List workerInfoList = new ArrayList<>(); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker1") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), Constants.GB, 0)); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker2") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), Constants.GB, 0)); - GetWorkerOptions options = GetWorkerOptions.defaults() - .setBlockWorkerInfos(workerInfoList).setBlockInfo(new BlockInfo().setLength(Constants.MB)); - Assert.assertEquals("worker2", - policy.getWorker(options) - .orElseThrow(() -> new IllegalStateException("Expected worker")).getHost()); - } - - /** - * Tests that no worker is chosen when the worker specified in the policy is not part of the - * worker list. - */ - @Test - public void noMatchingHost() { - SpecificHostPolicy policy = new SpecificHostPolicy("worker3"); - List workerInfoList = new ArrayList<>(); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker1F") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), Constants.GB, 0)); - workerInfoList.add(new BlockWorkerInfo(new WorkerNetAddress().setHost("worker2") - .setRpcPort(PORT).setDataPort(PORT).setWebPort(PORT), Constants.GB, 0)); - GetWorkerOptions options = GetWorkerOptions.defaults().setBlockWorkerInfos(workerInfoList) - .setBlockInfo(new BlockInfo().setLength(2 * (long) Constants.GB)); - Assert.assertFalse(policy.getWorker(options).isPresent()); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/options/GetWorkerOptionsTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/policy/options/GetWorkerOptionsTest.java deleted file mode 100644 index e26e582b01bb..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/policy/options/GetWorkerOptionsTest.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.block.policy.options; - -import static org.junit.Assert.assertEquals; - -import alluxio.Constants; -import alluxio.client.block.BlockWorkerInfo; -import alluxio.test.util.CommonUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; - -import org.junit.Test; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - -/** - * Tests for the {@link GetWorkerOptions} class. - */ -public final class GetWorkerOptionsTest { - private static final int PORT = 1; - - private final List mWorkerInfos = new ArrayList<>(); - - /** - * Tests for defaults {@link GetWorkerOptions}. - */ - @Test - public void defaults() throws IOException { - GetWorkerOptions options = GetWorkerOptions.defaults().setBlockInfo(new BlockInfo()); - assertEquals(null, options.getBlockWorkerInfos()); - assertEquals(0, options.getBlockInfo().getBlockId()); - assertEquals(0, options.getBlockInfo().getLength()); - } - - /** - * Tests for setBlockWorkerInfo. - */ - @Test - public void setBlockWorkerInfoTest() { - mWorkerInfos.clear(); - mWorkerInfos.add(new BlockWorkerInfo( - new WorkerNetAddress().setHost("worker1").setRpcPort(PORT).setDataPort(PORT) - .setWebPort(PORT), Constants.GB, 0)); - mWorkerInfos.add(new BlockWorkerInfo( - new WorkerNetAddress().setHost("worker2").setRpcPort(PORT).setDataPort(PORT) - .setWebPort(PORT), 2 * (long) Constants.GB, 0)); - GetWorkerOptions options = GetWorkerOptions.defaults(); - options.setBlockWorkerInfos(mWorkerInfos); - assertEquals(mWorkerInfos, options.getBlockWorkerInfos()); - } - - /** - * Tests for setBlockId and setBlockSize. - */ - @Test - public void fields() { - Random rand = new Random(); - long blockId = rand.nextLong(); - long blockSize = rand.nextLong(); - GetWorkerOptions options = GetWorkerOptions.defaults(); - BlockInfo info = new BlockInfo().setBlockId(blockId).setLength(blockSize); - options.setBlockInfo(info); - assertEquals(info, options.getBlockInfo()); - } - - @Test - public void equalTest() throws Exception { - CommonUtils.testEquals(GetWorkerOptions.class); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/BlockInStreamTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/stream/BlockInStreamTest.java deleted file mode 100644 index d6ebffbe5244..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/BlockInStreamTest.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.block.stream; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.when; - -import alluxio.ClientContext; -import alluxio.annotation.dora.DoraTestTodoItem; -import alluxio.client.file.FileSystemContext; -import alluxio.client.file.URIStatus; -import alluxio.client.file.options.InStreamOptions; -import alluxio.conf.Configuration; -import alluxio.conf.InstancedConfiguration; -import alluxio.util.io.BufferUtils; -import alluxio.util.network.NettyUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.FileInfo; -import alluxio.wire.WorkerNetAddress; -import alluxio.worker.block.BlockWorker; - -import io.grpc.stub.ClientCallStreamObserver; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentMatchers; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; - -import java.util.Collections; -import java.util.Optional; - -/** - * Tests the {@link BlockInStream} class's static methods. - */ -@RunWith(PowerMockRunner.class) -@PrepareForTest({NettyUtils.class}) -@DoraTestTodoItem(action = DoraTestTodoItem.Action.REMOVE, owner = "bowen", - comment = "recycle the test cases after we remove BlockStore") -@Ignore -public class BlockInStreamTest { - private FileSystemContext mMockContext; - private BlockInfo mInfo; - private InStreamOptions mOptions; - private final InstancedConfiguration mConf = Configuration.copyGlobal(); - - @Before - public void before() throws Exception { - BlockWorkerClient workerClient = Mockito.mock(BlockWorkerClient.class); - ClientCallStreamObserver requestObserver = Mockito.mock(ClientCallStreamObserver.class); - when(requestObserver.isReady()).thenReturn(true); - mMockContext = Mockito.mock(FileSystemContext.class); - when(mMockContext.acquireBlockWorkerClient(ArgumentMatchers.any(WorkerNetAddress.class))) - .thenReturn(new NoopClosableResource<>(workerClient)); - when(mMockContext.getClientContext()).thenReturn(ClientContext.create(mConf)); - when(mMockContext.getClusterConf()).thenReturn(mConf); - mInfo = new BlockInfo().setBlockId(1); - mOptions = new InStreamOptions( - new URIStatus(new FileInfo().setBlockIds(Collections.singletonList(1L))), mConf, - mMockContext); - } - - @Test - public void closeReaderAfterReadingAllData() throws Exception { - int chunkSize = 512; - TestDataReader.Factory factory = - new TestDataReader.Factory(chunkSize, BufferUtils.getIncreasingByteArray(2 * chunkSize)); - BlockInStream stream = new BlockInStream(factory, new WorkerNetAddress(), - BlockInStream.BlockInStreamSource.PROCESS_LOCAL, -1, 1024); - - byte[] res = new byte[chunkSize]; - int read; - read = stream.read(res, 0, chunkSize); - TestDataReader reader = factory.getDataReader(); - assertEquals(chunkSize, read); - assertNotNull(reader); - assertFalse(reader.isClosed()); - - // close data reader after reading all data - read = stream.read(res, 0, chunkSize); - assertEquals(chunkSize, read); - assertTrue(reader.isClosed()); - - read = stream.read(res, 0, chunkSize); - assertEquals(-1, read); - assertTrue(reader.isClosed()); - - stream.close(); - assertTrue(reader.isClosed()); - } - - @Test - public void createRemote() throws Exception { - WorkerNetAddress dataSource = new WorkerNetAddress(); - BlockInStream.BlockInStreamSource dataSourceType = BlockInStream.BlockInStreamSource.REMOTE; - BlockInStream stream = - BlockInStream.create(mMockContext, mInfo, dataSource, dataSourceType, mOptions); - assertEquals(GrpcDataReader.Factory.class.getName(), - stream.getDataReaderFactory().getClass().getName()); - } - - @Test - public void createUfs() throws Exception { - WorkerNetAddress dataSource = new WorkerNetAddress(); - BlockInStream.BlockInStreamSource dataSourceType = BlockInStream.BlockInStreamSource.UFS; - BlockInStream stream = - BlockInStream.create(mMockContext, mInfo, dataSource, dataSourceType, mOptions); - assertEquals(GrpcDataReader.Factory.class.getName(), - stream.getDataReaderFactory().getClass().getName()); - } - - @Test - public void createDomainSocketEnabled() throws Exception { - PowerMockito.mockStatic(NettyUtils.class); - PowerMockito - .when(NettyUtils.isDomainSocketAccessible(ArgumentMatchers.any(WorkerNetAddress.class), - ArgumentMatchers.any(InstancedConfiguration.class))) - .thenReturn(true); - PowerMockito - .when(NettyUtils.isDomainSocketSupported(ArgumentMatchers.any(WorkerNetAddress.class))) - .thenReturn(true); - WorkerNetAddress dataSource = new WorkerNetAddress(); - BlockInStream.BlockInStreamSource dataSourceType = BlockInStream.BlockInStreamSource.NODE_LOCAL; - BlockInStream stream = - BlockInStream.create(mMockContext, mInfo, dataSource, dataSourceType, mOptions); - assertEquals(GrpcDataReader.Factory.class.getName(), - stream.getDataReaderFactory().getClass().getName()); - } - - @Test - public void createProcessLocal() throws Exception { - WorkerNetAddress dataSource = new WorkerNetAddress(); - when(mMockContext.getNodeLocalWorker()).thenReturn(dataSource); - when(mMockContext.getClientContext()).thenReturn(ClientContext.create(mConf)); - Optional blockWorker = Optional.of(Mockito.mock(BlockWorker.class)); - when(mMockContext.getProcessLocalWorker()).thenReturn(blockWorker); - BlockInStream.BlockInStreamSource dataSourceType = - BlockInStream.BlockInStreamSource.PROCESS_LOCAL; - BlockInStream stream = - BlockInStream.create(mMockContext, mInfo, dataSource, dataSourceType, mOptions); - assertEquals(BlockWorkerDataReader.Factory.class.getName(), - stream.getDataReaderFactory().getClass().getName()); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/BlockOutStreamTest.java b/dora/core/client/fs/src/test/java/alluxio/client/block/stream/BlockOutStreamTest.java deleted file mode 100644 index eb66afd1f636..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/BlockOutStreamTest.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.block.stream; - -import alluxio.wire.WorkerNetAddress; - -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.nio.ByteBuffer; - -/** - * Tests {@link BlockOutStream}. - */ -public class BlockOutStreamTest { - private static final int CHUNK_SIZE = 128; - - @Test - public void packetWriteException() throws Exception { - DataWriter writer = new FailingTestDataWriter(ByteBuffer.allocate(CHUNK_SIZE)); - BlockOutStream bos = new BlockOutStream(writer, CHUNK_SIZE, new WorkerNetAddress()); - try { - bos.write(new byte[CHUNK_SIZE]); - Assert.fail("Expected write to throw an exception."); - } catch (IOException e) { - // Exception expected, continue. - } - // After an exception, we should still be able to cancel the stream. - // Test succeeds if we do not throw an exception in cancel. - bos.cancel(); - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestBlockInStream.java b/dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestBlockInStream.java deleted file mode 100644 index fd6f756525cc..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestBlockInStream.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.block.stream; - -import alluxio.network.protocol.databuffer.DataBuffer; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.annotations.VisibleForTesting; - -import java.io.IOException; -import java.nio.ByteBuffer; - -/** - * A {@link BlockInStream} which reads from the given byte array. The stream is able to track how - * much bytes that have been read from the extended BlockInStream. - */ -public class TestBlockInStream extends BlockInStream { - /** A field tracks how much bytes read. */ - private int mBytesRead; - private boolean mClosed; - - public TestBlockInStream(byte[] data, long id, long length, - BlockInStreamSource source) { - super(new Factory(data), - new WorkerNetAddress(), source, id, length); - mBytesRead = 0; - } - - @Override - public int read(ByteBuffer byteBuffer, int off, int len) throws IOException { - int bytesRead = super.read(byteBuffer, off, len); - if (bytesRead <= 0) { - return bytesRead; - } - mBytesRead += bytesRead; - return bytesRead; - } - - @Override - public int positionedRead(long pos, byte[] b, int off, int len) throws IOException { - int bytesRead = super.positionedRead(pos, b, off, len); - if (bytesRead <= 0) { - return bytesRead; - } - mBytesRead += bytesRead; - return bytesRead; - } - - public boolean isClosed() { - return mClosed; - } - - @VisibleForTesting - public DataReader getDataReader() { - return mDataReader; - } - - @VisibleForTesting - public DataBuffer getCurrentChunk() { - return mCurrentChunk; - } - - @Override - public void close() throws IOException { - mClosed = true; - super.close(); - } - - /** - * @return how many bytes been read - */ - public int getBytesRead() { - return mBytesRead; - } - - /** - * Factory class to create {@link TestDataReader}s. - */ - public static class Factory implements DataReader.Factory { - private final byte[] mData; - - /** - * Creates an instance of {@link LocalFileDataReader.Factory}. - * - * @param data the data to serve - */ - public Factory(byte[] data) { - mData = data; - } - - @Override - public DataReader create(long offset, long len) { - return new TestDataReader(mData, 128, offset, len); - } - - @Override - public void close() {} - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestBlockOutStream.java b/dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestBlockOutStream.java deleted file mode 100644 index 6f3b05249591..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestBlockOutStream.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.block.stream; - -import alluxio.wire.WorkerNetAddress; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Arrays; - -/** - * Test class for mocking {@link BlockOutStream} and exposing internal state. - */ -public class TestBlockOutStream extends BlockOutStream { - private final ByteBuffer mData; - private boolean mClosed; - private boolean mCanceled; - - /** - * Constructs a new {@link TestBlockOutStream} to be used in tests. - * - * @param data the data to test - * @param blockSize the block size - */ - public TestBlockOutStream(ByteBuffer data, long blockSize) { - super(new TestDataWriter(data), blockSize, new WorkerNetAddress()); - mData = data; - mClosed = false; - mCanceled = false; - } - - public byte[] getWrittenData() { - try { - super.flush(); - } catch (IOException e) { - throw new RuntimeException(e); - } - return Arrays.copyOfRange(mData.array(), 0, mData.position()); - } - - public boolean isClosed() { - return mClosed; - } - - public boolean isCanceled() { - return mCanceled; - } - - @Override - public void close() throws IOException { - super.close(); - mClosed = true; - } - - @Override - public void cancel() throws IOException { - if (mClosed) { - return; - } - super.cancel(); - mCanceled = true; - mClosed = true; - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestUnderFileSystemFileOutStream.java b/dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestUnderFileSystemFileOutStream.java deleted file mode 100644 index eb55e8623dd1..000000000000 --- a/dora/core/client/fs/src/test/java/alluxio/client/block/stream/TestUnderFileSystemFileOutStream.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.block.stream; - -import alluxio.wire.WorkerNetAddress; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Arrays; - -/** - * Test class for mocking {@link UnderFileSystemFileOutStream} and exposing internal state. - */ -public class TestUnderFileSystemFileOutStream extends UnderFileSystemFileOutStream { - private final ByteBuffer mData; - private boolean mClosed; - private boolean mCanceled; - - /** - * Constructs a new {@link TestUnderFileSystemFileOutStream} to be used in tests. - * - * @param data the data to test - */ - public TestUnderFileSystemFileOutStream(ByteBuffer data) { - super(new TestDataWriter(data), new WorkerNetAddress()); - mData = data; - mClosed = false; - mCanceled = false; - } - - public byte[] getWrittenData() { - try { - super.flush(); - } catch (IOException e) { - throw new RuntimeException(e); - } - return Arrays.copyOfRange(mData.array(), 0, mData.position()); - } - - public boolean isClosed() { - return mClosed; - } - - public boolean isCanceled() { - return mCanceled; - } - - @Override - public void close() throws IOException { - super.close(); - mClosed = true; - } - - @Override - public void cancel() throws IOException { - if (mClosed) { - return; - } - super.cancel(); - mCanceled = true; - mClosed = true; - } -} diff --git a/dora/core/client/fs/src/test/java/alluxio/client/file/options/OutStreamOptionsTest.java b/dora/core/client/fs/src/test/java/alluxio/client/file/options/OutStreamOptionsTest.java index a8726f5c2b64..ac6389605fea 100644 --- a/dora/core/client/fs/src/test/java/alluxio/client/file/options/OutStreamOptionsTest.java +++ b/dora/core/client/fs/src/test/java/alluxio/client/file/options/OutStreamOptionsTest.java @@ -12,7 +12,6 @@ package alluxio.client.file.options; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import alluxio.ClientContext; import alluxio.ConfigurationRule; @@ -20,9 +19,6 @@ import alluxio.client.AlluxioStorageType; import alluxio.client.UnderStorageType; import alluxio.client.WriteType; -import alluxio.client.block.policy.BlockLocationPolicy; -import alluxio.client.block.policy.LocalFirstPolicy; -import alluxio.client.block.policy.RoundRobinPolicy; import alluxio.client.file.FileSystemContext; import alluxio.conf.Configuration; import alluxio.conf.InstancedConfiguration; @@ -96,7 +92,6 @@ public void defaults() throws IOException { assertEquals(alluxioType, options.getAlluxioStorageType()); assertEquals(64 * Constants.MB, options.getBlockSizeBytes()); - assertTrue(options.getLocationPolicy() instanceof LocalFirstPolicy); assertEquals("test_user", options.getOwner()); assertEquals("test_group", options.getGroup()); assertEquals(ModeUtils.applyFileUMask(Mode.defaults(), @@ -115,8 +110,6 @@ public void defaults() throws IOException { public void fields() throws Exception { Random random = new Random(); long blockSize = random.nextLong(); - BlockLocationPolicy locationPolicy = new RoundRobinPolicy( - Configuration.global()); String owner = CommonUtils.randomAlphaNumString(10); String group = CommonUtils.randomAlphaNumString(10); Mode mode = new Mode((short) random.nextInt()); @@ -131,7 +124,6 @@ public void fields() throws Exception { ClientContext clientContext = ClientContext.create(mConf); OutStreamOptions options = OutStreamOptions.defaults(FileSystemContext.create(clientContext)); options.setBlockSizeBytes(blockSize); - options.setLocationPolicy(locationPolicy); options.setOwner(owner); options.setGroup(group); options.setMode(mode); @@ -139,7 +131,6 @@ public void fields() throws Exception { options.setWriteType(writeType); assertEquals(blockSize, options.getBlockSizeBytes()); - assertEquals(locationPolicy, options.getLocationPolicy()); assertEquals(owner, options.getOwner()); assertEquals(group, options.getGroup()); assertEquals(mode, options.getMode()); diff --git a/dora/core/common/src/main/java/alluxio/conf/PropertyKey.java b/dora/core/common/src/main/java/alluxio/conf/PropertyKey.java index 39b5520bf61a..d8a7d4aae146 100755 --- a/dora/core/common/src/main/java/alluxio/conf/PropertyKey.java +++ b/dora/core/common/src/main/java/alluxio/conf/PropertyKey.java @@ -5974,13 +5974,6 @@ public String toString() { .setConsistencyCheckLevel(ConsistencyCheckLevel.WARN) .setScope(Scope.CLIENT) .build(); - public static final PropertyKey USER_FILE_COPYFROMLOCAL_BLOCK_LOCATION_POLICY = - classBuilder(Name.USER_FILE_COPYFROMLOCAL_BLOCK_LOCATION_POLICY) - .setDefaultValue("alluxio.client.block.policy.RoundRobinPolicy") - .setDescription("The default location policy for choosing workers for writing a " - + "file's blocks using copyFromLocal command.") - .setScope(Scope.CLIENT) - .build(); public static final PropertyKey USER_FILE_DELETE_UNCHECKED = booleanBuilder(Name.USER_FILE_DELETE_UNCHECKED) .setDefaultValue(false) @@ -6123,14 +6116,6 @@ public String toString() { .setDefaultValue(false) .setScope(Scope.CLIENT) .build(); - public static final PropertyKey USER_BLOCK_WRITE_LOCATION_POLICY = - classBuilder(Name.USER_BLOCK_WRITE_LOCATION_POLICY) - .setDefaultValue("alluxio.client.block.policy.LocalFirstPolicy") - .setDescription("The default location policy for choosing workers for writing a " - + "file's blocks.") - .setConsistencyCheckLevel(ConsistencyCheckLevel.WARN) - .setScope(Scope.CLIENT) - .build(); public static final PropertyKey USER_BLOCK_AVOID_EVICTION_POLICY_RESERVED_BYTES = dataSizeBuilder(Name.USER_BLOCK_AVOID_EVICTION_POLICY_RESERVED_BYTES) .setDefaultValue("0MB") @@ -7012,55 +6997,6 @@ public String toString() { .setConsistencyCheckLevel(ConsistencyCheckLevel.WARN) .setScope(Scope.CLIENT) .build(); - public static final PropertyKey USER_UFS_BLOCK_READ_LOCATION_POLICY = - classBuilder(Name.USER_UFS_BLOCK_READ_LOCATION_POLICY) - .setDefaultValue("alluxio.client.block.policy.LocalFirstPolicy") - .setDescription(format("When an Alluxio client reads a file from the UFS, it " - + "delegates the read to an Alluxio worker. The client uses this policy to choose " - + "which worker to read through. Built-in choices: %s.", Arrays.asList( - javadocLink("alluxio.client.block.policy.CapacityBasedDeterministicHashPolicy"), - javadocLink("alluxio.client.block.policy.CapacityBaseRandomPolicy"), - javadocLink("alluxio.client.block.policy.DeterministicHashPolicy"), - javadocLink("alluxio.client.block.policy.LocalFirstAvoidEvictionPolicy"), - javadocLink("alluxio.client.block.policy.LocalFirstPolicy"), - javadocLink("alluxio.client.block.policy.MostAvailableFirstPolicy"), - javadocLink("alluxio.client.block.policy.RoundRobinPolicy"), - javadocLink("alluxio.client.block.policy.SpecificHostPolicy")))) - .setConsistencyCheckLevel(ConsistencyCheckLevel.WARN) - .setScope(Scope.CLIENT) - .build(); - public static final PropertyKey USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS = - intBuilder(Name.USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS) - .setDefaultValue(1) - .setDescription("When alluxio.user.ufs.block.read.location.policy is set to " - + "alluxio.client.block.policy.DeterministicHashPolicy or " - + "alluxio.client.block.policy.CapacityBasedDeterministicHashPolicy, " - + "this specifies the number of hash shards.") - .setConsistencyCheckLevel(ConsistencyCheckLevel.WARN) - .setScope(Scope.CLIENT) - .build(); - @Deprecated(message = "CapacityBaseRandomPolicy no longer caches block locations. " - + "To make sure a block is always assigned to the same worker, use DeterministicHashPolicy.") - public static final PropertyKey USER_UFS_BLOCK_READ_LOCATION_POLICY_CACHE_SIZE = - intBuilder(Name.USER_UFS_BLOCK_READ_LOCATION_POLICY_CACHE_SIZE) - .setDefaultValue(10000) - .setDescription("Deprecated - When alluxio.user.ufs.block.read.location.policy is set " - + "to alluxio.client.block.policy.CapacityBaseRandomPolicy, " - + "this specifies cache size of block location.") - .setConsistencyCheckLevel(ConsistencyCheckLevel.WARN) - .setScope(Scope.CLIENT) - .build(); - @Deprecated(message = "CapacityBaseRandomPolicy no longer caches block locations. " - + "To make sure a block is always assigned to the same worker, use DeterministicHashPolicy.") - public static final PropertyKey USER_UFS_BLOCK_READ_LOCATION_POLICY_CACHE_EXPIRATION_TIME = - durationBuilder(Name.USER_UFS_BLOCK_READ_LOCATION_POLICY_CACHE_EXPIRATION_TIME) - .setDefaultValue("10min") - .setDescription("Deprecated - When alluxio.user.ufs.block.read.location.policy is set " - + "to alluxio.client.block.policy.CapacityBaseRandomPolicy, " - + "this specifies cache expire time of block location.") - .setConsistencyCheckLevel(ConsistencyCheckLevel.WARN) - .setScope(Scope.CLIENT) - .build(); public static final PropertyKey USER_UFS_BLOCK_READ_CONCURRENCY_MAX = intBuilder(Name.USER_UFS_BLOCK_READ_CONCURRENCY_MAX) .setDefaultValue(Integer.MAX_VALUE) @@ -9158,8 +9094,6 @@ public static final class Name { "alluxio.user.block.worker.client.pool.min"; public static final String USER_BLOCK_WORKER_CLIENT_POOL_MAX = "alluxio.user.block.worker.client.pool.max"; - public static final String USER_BLOCK_WRITE_LOCATION_POLICY = - "alluxio.user.block.write.location.policy.class"; public static final String USER_CLIENT_CACHE_ASYNC_RESTORE_ENABLED = "alluxio.user.client.cache.async.restore.enabled"; public static final String USER_CLIENT_CACHE_ASYNC_WRITE_ENABLED = @@ -9238,8 +9172,6 @@ public static final class Name { public static final String USER_DATE_FORMAT_PATTERN = "alluxio.user.date.format.pattern"; public static final String USER_FILE_BUFFER_BYTES = "alluxio.user.file.buffer.bytes"; public static final String USER_FILE_RESERVED_BYTES = "alluxio.user.file.reserved.bytes"; - public static final String USER_FILE_COPYFROMLOCAL_BLOCK_LOCATION_POLICY = - "alluxio.user.file.copyfromlocal.block.location.policy.class"; public static final String USER_FILE_DELETE_UNCHECKED = "alluxio.user.file.delete.unchecked"; public static final String USER_FILE_MASTER_CLIENT_POOL_SIZE_MIN = @@ -9399,14 +9331,6 @@ public static final class Name { public static final String USER_RPC_RETRY_MAX_SLEEP_MS = "alluxio.user.rpc.retry.max.sleep"; public static final String USER_UFS_BLOCK_LOCATION_ALL_FALLBACK_ENABLED = "alluxio.user.ufs.block.location.all.fallback.enabled"; - public static final String USER_UFS_BLOCK_READ_LOCATION_POLICY = - "alluxio.user.ufs.block.read.location.policy"; - public static final String USER_UFS_BLOCK_READ_LOCATION_POLICY_DETERMINISTIC_HASH_SHARDS = - "alluxio.user.ufs.block.read.location.policy.deterministic.hash.shards"; - public static final String USER_UFS_BLOCK_READ_LOCATION_POLICY_CACHE_SIZE = - "alluxio.user.ufs.block.read.location.policy.cache.size"; - public static final String USER_UFS_BLOCK_READ_LOCATION_POLICY_CACHE_EXPIRATION_TIME = - "alluxio.user.ufs.block.read.location.policy.cache.expiration.time"; public static final String USER_UFS_BLOCK_READ_CONCURRENCY_MAX = "alluxio.user.ufs.block.read.concurrency.max"; public static final String USER_UNSAFE_DIRECT_LOCAL_IO_ENABLED = diff --git a/dora/core/common/src/main/java/alluxio/conf/RemovedKey.java b/dora/core/common/src/main/java/alluxio/conf/RemovedKey.java index aae109923a3d..f21df51742d5 100644 --- a/dora/core/common/src/main/java/alluxio/conf/RemovedKey.java +++ b/dora/core/common/src/main/java/alluxio/conf/RemovedKey.java @@ -100,14 +100,10 @@ private static String replacedSince(String version, String newProperty) { put("alluxio.user.block.worker.client.threads", removedSince(V2_0_0)); put("alluxio.user.failed.space.request.limits", removedSince(V2_0_0)); put("alluxio.user.file.cache.partially.read.block", removedSince(V2_0_0)); - put("alluxio.user.file.copyfromlocal.write.location.policy.class", replacedSince(V2_0_0, - PropertyKey.USER_FILE_COPYFROMLOCAL_BLOCK_LOCATION_POLICY.getName())); put("alluxio.user.file.seek.buffer.size.bytes", removedSince(V2_0_0)); put("alluxio.user.file.write.avoid.eviction.policy.reserved.size.bytes", replacedSince(V2_0_0, PropertyKey.USER_BLOCK_AVOID_EVICTION_POLICY_RESERVED_BYTES.getName())); - put("alluxio.user.file.write.location.policy.class", - replacedSince(V2_0_0, PropertyKey.USER_BLOCK_WRITE_LOCATION_POLICY.getName())); put("alluxio.user.heartbeat.interval", removedSince(V2_0_0)); put("alluxio.user.lineage.enabled", removedSince(V2_0_0)); put("alluxio.user.lineage.master.client.threads", removedSince(V2_0_0)); diff --git a/dora/shell/src/main/java/alluxio/cli/fs/command/CopyFromLocalCommand.java b/dora/shell/src/main/java/alluxio/cli/fs/command/CopyFromLocalCommand.java index 1873133442f7..26841ef0fa6b 100644 --- a/dora/shell/src/main/java/alluxio/cli/fs/command/CopyFromLocalCommand.java +++ b/dora/shell/src/main/java/alluxio/cli/fs/command/CopyFromLocalCommand.java @@ -14,8 +14,6 @@ import alluxio.annotation.PublicApi; import alluxio.client.file.FileSystem; import alluxio.client.file.FileSystemContext; -import alluxio.conf.InstancedConfiguration; -import alluxio.conf.PropertyKey; import alluxio.exception.AlluxioException; import alluxio.exception.status.InvalidArgumentException; @@ -44,22 +42,9 @@ public final class CopyFromLocalCommand extends AbstractFileSystemCommand { */ public CopyFromLocalCommand(FileSystemContext fsContext) { super(fsContext); - // The copyFromLocal command needs its own filesystem context because we overwrite the - // block location policy configuration. - // The original one can't be closed because it may still be in-use within the same shell. - InstancedConfiguration conf = new InstancedConfiguration( - fsContext.getClusterConf().copyProperties()); - conf.set(PropertyKey.USER_BLOCK_WRITE_LOCATION_POLICY, - conf.get(PropertyKey.USER_FILE_COPYFROMLOCAL_BLOCK_LOCATION_POLICY)); - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("copyFromLocal block write location policy is %s from property %s", - conf.get(PropertyKey.USER_BLOCK_WRITE_LOCATION_POLICY), - PropertyKey.USER_FILE_COPYFROMLOCAL_BLOCK_LOCATION_POLICY.getName())); - } - FileSystemContext updatedCtx = FileSystemContext.sFileSystemContextFactory.create(conf); - mFsContext = updatedCtx; - mFileSystem = FileSystem.Factory.create(updatedCtx); - mCpCommand = new CpCommand(updatedCtx); + mFsContext = fsContext; + mFileSystem = FileSystem.Factory.create(fsContext); + mCpCommand = new CpCommand(fsContext); } @Override diff --git a/dora/shell/src/main/java/alluxio/cli/fs/command/LoadCommand.java b/dora/shell/src/main/java/alluxio/cli/fs/command/LoadCommand.java index afa3965e3698..123c10d94286 100644 --- a/dora/shell/src/main/java/alluxio/cli/fs/command/LoadCommand.java +++ b/dora/shell/src/main/java/alluxio/cli/fs/command/LoadCommand.java @@ -14,33 +14,16 @@ import alluxio.AlluxioURI; import alluxio.annotation.PublicApi; import alluxio.cli.CommandUtils; -import alluxio.client.block.BlockStoreClient; -import alluxio.client.block.policy.BlockLocationPolicy; -import alluxio.client.block.stream.BlockInStream; -import alluxio.client.block.stream.BlockWorkerClient; import alluxio.client.file.FileSystemContext; -import alluxio.client.file.URIStatus; -import alluxio.client.file.options.InStreamOptions; -import alluxio.collections.Pair; -import alluxio.conf.AlluxioConfiguration; import alluxio.conf.PropertyKey; import alluxio.exception.AlluxioException; import alluxio.exception.status.InvalidArgumentException; -import alluxio.grpc.CacheRequest; import alluxio.grpc.JobProgressReportFormat; import alluxio.grpc.LoadJobPOptions; -import alluxio.grpc.OpenFilePOptions; import alluxio.job.JobDescription; import alluxio.job.LoadJobRequest; -import alluxio.proto.dataserver.Protocol; -import alluxio.resource.CloseableResource; -import alluxio.util.FileSystemOptionsUtils; import alluxio.util.FormatUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; import io.grpc.Status; import io.grpc.StatusRuntimeException; import org.apache.commons.cli.CommandLine; @@ -48,7 +31,6 @@ import org.apache.commons.cli.Options; import java.io.IOException; -import java.util.List; import java.util.Optional; import java.util.OptionalLong; import javax.annotation.concurrent.ThreadSafe; @@ -173,14 +155,11 @@ public int run(CommandLine cl) throws AlluxioException, IOException { + "Please only use it when the cluster has " + PropertyKey.DORA_ENABLED + "=false"); String[] args = cl.getArgs(); AlluxioURI path = new AlluxioURI(args[0]); - if (isOldFormat(cl)) { - runWildCardCmd(path, cl); - return 0; - } if (path.containsWildcard()) { throw new UnsupportedOperationException("Load does not support wildcard path"); } + throwIfOldFormat(cl); if (cl.hasOption(SUBMIT_OPTION.getLongOpt())) { OptionalLong bandwidth = OptionalLong.empty(); @@ -228,20 +207,19 @@ public String getDescription() { @Override public void validateArgs(CommandLine cl) throws InvalidArgumentException { CommandUtils.checkNumOfArgsNoLessThan(this, cl, 1); - if (!isOldFormat(cl)) { - int commands = 0; - if (cl.hasOption(SUBMIT_OPTION.getLongOpt())) { - commands++; - } - if (cl.hasOption(STOP_OPTION.getLongOpt())) { - commands++; - } - if (cl.hasOption(PROGRESS_OPTION.getLongOpt())) { - commands++; - } - if (commands != 1) { - throw new InvalidArgumentException("Must have one of submit / stop / progress"); - } + throwIfOldFormat(cl); + int commands = 0; + if (cl.hasOption(SUBMIT_OPTION.getLongOpt())) { + commands++; + } + if (cl.hasOption(STOP_OPTION.getLongOpt())) { + commands++; + } + if (cl.hasOption(PROGRESS_OPTION.getLongOpt())) { + commands++; + } + if (commands != 1) { + throw new InvalidArgumentException("Must have one of submit / stop / progress"); } } @@ -308,91 +286,17 @@ private int getProgress(AlluxioURI path, JobProgressReportFormat format, } } - private boolean isOldFormat(CommandLine cl) { - return cl.getOptions().length == 0 - || (cl.getOptions().length == 1 && cl.hasOption(LOCAL_OPTION.getLongOpt())); + private void throwIfOldFormat(CommandLine cl) { + if (cl.getOptions().length == 0 + || (cl.getOptions().length == 1 && cl.hasOption(LOCAL_OPTION.getLongOpt()))) { + throw new IllegalArgumentException("load command no longer supports the old format"); + } } @Override protected void runPlainPath(AlluxioURI plainPath, CommandLine cl) throws AlluxioException, IOException { - Preconditions.checkState( - isOldFormat(cl), - "The new load command should not hit this code path"); - oldLoad(plainPath, cl.hasOption(LOCAL_OPTION.getLongOpt())); - } - - /** - * Loads a file or directory in Alluxio space, makes it resident in Alluxio. - * - * @param filePath The {@link AlluxioURI} path to load into Alluxio - * @param local whether to load data to local worker even when the data is already loaded remotely - */ - private void oldLoad(AlluxioURI filePath, boolean local) - throws AlluxioException, IOException { - URIStatus status = mFileSystem.getStatus(filePath); - if (status.isFolder()) { - List statuses = mFileSystem.listStatus(filePath); - for (URIStatus uriStatus : statuses) { - AlluxioURI newPath = new AlluxioURI(uriStatus.getPath()); - oldLoad(newPath, local); - } - } else { - if (local) { - if (!mFsContext.hasNodeLocalWorker()) { - System.out.println( - "When local option is specified, there must be a local worker available"); - return; - } - } else if (status.getInAlluxioPercentage() == 100) { - // The file has already been fully loaded into Alluxio. - System.out.println(filePath + " already in Alluxio fully"); - return; - } - runLoadTask(filePath, status, local); - } - System.out.println(filePath + " loaded"); - } - - private void runLoadTask(AlluxioURI filePath, URIStatus status, boolean local) - throws IOException { - AlluxioConfiguration conf = mFsContext.getPathConf(filePath); - OpenFilePOptions options = FileSystemOptionsUtils.openFileDefaults(conf); - BlockLocationPolicy policy = Preconditions.checkNotNull( - BlockLocationPolicy.Factory - .create(conf.getClass(PropertyKey.USER_UFS_BLOCK_READ_LOCATION_POLICY), conf), - "UFS read location policy Required when loading files"); - WorkerNetAddress dataSource; - List blockIds = status.getBlockIds(); - for (long blockId : blockIds) { - if (local) { - dataSource = mFsContext.getNodeLocalWorker(); - } else { // send request to data source - BlockStoreClient blockStore = BlockStoreClient.create(mFsContext); - Pair dataSourceAndType = blockStore - .getDataSourceAndType(status.getBlockInfo(blockId), status, policy, ImmutableMap.of()); - dataSource = dataSourceAndType.getFirst(); - } - Protocol.OpenUfsBlockOptions openUfsBlockOptions = - new InStreamOptions(status, options, conf, mFsContext).getOpenUfsBlockOptions(blockId); - BlockInfo info = status.getBlockInfo(blockId); - long blockLength = info.getLength(); - String host = dataSource.getHost(); - // issues#11172: If the worker is in a container, use the container hostname - // to establish the connection. - if (!dataSource.getContainerHost().equals("")) { - host = dataSource.getContainerHost(); - } - CacheRequest request = CacheRequest.newBuilder().setBlockId(blockId).setLength(blockLength) - .setOpenUfsBlockOptions(openUfsBlockOptions).setSourceHost(host) - .setSourcePort(dataSource.getDataPort()).build(); - try (CloseableResource blockWorker = - mFsContext.acquireBlockWorkerClient(dataSource)) { - blockWorker.get().cache(request); - } catch (Exception e) { - throw new RuntimeException(String.format("Failed to complete cache request from %s for " - + "block %d of file %s: %s", dataSource, blockId, status.getPath(), e), e); - } - } + // TODO(jiacheng): refactor LoadCommand so the main logic is executed here + throw new IllegalStateException("Should not reach here!"); } } diff --git a/dora/stress/shell/src/main/java/alluxio/stress/cli/client/ClientIOWritePolicy.java b/dora/stress/shell/src/main/java/alluxio/stress/cli/client/ClientIOWritePolicy.java deleted file mode 100644 index 14a7cab7208a..000000000000 --- a/dora/stress/shell/src/main/java/alluxio/stress/cli/client/ClientIOWritePolicy.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.stress.cli.client; - -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.BlockLocationPolicy; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.base.MoreObjects; -import com.google.common.base.Objects; -import com.google.common.collect.Lists; - -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicInteger; -import javax.annotation.concurrent.NotThreadSafe; - -/** - * Write type for restricting the writes to a set number of workers. This should only be used for - * testing and benchmarking. - */ -@NotThreadSafe -public final class ClientIOWritePolicy implements BlockLocationPolicy { - private static final AtomicInteger MAX_WORKERS = new AtomicInteger(1); - - private List mWorkerInfoList; - private int mIndex; - private volatile boolean mInitialized = false; - - /** - * Updates the global state to reflect the max number of workers to write to. This must be set - * before writes are performed to take effect. - * - * @param maxWorkers the max number of workers to write to - */ - public static void setMaxWorkers(int maxWorkers) { - MAX_WORKERS.set(maxWorkers); - } - - /** - * Constructs a new {@link ClientIOWritePolicy}. - * needed for instantiation in {@link BlockLocationPolicy.Factory}. - * - * @param ignoredConf is unused - */ - public ClientIOWritePolicy(AlluxioConfiguration ignoredConf) {} - - /** - * - * @param options options - * @return the address of the worker to write to - */ - @Override - public synchronized Optional getWorker(GetWorkerOptions options) { - Map eligibleWorkers = new HashMap<>(); - for (BlockWorkerInfo info : options.getBlockWorkerInfos()) { - eligibleWorkers.put(info.getNetAddress(), info); - } - - if (!mInitialized) { - mWorkerInfoList = Lists.newArrayList(options.getBlockWorkerInfos()); - // sort by hashcode - mWorkerInfoList.sort(Comparator.comparing(w -> w.getNetAddress().getHost())); - // take the first subset - mWorkerInfoList = - mWorkerInfoList.subList(0, Math.min(MAX_WORKERS.get(), mWorkerInfoList.size())); - if (mWorkerInfoList.size() < MAX_WORKERS.get()) { - throw new IllegalStateException(String - .format("Not enough eligible workers. expected: %d actual: %d", MAX_WORKERS.get(), - mWorkerInfoList.size())); - } - mIndex = 0; - mInitialized = true; - } - - for (int i = 0; i < mWorkerInfoList.size(); i++) { - WorkerNetAddress candidate = mWorkerInfoList.get(mIndex).getNetAddress(); - mIndex = (mIndex + 1) % mWorkerInfoList.size(); - - BlockWorkerInfo workerInfo = eligibleWorkers.get(candidate); - if (workerInfo != null && workerInfo.getCapacityBytes() >= options.getBlockInfo() - .getLength()) { - return Optional.of(candidate); - } - } - - return Optional.empty(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof ClientIOWritePolicy)) { - return false; - } - ClientIOWritePolicy that = (ClientIOWritePolicy) o; - return Objects.equal(mWorkerInfoList, that.mWorkerInfoList) - && Objects.equal(mIndex, that.mIndex) - && Objects.equal(mInitialized, that.mInitialized); - } - - @Override - public int hashCode() { - return Objects.hashCode(mWorkerInfoList, mIndex, mInitialized); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("workerInfoList", mWorkerInfoList) - .add("index", mIndex) - .add("initialized", mInitialized) - .toString(); - } -} diff --git a/dora/stress/shell/src/main/java/alluxio/stress/cli/client/StressClientIOBench.java b/dora/stress/shell/src/main/java/alluxio/stress/cli/client/StressClientIOBench.java index 0e076543131a..3c2238bab7fd 100644 --- a/dora/stress/shell/src/main/java/alluxio/stress/cli/client/StressClientIOBench.java +++ b/dora/stress/shell/src/main/java/alluxio/stress/cli/client/StressClientIOBench.java @@ -141,16 +141,12 @@ public void prepare() throws Exception { } } - ClientIOWritePolicy.setMaxWorkers(mParameters.mWriteNumWorkers); - // set hdfs conf for all test clients Configuration hdfsConf = new Configuration(); // do not cache these clients hdfsConf.set( String.format("fs.%s.impl.disable.cache", (new URI(mParameters.mBasePath)).getScheme()), "true"); - hdfsConf.set(PropertyKey.Name.USER_BLOCK_WRITE_LOCATION_POLICY, - ClientIOWritePolicy.class.getName()); for (Map.Entry entry : mParameters.mConf.entrySet()) { hdfsConf.set(entry.getKey(), entry.getValue()); } diff --git a/dora/stress/shell/src/main/java/alluxio/stress/cli/worker/StressWorkerBench.java b/dora/stress/shell/src/main/java/alluxio/stress/cli/worker/StressWorkerBench.java index 9a4d34e00ff8..a90df31f4474 100644 --- a/dora/stress/shell/src/main/java/alluxio/stress/cli/worker/StressWorkerBench.java +++ b/dora/stress/shell/src/main/java/alluxio/stress/cli/worker/StressWorkerBench.java @@ -18,7 +18,6 @@ import alluxio.grpc.WritePType; import alluxio.stress.BaseParameters; import alluxio.stress.cli.AbstractStressBench; -import alluxio.stress.cli.client.ClientIOWritePolicy; import alluxio.stress.common.FileSystemParameters; import alluxio.stress.worker.WorkerBenchParameters; import alluxio.stress.worker.WorkerBenchTaskResult; @@ -98,9 +97,6 @@ public String getBenchDescription() { @SuppressFBWarnings("BC_UNCONFIRMED_CAST") public void prepare() throws Exception { - // Read and write to one worker - ClientIOWritePolicy.setMaxWorkers(1); - // initialize the base, for only the non-distributed task (the cluster launching task) Path path = new Path(mParameters.mBasePath); int fileSize = (int) FormatUtils.parseSpaceSize(mParameters.mFileSize); @@ -139,10 +135,6 @@ public void prepare() throws Exception { FileSystemParameters.WRITE_TYPE_OPTION_NAME, WritePType.MUST_CACHE, "--free")); } hdfsConf.set(PropertyKey.Name.USER_FILE_WRITE_TYPE_DEFAULT, mParameters.mWriteType); - hdfsConf.set(PropertyKey.Name.USER_BLOCK_WRITE_LOCATION_POLICY, - ClientIOWritePolicy.class.getName()); - hdfsConf.set(PropertyKey.Name.USER_UFS_BLOCK_READ_LOCATION_POLICY, - ClientIOWritePolicy.class.getName()); FileSystem prepareFs = FileSystem.get(new URI(mParameters.mBasePath), hdfsConf); if (!mParameters.mSkipCreation) { @@ -179,10 +171,6 @@ public void prepare() throws Exception { hdfsConf.set( String.format("fs.%s.impl.disable.cache", (new URI(mParameters.mBasePath)).getScheme()), "true"); - hdfsConf.set(PropertyKey.Name.USER_BLOCK_WRITE_LOCATION_POLICY, - ClientIOWritePolicy.class.getName()); - hdfsConf.set(PropertyKey.Name.USER_UFS_BLOCK_READ_LOCATION_POLICY, - ClientIOWritePolicy.class.getName()); for (Map.Entry entry : mParameters.mConf.entrySet()) { hdfsConf.set(entry.getKey(), entry.getValue()); } diff --git a/dora/tests/src/test/java/alluxio/client/fs/RemoteReadIntegrationTest.java b/dora/tests/src/test/java/alluxio/client/fs/RemoteReadIntegrationTest.java index 3f47354f9913..7699d54f567c 100644 --- a/dora/tests/src/test/java/alluxio/client/fs/RemoteReadIntegrationTest.java +++ b/dora/tests/src/test/java/alluxio/client/fs/RemoteReadIntegrationTest.java @@ -13,21 +13,15 @@ import alluxio.AlluxioURI; import alluxio.annotation.dora.DoraTestTodoItem; -import alluxio.client.block.BlockStoreClient; -import alluxio.client.block.stream.BlockInStream; -import alluxio.client.block.stream.BlockInStream.BlockInStreamSource; import alluxio.client.file.FileInStream; import alluxio.client.file.FileOutStream; import alluxio.client.file.FileSystem; import alluxio.client.file.FileSystemContext; import alluxio.client.file.FileSystemTestUtils; import alluxio.client.file.FileSystemUtils; -import alluxio.client.file.URIStatus; -import alluxio.client.file.options.InStreamOptions; import alluxio.conf.Configuration; import alluxio.conf.PropertyKey; import alluxio.exception.PreconditionMessage; -import alluxio.exception.status.NotFoundException; import alluxio.grpc.CreateFilePOptions; import alluxio.grpc.OpenFilePOptions; import alluxio.grpc.ReadPType; @@ -39,8 +33,6 @@ import alluxio.util.CommonUtils; import alluxio.util.io.BufferUtils; import alluxio.util.io.PathUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.WorkerNetAddress; import org.junit.After; import org.junit.Assert; @@ -50,8 +42,6 @@ import org.junit.Test; import org.junit.rules.ExpectedException; -import java.util.Arrays; - /** * Integration tests for reading from a remote worker. */ @@ -241,103 +231,6 @@ public void readTest3() throws Exception { } } - /** - * Tests the single byte read API from a remote location when the data is in an Alluxio worker. - */ - @Test - public void readTest4() throws Exception { - String uniqPath = PathUtils.uniqPath(); - for (int k = MIN_LEN + DELTA; k <= MAX_LEN; k += DELTA) { - AlluxioURI uri = new AlluxioURI(uniqPath + "/file_" + k); - FileSystemTestUtils.createByteFile(mFileSystem, uri, mWriteAlluxio, k); - - URIStatus status = mFileSystem.getStatus(uri); - InStreamOptions options = new InStreamOptions(status, Configuration.global()); - long blockId = status.getBlockIds().get(0); - BlockStoreClient blockStore = - BlockStoreClient.create(FileSystemContext.create(Configuration.global())); - BlockInfo info = blockStore.getInfo(blockId); - WorkerNetAddress workerAddr = info.getLocations().get(0).getWorkerAddress(); - BlockInStream is = - BlockInStream.create(mFsContext, options.getBlockInfo(blockId), - workerAddr, BlockInStreamSource.REMOTE, options); - byte[] ret = new byte[k]; - int value = is.read(); - int cnt = 0; - while (value != -1) { - Assert.assertTrue(value >= 0); - Assert.assertTrue(value < 256); - ret[cnt++] = (byte) value; - value = is.read(); - } - Assert.assertEquals(cnt, k); - Assert.assertTrue(BufferUtils.equalIncreasingByteArray(k, ret)); - is.close(); - FileSystemUtils.waitForAlluxioPercentage(mFileSystem, uri, 100); - } - } - - /** - * Tests the batch read API from a remote location when the data is only in an Alluxio worker. - */ - @Test - public void readTest5() throws Exception { - String uniqPath = PathUtils.uniqPath(); - for (int k = MIN_LEN + DELTA; k <= MAX_LEN; k += DELTA) { - AlluxioURI uri = new AlluxioURI(uniqPath + "/file_" + k); - FileSystemTestUtils.createByteFile(mFileSystem, uri, mWriteAlluxio, k); - - URIStatus status = mFileSystem.getStatus(uri); - InStreamOptions options = new InStreamOptions(status, Configuration.global()); - long blockId = status.getBlockIds().get(0); - BlockInfo info = - BlockStoreClient.create(FileSystemContext.create(Configuration.global())) - .getInfo(blockId); - WorkerNetAddress workerAddr = info.getLocations().get(0).getWorkerAddress(); - BlockInStream is = - BlockInStream.create(mFsContext, options.getBlockInfo(blockId), - workerAddr, BlockInStreamSource.REMOTE, options); - byte[] ret = new byte[k]; - int read = is.read(ret); - Assert - .assertTrue(BufferUtils.equalIncreasingByteArray(read, Arrays.copyOfRange(ret, 0, read))); - is.close(); - FileSystemUtils.waitForAlluxioPercentage(mFileSystem, uri, 100); - } - } - - /** - * Tests the batch read API with offset and length from a remote location when the data is in an - * Alluxio worker. - */ - @Test - public void readTest6() throws Exception { - String uniqPath = PathUtils.uniqPath(); - for (int k = MIN_LEN + DELTA; k <= MAX_LEN; k += DELTA) { - AlluxioURI uri = new AlluxioURI(uniqPath + "/file_" + k); - FileSystemTestUtils.createByteFile(mFileSystem, uri, mWriteAlluxio, k); - - URIStatus status = mFileSystem.getStatus(uri); - InStreamOptions options = new InStreamOptions(status, Configuration.global()); - long blockId = status.getBlockIds().get(0); - BlockInfo info = - BlockStoreClient.create(FileSystemContext - .create(Configuration.global())).getInfo(blockId); - WorkerNetAddress workerAddr = info.getLocations().get(0).getWorkerAddress(); - BlockInStream is = - BlockInStream.create(mFsContext, options.getBlockInfo(blockId), - workerAddr, BlockInStreamSource.REMOTE, options); - byte[] ret = new byte[k / 2]; - int read = 0; - while (read < k / 2) { - read += is.read(ret, read, k / 2 - read); - } - Assert.assertTrue(BufferUtils.equalIncreasingByteArray(read, ret)); - is.close(); - FileSystemUtils.waitForAlluxioPercentage(mFileSystem, uri, 100); - } - } - /** * Tests the batch read API from a remote location when the data is only in the underlying * storage. @@ -548,50 +441,4 @@ public void seekAroundLocalBlock() throws Exception { Assert.assertEquals(99, is.read()); is.close(); } - - /** - * Tests remote reads lock blocks correctly. - */ - @Test - public void remoteReadLock() throws Exception { - String uniqPath = PathUtils.uniqPath(); - for (int k = MIN_LEN + DELTA; k <= MAX_LEN; k += DELTA) { - AlluxioURI uri = new AlluxioURI(uniqPath + "/file_" + k); - FileSystemTestUtils.createByteFile(mFileSystem, uri, mWriteAlluxio, k); - - URIStatus status = mFileSystem.getStatus(uri); - InStreamOptions options = new InStreamOptions(status, Configuration.global()); - long blockId = status.getBlockIds().get(0); - BlockInfo info = BlockStoreClient - .create(FileSystemContext.create(Configuration.global())).getInfo(blockId); - - WorkerNetAddress workerAddr = info.getLocations().get(0).getWorkerAddress(); - BlockInStream is = - BlockInStream.create(mFsContext, options.getBlockInfo(blockId), - workerAddr, BlockInStreamSource.REMOTE, options); - Assert.assertEquals(0, is.read()); - mFileSystem.delete(uri); - - // The file has been deleted. - Assert.assertFalse(mFileSystem.exists(uri)); - // Look! We can still read the deleted file since we have a lock! - byte[] ret = new byte[k / 2]; - Assert.assertTrue(is.read(ret, 0, k / 2) > 0); - is.close(); - Assert.assertFalse(mFileSystem.exists(uri)); - - // Try to create an in stream again, and it should fail. - BlockInStream is2 = null; - try { - is2 = BlockInStream.create(mFsContext, options.getBlockInfo(blockId), - workerAddr, BlockInStreamSource.REMOTE, options); - } catch (NotFoundException e) { - // Expected since the file has been deleted. - } finally { - if (is2 != null) { - is2.close(); - } - } - } - } } diff --git a/dora/tests/src/test/java/alluxio/client/fs/io/BufferedBlockInStreamIntegrationTest.java b/dora/tests/src/test/java/alluxio/client/fs/io/BufferedBlockInStreamIntegrationTest.java index 4118cfc3e058..3041168eab0e 100644 --- a/dora/tests/src/test/java/alluxio/client/fs/io/BufferedBlockInStreamIntegrationTest.java +++ b/dora/tests/src/test/java/alluxio/client/fs/io/BufferedBlockInStreamIntegrationTest.java @@ -13,7 +13,6 @@ import alluxio.AlluxioURI; import alluxio.annotation.dora.DoraTestTodoItem; -import alluxio.client.block.stream.BlockInStream; import alluxio.client.file.FileInStream; import alluxio.client.file.FileSystem; import alluxio.client.file.FileSystemTestUtils; @@ -36,7 +35,7 @@ import java.util.List; /** - * Integration tests for {@link BlockInStream}. + * Integration tests for {@link FileInStream}. */ @Ignore @DoraTestTodoItem(action = DoraTestTodoItem.Action.FIX, owner = "bowen", diff --git a/dora/tests/src/test/java/alluxio/client/fs/io/FileOutStreamIntegrationTest.java b/dora/tests/src/test/java/alluxio/client/fs/io/FileOutStreamIntegrationTest.java index 560cd0ec2092..7f0985374140 100644 --- a/dora/tests/src/test/java/alluxio/client/fs/io/FileOutStreamIntegrationTest.java +++ b/dora/tests/src/test/java/alluxio/client/fs/io/FileOutStreamIntegrationTest.java @@ -157,31 +157,6 @@ public void writeTwoByteArrays() throws Exception { } } - /** - * Tests writing to a file and specify the location to be localhost. - */ - @Test - @LocalAlluxioClusterResource.Config(confParams = { - PropertyKey.Name.USER_BLOCK_WRITE_LOCATION_POLICY, - "alluxio.client.block.policy.LocalFirstPolicy" - }) - public void writeSpecifyLocal() throws Exception { - AlluxioURI filePath = new AlluxioURI(PathUtils.uniqPath()); - final int length = 2; - CreateFilePOptions op = CreateFilePOptions.newBuilder().setWriteType(mWriteType.toProto()) - .setRecursive(true).build(); - try (FileOutStream os = mFileSystem.createFile(filePath, op)) { - os.write((byte) 0); - os.write((byte) 1); - } - if (mWriteType.getAlluxioStorageType().isStore()) { - checkFileInAlluxio(filePath, length); - } - if (mWriteType.getUnderStorageType().isSyncPersist()) { - checkFileInUnderStorage(filePath, length); - } - } - /** * Tests writing to a file for longer than HEARTBEAT_INTERVAL_MS to make sure the sessionId * doesn't change. Tracks [ALLUXIO-171]. diff --git a/dora/tests/src/test/java/alluxio/server/ft/MultiWorkerIntegrationTest.java b/dora/tests/src/test/java/alluxio/server/ft/MultiWorkerIntegrationTest.java deleted file mode 100644 index 12466628d7ed..000000000000 --- a/dora/tests/src/test/java/alluxio/server/ft/MultiWorkerIntegrationTest.java +++ /dev/null @@ -1,227 +0,0 @@ -/* - * 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.server.ft; - -import static org.junit.Assert.assertEquals; - -import alluxio.AlluxioURI; -import alluxio.Constants; -import alluxio.annotation.dora.DoraTestTodoItem; -import alluxio.client.WriteType; -import alluxio.client.block.BlockStoreClient; -import alluxio.client.block.BlockWorkerInfo; -import alluxio.client.block.policy.BlockLocationPolicy; -import alluxio.client.block.policy.options.GetWorkerOptions; -import alluxio.client.file.FileInStream; -import alluxio.client.file.FileSystem; -import alluxio.client.file.FileSystemContext; -import alluxio.client.file.FileSystemTestUtils; -import alluxio.client.file.URIStatus; -import alluxio.client.file.options.InStreamOptions; -import alluxio.client.file.options.OutStreamOptions; -import alluxio.conf.AlluxioConfiguration; -import alluxio.conf.Configuration; -import alluxio.conf.PropertyKey; -import alluxio.grpc.CreateFilePOptions; -import alluxio.grpc.OpenFilePOptions; -import alluxio.grpc.WritePType; -import alluxio.testutils.BaseIntegrationTest; -import alluxio.testutils.LocalAlluxioClusterResource; -import alluxio.util.io.BufferUtils; -import alluxio.wire.BlockInfo; -import alluxio.wire.FileBlockInfo; -import alluxio.wire.WorkerNetAddress; - -import com.google.common.io.ByteStreams; -import org.apache.commons.io.IOUtils; -import org.junit.Assert; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; -import java.util.Optional; -import java.util.stream.StreamSupport; - -/** - * Tests a cluster containing multiple workers. - */ -@Ignore -@DoraTestTodoItem(action = DoraTestTodoItem.Action.FIX, owner = "bowen", - comment = "fix the tests") -public final class MultiWorkerIntegrationTest extends BaseIntegrationTest { - private static final int NUM_WORKERS = 4; - private static final int WORKER_MEMORY_SIZE_BYTES = Constants.MB; - private static final int BLOCK_SIZE_BYTES = WORKER_MEMORY_SIZE_BYTES / 2; - - public static class FindFirstBlockLocationPolicy implements BlockLocationPolicy { - // Set this prior to sending the create request to FSM. - private static WorkerNetAddress sWorkerAddress; - - public FindFirstBlockLocationPolicy(AlluxioConfiguration ignored) {} - - @Override - public Optional getWorker(GetWorkerOptions options) { - return StreamSupport.stream(options.getBlockWorkerInfos().spliterator(), false) - .filter(x -> x.getNetAddress().equals(sWorkerAddress)).findFirst() - .map(BlockWorkerInfo::getNetAddress); - } - } - - @Rule - public LocalAlluxioClusterResource mResource = - new LocalAlluxioClusterResource.Builder() - .setProperty(PropertyKey.WORKER_RAMDISK_SIZE, WORKER_MEMORY_SIZE_BYTES) - .setProperty(PropertyKey.USER_BLOCK_SIZE_BYTES_DEFAULT, BLOCK_SIZE_BYTES) - .setProperty(PropertyKey.USER_FILE_BUFFER_BYTES, BLOCK_SIZE_BYTES) - .setNumWorkers(NUM_WORKERS) - .build(); - - @Test - @LocalAlluxioClusterResource.Config(confParams = { - PropertyKey.Name.USER_BLOCK_WRITE_LOCATION_POLICY, - "alluxio.client.block.policy.RoundRobinPolicy", - }) - public void writeLargeFile() throws Exception { - int fileSize = NUM_WORKERS * WORKER_MEMORY_SIZE_BYTES; - AlluxioURI file = new AlluxioURI("/test"); - - FileSystem fs = mResource.get().getClient(); - FileSystemTestUtils.createByteFile(fs, file.getPath(), fileSize, - CreateFilePOptions.newBuilder().setWriteType(WritePType.MUST_CACHE).build()); - URIStatus status = fs.getStatus(file); - assertEquals(100, status.getInAlluxioPercentage()); - try (FileInStream inStream = fs.openFile(file)) { - assertEquals(fileSize, IOUtils.toByteArray(inStream).length); - } - } - - @Test - @LocalAlluxioClusterResource.Config(confParams = { - PropertyKey.Name.USER_BLOCK_SIZE_BYTES_DEFAULT, "16MB", - PropertyKey.Name.USER_STREAMING_READER_CHUNK_SIZE_BYTES, "64KB", - PropertyKey.Name.USER_BLOCK_READ_RETRY_MAX_DURATION, "1s", - PropertyKey.Name.WORKER_RAMDISK_SIZE, "1GB"}) - public void readRecoverFromLostWorker() throws Exception { - int offset = 17 * Constants.MB; - int length = 33 * Constants.MB; - int total = offset + length; - // creates a test file on one worker - AlluxioURI filePath = new AlluxioURI("/test"); - createFileOnWorker(total, filePath, mResource.get().getWorkerAddress()); - FileSystem fs = mResource.get().getClient(); - try (FileInStream in = fs.openFile(filePath, OpenFilePOptions.getDefaultInstance())) { - byte[] buf = new byte[total]; - int size = in.read(buf, 0, offset); - replicateFileBlocks(filePath); - mResource.get().getWorkerProcess().stop(); - size += in.read(buf, offset, length); - - Assert.assertEquals(total, size); - Assert.assertTrue(BufferUtils.equalIncreasingByteArray(offset, size, buf)); - } - } - - @Test - @LocalAlluxioClusterResource.Config(confParams = { - PropertyKey.Name.USER_BLOCK_SIZE_BYTES_DEFAULT, "4MB", - PropertyKey.Name.USER_STREAMING_READER_CHUNK_SIZE_BYTES, "64KB", - PropertyKey.Name.USER_BLOCK_READ_RETRY_MAX_DURATION, "1s", - PropertyKey.Name.WORKER_RAMDISK_SIZE, "1GB"}) - public void readOneRecoverFromLostWorker() throws Exception { - int offset = Constants.MB; - int length = 5 * Constants.MB; - int total = offset + length; - // creates a test file on one worker - AlluxioURI filePath = new AlluxioURI("/test"); - FileSystem fs = mResource.get().getClient(); - createFileOnWorker(total, filePath, mResource.get().getWorkerAddress()); - try (FileInStream in = fs.openFile(filePath, OpenFilePOptions.getDefaultInstance())) { - byte[] buf = new byte[total]; - int size = in.read(buf, 0, offset); - replicateFileBlocks(filePath); - mResource.get().getWorkerProcess().stop(); - for (int i = 0; i < length; i++) { - int result = in.read(); - Assert.assertEquals(result, (i + size) & 0xff); - } - } - } - - @Test - @LocalAlluxioClusterResource.Config(confParams = { - PropertyKey.Name.USER_BLOCK_SIZE_BYTES_DEFAULT, "4MB", - PropertyKey.Name.USER_STREAMING_READER_CHUNK_SIZE_BYTES, "64KB", - PropertyKey.Name.USER_BLOCK_READ_RETRY_MAX_DURATION, "1s", - PropertyKey.Name.WORKER_RAMDISK_SIZE, "1GB"}) - public void positionReadRecoverFromLostWorker() throws Exception { - int offset = Constants.MB; - int length = 7 * Constants.MB; - int total = offset + length; - // creates a test file on one worker - AlluxioURI filePath = new AlluxioURI("/test"); - FileSystem fs = mResource.get().getClient(); - createFileOnWorker(total, filePath, mResource.get().getWorkerAddress()); - try (FileInStream in = fs.openFile(filePath, OpenFilePOptions.getDefaultInstance())) { - byte[] buf = new byte[length]; - replicateFileBlocks(filePath); - mResource.get().getWorkerProcess().stop(); - int size = in.positionedRead(offset, buf, 0, length); - - Assert.assertEquals(length, size); - Assert.assertTrue(BufferUtils.equalIncreasingByteArray(offset, size, buf)); - } - } - - private void createFileOnWorker(int total, AlluxioURI filePath, WorkerNetAddress address) - throws IOException { - FindFirstBlockLocationPolicy.sWorkerAddress = address; - Class previousPolicy = Configuration.getClass( - PropertyKey.USER_BLOCK_WRITE_LOCATION_POLICY); - // This only works because the client instance hasn't been created yet. - Configuration.set(PropertyKey.USER_BLOCK_WRITE_LOCATION_POLICY, - FindFirstBlockLocationPolicy.class.getName()); - FileSystemTestUtils.createByteFile(mResource.get().getClient(), filePath, - CreateFilePOptions.newBuilder().setWriteType(WritePType.MUST_CACHE).build(), - total); - Configuration.set(PropertyKey.USER_BLOCK_WRITE_LOCATION_POLICY, previousPolicy); - } - - private void replicateFileBlocks(AlluxioURI filePath) throws Exception { - FileSystemContext fsContext = FileSystemContext.create(Configuration.global()); - BlockStoreClient store = BlockStoreClient.create(fsContext); - URIStatus status = mResource.get().getClient().getStatus(filePath); - List blocks = status.getFileBlockInfos(); - List workers = fsContext.getCachedWorkers(); - - for (FileBlockInfo block : blocks) { - BlockInfo blockInfo = block.getBlockInfo(); - WorkerNetAddress src = blockInfo.getLocations().get(0).getWorkerAddress(); - WorkerNetAddress dest = workers.stream() - .filter(candidate -> !candidate.getNetAddress().equals(src)) - .findFirst() - .orElseThrow(() -> new IllegalStateException("Expected worker")) - .getNetAddress(); - try (OutputStream outStream = store.getOutStream(blockInfo.getBlockId(), - blockInfo.getLength(), dest, OutStreamOptions.defaults(fsContext) - .setBlockSizeBytes(8 * Constants.MB).setWriteType(WriteType.MUST_CACHE))) { - try (InputStream inStream = store.getInStream(blockInfo.getBlockId(), - new InStreamOptions(status, Configuration.global()))) { - ByteStreams.copy(inStream, outStream); - } - } - } - } -} From bfcc73904a35ebbce48c4873686f9d9c84ed9d30 Mon Sep 17 00:00:00 2001 From: Huang Hua Date: Mon, 31 Jul 2023 18:13:17 +0800 Subject: [PATCH 02/12] Set IsFromUFS flag in URIStatus to reduce client fall back time With this flag, client read will fall back to UFS when previous getStatus() falls back to UFS. Signed-off-by: Huang Hua --- .../client/file/DoraCacheFileSystem.java | 7 +++-- .../java/alluxio/client/file/URIStatus.java | 28 +++++++++++++++++++ 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/dora/core/client/fs/src/main/java/alluxio/client/file/DoraCacheFileSystem.java b/dora/core/client/fs/src/main/java/alluxio/client/file/DoraCacheFileSystem.java index c726280afb35..cbca547a0679 100644 --- a/dora/core/client/fs/src/main/java/alluxio/client/file/DoraCacheFileSystem.java +++ b/dora/core/client/fs/src/main/java/alluxio/client/file/DoraCacheFileSystem.java @@ -131,7 +131,7 @@ protected DoraCacheFileSystem(FileSystem fs, FileSystemContext context, public URIStatus getStatus(AlluxioURI path, GetStatusPOptions options) throws IOException, AlluxioException { AlluxioURI ufsFullPath = convertAlluxioPathToUFSPath(path); - + LOG.debug("DoraCacheFileSystem getStatus for " + ufsFullPath); if (!mMetadataCacheEnabled) { return mDelegatedFileSystem.getStatus(ufsFullPath, options); } @@ -152,7 +152,7 @@ public URIStatus getStatus(AlluxioURI path, GetStatusPOptions options) UFS_FALLBACK_COUNTER.inc(); LOG.debug("Dora client get status error ({} times). Fall back to UFS.", UFS_FALLBACK_COUNTER.getCount(), ex); - return mDelegatedFileSystem.getStatus(ufsFullPath, options); + return mDelegatedFileSystem.getStatus(ufsFullPath, options).setIsFromUFS(); } } @@ -176,6 +176,9 @@ public FileInStream openFile(URIStatus status, OpenFilePOptions options) OpenFilePOptions mergedOptions = FileSystemOptionsUtils.openFileDefaults(conf) .toBuilder().mergeFrom(options).build(); try { + if (status.getIsFromUFS()) { + throw new RuntimeException("Status is retrieved from UFS by falling back"); + } Protocol.OpenUfsBlockOptions openUfsBlockOptions = Protocol.OpenUfsBlockOptions.newBuilder().setUfsPath(status.getUfsPath()) .setOffsetInFile(0).setBlockSize(status.getLength()) diff --git a/dora/core/common/src/main/java/alluxio/client/file/URIStatus.java b/dora/core/common/src/main/java/alluxio/client/file/URIStatus.java index f7bc3528c8d9..f4a927396d78 100644 --- a/dora/core/common/src/main/java/alluxio/client/file/URIStatus.java +++ b/dora/core/common/src/main/java/alluxio/client/file/URIStatus.java @@ -43,6 +43,8 @@ public class URIStatus { /** Context associated with this URI, possibly set by other external engines (e.g., presto). */ private final CacheContext mCacheContext; + private boolean mIsFromUFS; + /** * Constructs an instance of this class from a {@link FileInfo}. * @@ -61,6 +63,32 @@ public URIStatus(FileInfo info) { public URIStatus(FileInfo info, @Nullable CacheContext context) { mInfo = Preconditions.checkNotNull(info, "info"); mCacheContext = context; + mIsFromUFS = false; + } + + /** + * @return if the FileInfo is from UFS instead of worker + * + * If true, it means this status is retrieved by falling back to UFS. + */ + public boolean getIsFromUFS() { + return mIsFromUFS; + } + + /** + * Sets flag that the FileInfo is retrieved from UFS instead of worker. + * + * Usually, client calls getStatus() and the FileInfo is retrieved from worker. + * But if there is something wrong with the worker or the connection between client + * and worker is lost, client will fall back to UFS. If that happens, this function + * is called to set the flag. So, the subsequent openFile(status, path) will also fall back + * to UFS immediately. + * + * @return this object itself + */ + public URIStatus setIsFromUFS() { + mIsFromUFS = true; + return this; } /** From 043860b67d8b3eeac74c00c32216f196372ac75d Mon Sep 17 00:00:00 2001 From: Jason Tieu <6509369+tieujason330@users.noreply.github.com> Date: Mon, 31 Jul 2023 14:18:46 -0700 Subject: [PATCH 03/12] Disable http keep-alive to address maven deps 'Connection Reset' errors ### What changes are proposed in this pull request? Disable keep-alive for HTTP requests to deal with connection resets when fetching maven deps from central repo. Unable to repro ([ref](https://stackoverflow.com/questions/55899091/maven-retry-dependency-download-if-failed)) ### Why are the changes needed? Trying to address tarball build failures via CI due to maven errors. ### Does this PR introduce any user facing changes? no pr-link: Alluxio/alluxio#17864 change-id: cid-767a7869d93ca2dbaee4e036d2bd7779ab7618e9 --- dev/scripts/src/alluxio.org/build/cmd/build.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/dev/scripts/src/alluxio.org/build/cmd/build.go b/dev/scripts/src/alluxio.org/build/cmd/build.go index ad2566f2e4b7..423481323c61 100644 --- a/dev/scripts/src/alluxio.org/build/cmd/build.go +++ b/dev/scripts/src/alluxio.org/build/cmd/build.go @@ -199,14 +199,15 @@ func buildTarball(opts *buildOpts) error { func constructMavenCmd(mvnArgs []string) string { cmd := []string{ "mvn", - "-am", // "also make": build dependent projects if a project list via `-pl` is specified - "clean", // remove previously generated files - "install", // maven build - "-DskipTests", // skip unit tests - "-Dfindbugs.skip", // skip findbugs static analysis check - "-Dmaven.javadoc.skip", // skip javadoc generation - "-Dcheckstyle.skip", // skip checkstyle static check - "-Prelease", // release profile specified in root pom.xml, to build dependency-reduced-pom.xml generated by shading plugin + "-am", // "also make": build dependent projects if a project list via `-pl` is specified + "clean", // remove previously generated files + "install", // maven build + "-DskipTests", // skip unit tests + "-Dfindbugs.skip", // skip findbugs static analysis check + "-Dmaven.javadoc.skip", // skip javadoc generation + "-Dcheckstyle.skip", // skip checkstyle static check + "-Prelease", // release profile specified in root pom.xml, to build dependency-reduced-pom.xml generated by shading plugin + "-Dhttp.keepAlive=false", // disable keep-alive for HTTP requests to deal with connection resets when fetching maven dependencies } if len(mvnArgs) > 0 { cmd = append(cmd, mvnArgs...) From e19246767d3853f1b09919905cbd2fc795adc257 Mon Sep 17 00:00:00 2001 From: Beinan Date: Mon, 31 Jul 2023 15:12:27 -0700 Subject: [PATCH 04/12] Add metrics for exteranl storage call from alluxio worker ### What changes are proposed in this pull request? Please outline the changes and how this PR fixes the issue. ### Why are the changes needed? Please clarify why the changes are needed. For instance, 1. If you propose a new API, clarify the use case for a new API. 2. If you fix a bug, describe the bug. ### Does this PR introduce any user facing changes? Please list the user-facing changes introduced by your change, including 1. change in user-facing APIs 2. addition or removal of property keys 3. webui pr-link: Alluxio/alluxio#17866 change-id: cid-b6c2036ad76c65854135d660a2de1ca486455298 --- .../main/java/alluxio/client/file/cache/LocalCacheManager.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dora/core/client/fs/src/main/java/alluxio/client/file/cache/LocalCacheManager.java b/dora/core/client/fs/src/main/java/alluxio/client/file/cache/LocalCacheManager.java index 9fe69da073af..1905d8173db7 100644 --- a/dora/core/client/fs/src/main/java/alluxio/client/file/cache/LocalCacheManager.java +++ b/dora/core/client/fs/src/main/java/alluxio/client/file/cache/LocalCacheManager.java @@ -637,6 +637,7 @@ public int getAndLoad(PageId pageId, int pageOffset, int bytesToRead, ReadTarget int bytesRead = get(pageId, pageOffset, bytesToRead, buffer, cacheContext); if (bytesRead > 0) { + MetricsSystem.counter(MetricKey.CLIENT_CACHE_HIT_REQUESTS.getName()).inc(); return bytesRead; } // on local cache miss, read a complete page from external storage. This will always make @@ -653,6 +654,7 @@ public int getAndLoad(PageId pageId, int pageOffset, int bytesToRead, ReadTarget buffer.writeBytes(page, pageOffset, bytesToRead); MetricsSystem.meter(MetricKey.CLIENT_CACHE_BYTES_REQUESTED_EXTERNAL.getName()) .mark(bytesToRead); + MetricsSystem.counter(MetricKey.CLIENT_CACHE_EXTERNAL_REQUESTS.getName()).inc(); cacheContext.incrementCounter( MetricKey.CLIENT_CACHE_BYTES_REQUESTED_EXTERNAL.getMetricName(), BYTE, bytesToRead); From 7b72ace012d062332f2f6f776bb51910a174e189 Mon Sep 17 00:00:00 2001 From: Beinan Date: Mon, 31 Jul 2023 15:12:43 -0700 Subject: [PATCH 05/12] Add metrics for exteranl storage call from alluxio worker ### What changes are proposed in this pull request? Please outline the changes and how this PR fixes the issue. ### Why are the changes needed? Please clarify why the changes are needed. For instance, 1. If you propose a new API, clarify the use case for a new API. 2. If you fix a bug, describe the bug. ### Does this PR introduce any user facing changes? Please list the user-facing changes introduced by your change, including 1. change in user-facing APIs 2. addition or removal of property keys 3. webui pr-link: Alluxio/alluxio#17866 change-id: cid-b6c2036ad76c65854135d660a2de1ca486455298 From d351160e2cf0b5c37967333fe6de7bfc93569ca0 Mon Sep 17 00:00:00 2001 From: jja725 Date: Mon, 31 Jul 2023 20:28:00 -0700 Subject: [PATCH 06/12] [DOCFIX] Fix load doc ### What changes are proposed in this pull request? Fix load cli doc and remove job service doc ### Why are the changes needed? doc quality ### Does this PR introduce any user facing changes? na pr-link: Alluxio/alluxio#17868 change-id: cid-628e4b5b68919245a7bcdbd3ae859c0bdc6a2e5f --- docs/en/operation/User-CLI.md | 271 +--------------------------------- 1 file changed, 3 insertions(+), 268 deletions(-) diff --git a/docs/en/operation/User-CLI.md b/docs/en/operation/User-CLI.md index 7e3ac1da3122..52e1d5bf6773 100644 --- a/docs/en/operation/User-CLI.md +++ b/docs/en/operation/User-CLI.md @@ -158,68 +158,6 @@ $ ./bin/alluxio getConf --unit S alluxio.master.journal.flush.timeout > Note: This command does not require the Alluxio cluster to be running. -### job - -The `job` command is a tool for interacting with the job service. - -The usage is `job [generic options]` -where `[generic options]` can be one of the following values: -* `leader`: Prints the hostname of the job master service leader. -* `ls`: Prints the IDs of the most recent jobs, running and finished, in the history up to the capacity set in `alluxio.job.master.job.capacity`. -* `stat [-v] `:Displays the status info for the specific job. Use -v flag to display the status of every task. -* `cancel `: Cancels the job with the corresponding id asynchronously. - -Print the hostname of the job master service leader: -```shell -$ ./bin/alluxio job leader -``` - -Print the IDs, job names, and completion status of the most recently created jobs: -```shell -$ ./bin/alluxio job ls - -1576539334518 Load COMPLETED -1576539334519 Load CREATED -1576539334520 Load CREATED -1576539334521 Load CREATED -1576539334522 Load CREATED -1576539334523 Load CREATED -1576539334524 Load CREATED -1576539334525 Load CREATED -1576539334526 Load CREATED -``` - -Display the status info for the specific job: -```shell -$ bin/alluxio job stat -v 1579102592778 - -ID: 1579102592778 -Name: Migrate -Description: MigrateConfig{source=/test, destination=/test2, writeType=ASYNC_THROUGH, overwrite=true, delet... -Status: CANCELED -Task 0 -Worker: localhost -Status: CANCELED -Task 1 -Worker: localhost -Status: CANCELED -Task 2 -Worker: localhost -Status: CANCELED - -... -``` - -Cancel the job asynchronously based on a specific job: -```shell -$ bin/alluxio job cancel 1579102592778 - -$ bin/alluxio job stat 1579102592778 | grep "Status" -Status: CANCELED -``` - -> Note: This command requires the Alluxio cluster to be running. - ### logLevel The `logLevel` command returns the current value of or updates the log level of a particular class @@ -534,175 +472,6 @@ $ ./bin/alluxio fs copyToLocal /output/part-00000 part-00000 $ wc -l part-00000 ``` -### distributedCp - -The `distributedCp` command copies a file or directory in the Alluxio file system distributed across workers -using the job service. By default, the command runs synchronously and the user will get a `JOB_CONTROL_ID` after the command successfully submits the job to be executed. -The command will wait until the job is complete, at which point the user will see the list of files copied and statistics on which files completed or failed. -The command can also run in async mode with the `--async` flag. Similar to before, the user will get a `JOB_CONTROL_ID` after the command successfully submits the job. -The difference is that the command will not wait for the job to finish. -Users can use the [`getCmdStatus`](#getCmdStatus) command with the `JOB_CONTROL_ID` as an argument to check detailed status information about the job. - -If the source designates a directory, `distributedCp` copies the entire subtree at source to the destination. - -**Options:** -* `--active-jobs`: Limits how many jobs can be submitted to the Alluxio job service at the same time. -Later jobs must wait until some earlier jobs to finish. The default value is `3000`. -A lower value means slower execution but also being nicer to the other users of the job service. -* `--overwrite`: Whether to overwrite the destination. Default is true. -* `--batch-size`: Specifies how many files to be batched into one request. The default value is `20`. Notice that if some task failed in the batched job, the whole batched job would fail with some completed tasks and some failed tasks. -* `--async`: Specifies whether to wait for command execution to finish. If not explicitly shown then default to run synchronously. - -```shell -$ ./bin/alluxio fs distributedCp --active-jobs 2000 /data/1023 /data/1024 - -Sample Output: -Please wait for command submission to finish.. -Submitted successfully, jobControlId = JOB_CONTROL_ID_1 -Waiting for the command to finish ... -Get command status information below: -Successfully copied path /data/1023/$FILE_PATH_1 -Successfully copied path /data/1023/$FILE_PATH_2 -Successfully copied path /data/1023/$FILE_PATH_3 -Total completed file count is 3, failed file count is 0 -Finished running the command, jobControlId = JOB_CONTROL_ID_1 -``` - -Turn on async submission mode. Run this command to get JOB_CONTROL_ID, then use getCmdStatus to check command detailed status: -```shell -$ ./bin/alluxio fs distributedCp /data/1023 /data/1025 --async - -Sample Output: -Entering async submission mode. -Please wait for command submission to finish.. -Submitted migrate job successfully, jobControlId = JOB_CONTROL_ID_2 -``` - -### distributedLoad - -The `distributedLoad` command loads a file or directory from the under storage system into Alluxio storage distributed -across workers using the job service. The job is a no-op if the file is already loaded into Alluxio. -By default, the command runs synchronously and the user will get a `JOB_CONTROL_ID` after the command successfully submits the job to be executed. -The command will wait until the job is complete, at which point the user will see the list of files loaded and statistics on which files completed or failed. -The command can also run in async mode with the `--async` flag. Similar to before, the user will get a `JOB_CONTROL_ID` after the command successfully submits the job. -The difference is that the command will not wait for the job to finish. -Users can use the [`getCmdStatus`](#getCmdStatus) command with the `JOB_CONTROL_ID` as an argument to check detailed status information about the job. - -If `distributedLoad` is run on a directory, files in the directory will be recursively loaded and each file will be loaded -on a random worker. - -**Options** - -* `--replication`: Specifies how many workers to load each file into. The default value is `1`. -* `--active-jobs`: Limits how many jobs can be submitted to the Alluxio job service at the same time. -Later jobs must wait until some earlier jobs to finish. The default value is `3000`. -A lower value means slower execution but also being nicer to the other users of the job service. -* `--batch-size`: Specifies how many files to be batched into one request. The default value is `20`. Notice that if some task failed in the batched job, the whole batched job would fail with some completed tasks and some failed tasks. -* `--host-file `: Specifies a file contains worker hosts to load target data, each line has a worker host. -* `--hosts`: Specifies a list of worker hosts separated by comma to load target data. -* `--excluded-host-file `: Specifies a file contains worker hosts which shouldn't load target data, each line has a worker host. -* `--excluded-hosts`: Specifies a list of worker hosts separated by comma which shouldn't load target data. -* `--locality-file `: Specifies a file contains worker locality to load target data, each line has a locality. -* `--locality`: Specifies a list of worker locality separated by comma to load target data. -* `--excluded-locality-file `: Specifies a file contains worker locality which shouldn't load target data, each line has a worker locality. -* `--excluded-locality`: Specifies a list of worker locality separated by comma which shouldn't load target data. -* `--index`: Specifies a file that lists all files to be loaded -* `--passive-cache`: Specifies using direct cache request or passive cache with read(old implementation) -* `--async`: Specifies whether to wait for command execution to finish. If not explicitly shown then default to run synchronously. - -```shell -$ ./bin/alluxio fs distributedLoad --replication 2 --active-jobs 2000 /data/today - -Sample Output: -Please wait for command submission to finish.. -Submitted successfully, jobControlId = JOB_CONTROL_ID_3 -Waiting for the command to finish ... -Get command status information below: -Successfully loaded path /data/today/$FILE_PATH_1 -Successfully loaded path /data/today/$FILE_PATH_2 -Successfully loaded path /data/today/$FILE_PATH_3 -Total completed file count is 3, failed file count is 0 -Finished running the command, jobControlId = JOB_CONTROL_ID_3 -``` -Turn on async submission mode. Run this command to get JOB_CONTROL_ID, then use getCmdStatus to check command detailed status: -```shell -$ ./bin/alluxio fs distributedLoad /data/today --async - -Sample Output: -Entering async submission mode. -Please wait for command submission to finish.. -Submitted distLoad job successfully, jobControlId = JOB_CONTROL_ID_4 -``` - -Or you can include some workers or exclude some workers by using options `--host-file `, `--hosts`, `--excluded-host-file `, -`--excluded-hosts`, `--locality-file `, `--locality`, `--excluded-host-file ` and `--excluded-locality`. - -Note: Do not use `--host-file `, `--hosts`, `--locality-file `, `--locality` with -`--excluded-host-file `, `--excluded-hosts`, `--excluded-host-file `, `--excluded-locality` together. - -```shell -# Only include host1 and host2 -$ ./bin/alluxio fs distributedLoad /data/today --hosts host1,host2 -``` - -```shell -# Only include the workset from host file /tmp/hostfile -$ ./bin/alluxio fs distributedLoad /data/today --host-file /tmp/hostfile -``` - -```shell -# Include all workers except host1 and host2 -$ ./bin/alluxio fs distributedLoad /data/today --excluded-hosts host1,host2 -``` - -```shell -# Include all workers except the workerset in the excluded host file /tmp/hostfile-exclude -$ ./bin/alluxio fs distributedLoad /data/today --excluded-file /tmp/hostfile-exclude -``` - -```shell -# Include workers which's locality identify belong to ROCK1 or ROCK2 -$ ./bin/alluxio fs distributedLoad /data/today --locality ROCK1,ROCK2 -``` - -```shell -# Include workers which's locality identify belong to the localities in the locality file -$ ./bin/alluxio fs distributedLoad /data/today --locality-file /tmp/localityfile -``` - -```shell -# Include all workers except which's locality belong to ROCK1 or ROCK2 -$ ./bin/alluxio fs distributedLoad /data/today --excluded-locality ROCK1,ROCK2 -``` - -```shell -# Include all workers except which's locality belong to the localities in the excluded locality file -$ ./bin/alluxio fs distributedLoad /data/today --excluded-locality-file /tmp/localityfile-exclude -``` - -**Conflict Cases:** - -* The `--hosts` and `--locality` are `OR` relationship, so host2,host3 and workers in ROCK2,ROCKS3 will be included: -```shell -$ ./bin/alluxio fs distributedLoad /data/today --locality ROCK2,ROCK3 --hosts host2,host3 -``` - -* The `--excluded-hosts` and `--excluded-locality` are `OR` relationship, so host2,host3 and workers in ROCK2,ROCKS3 will be excluded: -```shell -$ ./bin/alluxio fs distributedLoad /data/today --excluded-hosts host2,host3 --excluded-locality ROCK2,ROCK3 -``` - -### distributedMv - -The `distributedMv` command moves a file or directory in the Alluxio file system distributed across workers -using the job service. - -If the source designates a directory, `distributedMv` moves the entire subtree at source to the destination. - -```shell -$ ./bin/alluxio fs distributedMv /data/1023 /data/1024 -``` - ### head The `head` command prints the first 1 KB of data in a file to the shell. @@ -739,16 +508,14 @@ $ ./bin/alluxio fs leader ### load -The `load` command moves data from the under storage system into Alluxio storage. +The `load` command load data/metadata from the under storage system into Alluxio storage. For example, `load` can be used to prefetch data for analytics jobs. If `load` is run on a directory, files in the directory will be recursively loaded. ```shell -$ ./bin/alluxio fs load --submit [--bandwidth N] [--verify] [--partial-listing] +$ ./bin/alluxio fs load --submit [--metadata-only] ``` **Options:** -* `--bandwidth` option specify how much ufs bandwidth we want to use to load files. -* `--verify` option specify whether we want to verify that all the files are loaded. -* `--partial-listing` option specify using batch listStatus API or traditional listStatus. We would retire this option when batch listStatus API gets mature. +* `--metadata-only` option specify whether loading metadata only After submit the command, you can check the status by running the following ```shell @@ -774,38 +541,6 @@ Progress for loading path '/dir-99': $ ./bin/alluxio fs load --stop ``` -```shell -# If you just want sequential execution for couple files, you can use the following old version -$ ./bin/alluxio fs load -``` -If there is a Alluxio worker on the machine this command is run from, the data will be loaded to that worker. -Otherwise, a random worker will be selected to serve the data. - -If the data is already loaded into Alluxio, load is a no-op unless the `--local flag` is used. -The `--local` flag forces the data to be loaded to a local worker -even if the data is already available on a remote worker. -```shell -$ ./bin/alluxio fs load --local -``` - -### loadMetadata - -The `loadMetadata` command loads metadata about a path in the UFS to Alluxio. -No data will be transferred. -This command is a client-side optimization without storing all returned `ls` results, preventing OOM for massive amount of small files. -This is useful when data has been added to the UFS outside of Alluxio and users are expected to reference the new data. -This command is more efficient than using the `ls` command since it does not store any directory or file information to be returned. - -**Options:** -* `-R` option recursively loads metadata in subdirectories -* `-F` option updates the metadata of the existing file forcibly - -For example, `loadMetadata` can be used to load metadata for a path in the UFS. -The -F option will force the loading of metadata even if there are existing metadata entries for the path. -```shell -$ ./bin/alluxio fs loadMetadata -R -F -``` - ### ls The `ls` command lists all the immediate children in a directory and displays the file size, last From 93082ccc0ff1a0631f05597b2d125d6a4670ae14 Mon Sep 17 00:00:00 2001 From: elega <445092967@qq.com> Date: Tue, 1 Aug 2023 17:23:11 +0800 Subject: [PATCH 07/12] Add multipart upload interface & hdfs minicluster test ### What changes are proposed in this pull request? Add multipart upload interface & add HDFS unit tests using HDFS minicluster ### Why are the changes needed? To increase code quality ### Does this PR introduce any user facing changes? Please list the user-facing changes introduced by your change, including 1. change in user-facing APIs 2. addition or removal of property keys 3. webui pr-link: Alluxio/alluxio#17862 change-id: cid-e01bdb47bdeb9218354040f69587a3550972bdc5 --- .../main/proto/grpc/file_system_master.proto | 1 + .../client/file/ufs/UfsBaseFileSystem.java | 3 + .../underfs/ByteBufferResourcePool.java | 50 ++++++ .../alluxio/underfs/MultipartUploader.java | 59 +++++++ .../underfs/options/CreateOptions.java | 17 ++ .../java/alluxio/fuse/AlluxioFuseUtils.java | 13 +- dora/underfs/hdfs/pom.xml | 29 ++++ .../underfs/hdfs/HdfsUnderFileSystem.java | 8 +- .../HdfsUnderFileSystemIntegrationTest.java | 33 ++++ ...dfsUnderFileSystemIntegrationTestBase.java | 147 ++++++++++++++++++ 10 files changed, 356 insertions(+), 4 deletions(-) create mode 100644 dora/core/common/src/main/java/alluxio/underfs/ByteBufferResourcePool.java create mode 100644 dora/core/common/src/main/java/alluxio/underfs/MultipartUploader.java create mode 100644 dora/underfs/hdfs/src/test/java/alluxio/underfs/hdfs/hdfs3/HdfsUnderFileSystemIntegrationTest.java create mode 100644 dora/underfs/hdfs/src/test/java/alluxio/underfs/hdfs/hdfs3/HdfsUnderFileSystemIntegrationTestBase.java diff --git a/common/transport/src/main/proto/grpc/file_system_master.proto b/common/transport/src/main/proto/grpc/file_system_master.proto index 639f087bdd0c..87285dd51545 100644 --- a/common/transport/src/main/proto/grpc/file_system_master.proto +++ b/common/transport/src/main/proto/grpc/file_system_master.proto @@ -152,6 +152,7 @@ message CreateFilePOptions { // If specified, the data will be written to the certain worker optional grpc.WorkerNetAddress workerLocation = 15; optional bool isAtomicWrite = 16; + optional bool useMultipartUpload = 17; } message CreateFilePRequest { /** the path of the file */ diff --git a/dora/core/client/fs/src/main/java/alluxio/client/file/ufs/UfsBaseFileSystem.java b/dora/core/client/fs/src/main/java/alluxio/client/file/ufs/UfsBaseFileSystem.java index 64667845dbc7..9149bbf9911d 100644 --- a/dora/core/client/fs/src/main/java/alluxio/client/file/ufs/UfsBaseFileSystem.java +++ b/dora/core/client/fs/src/main/java/alluxio/client/file/ufs/UfsBaseFileSystem.java @@ -184,6 +184,9 @@ public FileOutStream createFile(AlluxioURI path, CreateFilePOptions options) { if (options.hasIsAtomicWrite()) { ufsOptions.setEnsureAtomic(options.getIsAtomicWrite()); } + if (options.hasUseMultipartUpload()) { + ufsOptions.setMultipartUploadEnabled(options.getUseMultipartUpload()); + } return new UfsFileOutStream(mUfs.get().create(path.getPath(), ufsOptions)); }); } diff --git a/dora/core/common/src/main/java/alluxio/underfs/ByteBufferResourcePool.java b/dora/core/common/src/main/java/alluxio/underfs/ByteBufferResourcePool.java new file mode 100644 index 000000000000..ed82189a2fb7 --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/underfs/ByteBufferResourcePool.java @@ -0,0 +1,50 @@ +/* + * 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.underfs; + +import alluxio.resource.ResourcePool; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * A simple byte buffer resource pool. Byte buffer can be replaced with netty ByteBuf in the future. + */ +public class ByteBufferResourcePool extends ResourcePool { + private final int mBufferSize; + + /** + * Creates an instance. + * @param maxCapacity the max capacity + * @param bufferSize the buffer size + */ + public ByteBufferResourcePool(int maxCapacity, int bufferSize) { + super(maxCapacity); + mBufferSize = bufferSize; + } + + @Override + public void close() throws IOException { + // No-op + } + + @Override + public void release(ByteBuffer resource) { + resource.clear(); + super.release(resource); + } + + @Override + public ByteBuffer createNewResource() { + return ByteBuffer.allocate(mBufferSize); + } +} diff --git a/dora/core/common/src/main/java/alluxio/underfs/MultipartUploader.java b/dora/core/common/src/main/java/alluxio/underfs/MultipartUploader.java new file mode 100644 index 000000000000..6307e3e7db9a --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/underfs/MultipartUploader.java @@ -0,0 +1,59 @@ +/* + * 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.underfs; + +import com.google.common.util.concurrent.ListenableFuture; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * The multipart uploader interface to support multipart uploading. + * The interface is inspired by hadoop {@link org.apache.hadoop.fs.impl.FileSystemMultipartUploader} + */ +public interface MultipartUploader { + /** + * Initialize a multipart upload. + * @throws IOException IO failure + */ + void startUpload() throws IOException; + + /** + * Put part as part of a multipart upload. + * It is possible to have parts uploaded in any order (or in parallel). + * stream after reading in the data. + * @param b the byte array to put. The byte buffer must have been flipped and be ready to read + * @param partNumber the part number of this file part + * @return a future of the async upload task + * @throws IOException IO failure + */ + ListenableFuture putPart(ByteBuffer b, int partNumber) + throws IOException; + + /** + * Complete a multipart upload. + * @throws IOException IO failure + */ + void complete() throws IOException; + + /** + * Aborts a multipart upload. + * @throws IOException IO failure + */ + void abort() throws IOException; + + /** + * Wait for the ongoing uploads to complete. + * @throws IOException IO failure + */ + void flush() throws IOException; +} diff --git a/dora/core/common/src/main/java/alluxio/underfs/options/CreateOptions.java b/dora/core/common/src/main/java/alluxio/underfs/options/CreateOptions.java index 0ebc59fec726..f06c4255622a 100644 --- a/dora/core/common/src/main/java/alluxio/underfs/options/CreateOptions.java +++ b/dora/core/common/src/main/java/alluxio/underfs/options/CreateOptions.java @@ -40,6 +40,7 @@ public class CreateOptions { private String mGroup; private Mode mMode; private AccessControlList mAcl; + private boolean mMultipartUploadEnabled; /** * @param conf Alluxio configuration @@ -104,6 +105,13 @@ public boolean isEnsureAtomic() { return mEnsureAtomic; } + /** + * @return true, if multipart upload is enabled + */ + public boolean isMultipartUploadEnabled() { + return mMultipartUploadEnabled; + } + /** * Sets an initial acl for the newly created file. * @@ -167,6 +175,15 @@ public CreateOptions setMode(Mode mode) { return this; } + /** + * @param value if multipart upload is enabled + * @return the updated object + */ + public CreateOptions setMultipartUploadEnabled(boolean value) { + mMultipartUploadEnabled = value; + return this; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/dora/integration/fuse/src/main/java/alluxio/fuse/AlluxioFuseUtils.java b/dora/integration/fuse/src/main/java/alluxio/fuse/AlluxioFuseUtils.java index 6e16b841ced0..1ff2773b7a7f 100644 --- a/dora/integration/fuse/src/main/java/alluxio/fuse/AlluxioFuseUtils.java +++ b/dora/integration/fuse/src/main/java/alluxio/fuse/AlluxioFuseUtils.java @@ -74,6 +74,7 @@ import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; import javax.annotation.concurrent.ThreadSafe; /** @@ -95,9 +96,19 @@ public final class AlluxioFuseUtils { public static final long MODE_NOT_SET_VALUE = -1; public static final String MACFUSE_SUPPORT_MINIMUM_OS_VERSION = "10.9"; + private static Supplier sDefaultCreateFilePOptionBuilderSupplier = + CreateFilePOptions::newBuilder; private AlluxioFuseUtils() {} + /** + * @param supplier the new supplier + */ + public static void updateDefaultCreateFilePOptionBuilderSupplier( + Supplier supplier) { + sDefaultCreateFilePOptionBuilderSupplier = supplier; + } + /** * Checks the input file length. * @@ -124,7 +135,7 @@ public static int checkNameLength(AlluxioURI uri) { */ public static FileOutStream createFile(FileSystem fileSystem, AuthPolicy authPolicy, AlluxioURI uri, CreateFileStatus fileStatus) { - CreateFilePOptions.Builder optionsBuilder = CreateFilePOptions.newBuilder(); + CreateFilePOptions.Builder optionsBuilder = sDefaultCreateFilePOptionBuilderSupplier.get(); if (fileStatus.getMode() != MODE_NOT_SET_VALUE) { optionsBuilder.setMode(new Mode((short) fileStatus.getMode()).toProto()); } diff --git a/dora/underfs/hdfs/pom.xml b/dora/underfs/hdfs/pom.xml index d6b1de725b6a..9524e228a2c7 100644 --- a/dora/underfs/hdfs/pom.xml +++ b/dora/underfs/hdfs/pom.xml @@ -67,6 +67,29 @@ ${ufs.hadoop.version} + + + + org.apache.maven.plugins + maven-compiler-plugin + + + default-testCompile + test-compile + + + + **/hdfs3/** + + + + testCompile + + + + + + @@ -81,6 +104,12 @@ alluxio-shaded-hadoop ${ufs.hadoop.version} + + org.apache.hadoop + hadoop-minicluster + ${ufs.hadoop.version} + test + diff --git a/dora/underfs/hdfs/src/main/java/alluxio/underfs/hdfs/HdfsUnderFileSystem.java b/dora/underfs/hdfs/src/main/java/alluxio/underfs/hdfs/HdfsUnderFileSystem.java index a411d223972c..7a56a010492f 100755 --- a/dora/underfs/hdfs/src/main/java/alluxio/underfs/hdfs/HdfsUnderFileSystem.java +++ b/dora/underfs/hdfs/src/main/java/alluxio/underfs/hdfs/HdfsUnderFileSystem.java @@ -41,6 +41,7 @@ import alluxio.util.UnderFileSystemUtils; import alluxio.util.network.NetworkAddressUtils; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; @@ -82,7 +83,7 @@ public class HdfsUnderFileSystem extends ConsistentUnderFileSystem implements AtomicFileOutputStreamCallback { private static final Logger LOG = LoggerFactory.getLogger(HdfsUnderFileSystem.class); - private static final int MAX_TRY = 5; + protected static final int MAX_TRY = 5; protected static final String HDFS_USER = ""; /** Name of the class for the HDFS Acl provider. */ protected static final String HDFS_ACL_PROVIDER_CLASS = @@ -115,7 +116,7 @@ public class HdfsUnderFileSystem extends ConsistentUnderFileSystem "dfs.checksum.combine.mode"; private final LoadingCache mUserFs; - private final HdfsAclProvider mHdfsAclProvider; + protected final HdfsAclProvider mHdfsAclProvider; private HdfsActiveSyncProvider mHdfsActiveSyncer; @@ -887,7 +888,8 @@ public boolean isSeekable() { /** * @return the underlying HDFS {@link FileSystem} object */ - protected FileSystem getFs() throws IOException { + @VisibleForTesting + public FileSystem getFs() throws IOException { try { // TODO(gpang): handle different users return mUserFs.get(HDFS_USER); diff --git a/dora/underfs/hdfs/src/test/java/alluxio/underfs/hdfs/hdfs3/HdfsUnderFileSystemIntegrationTest.java b/dora/underfs/hdfs/src/test/java/alluxio/underfs/hdfs/hdfs3/HdfsUnderFileSystemIntegrationTest.java new file mode 100644 index 000000000000..022a02ef4811 --- /dev/null +++ b/dora/underfs/hdfs/src/test/java/alluxio/underfs/hdfs/hdfs3/HdfsUnderFileSystemIntegrationTest.java @@ -0,0 +1,33 @@ +/* + * 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.underfs.hdfs.hdfs3; + +import org.junit.Test; + +import java.io.IOException; + +public class HdfsUnderFileSystemIntegrationTest extends HdfsUnderFileSystemIntegrationTestBase { + @Test + public void testWriteEmptyFile() throws Exception { + writeEmptyFileTest(); + } + + @Test + public void testWriteMultiBlockFileTest() throws Exception { + writeMultiBlockFileTest("/test_file"); + } + + @Test(expected = IOException.class) + public void testException() throws Exception { + hdfsDownDuringUploadTest(); + } +} diff --git a/dora/underfs/hdfs/src/test/java/alluxio/underfs/hdfs/hdfs3/HdfsUnderFileSystemIntegrationTestBase.java b/dora/underfs/hdfs/src/test/java/alluxio/underfs/hdfs/hdfs3/HdfsUnderFileSystemIntegrationTestBase.java new file mode 100644 index 000000000000..2a1053630945 --- /dev/null +++ b/dora/underfs/hdfs/src/test/java/alluxio/underfs/hdfs/hdfs3/HdfsUnderFileSystemIntegrationTestBase.java @@ -0,0 +1,147 @@ +/* + * 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.underfs.hdfs.hdfs3; + +import static junit.framework.TestCase.assertEquals; + +import alluxio.AlluxioURI; +import alluxio.underfs.UnderFileSystemConfiguration; +import alluxio.underfs.hdfs.HdfsUnderFileSystem; +import alluxio.underfs.options.CreateOptions; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.text.CharacterPredicates; +import org.apache.commons.text.RandomStringGenerator; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +public class HdfsUnderFileSystemIntegrationTestBase { + @Rule + public TemporaryFolder mTemp = new TemporaryFolder(); + protected final Configuration mHdfsConfiguration = new Configuration(); + + protected MiniDFSCluster mCluster; + protected HdfsUnderFileSystem mUfs; + + private static final int BLOCK_SIZE = 1024 * 1024; + + @Before + public void before() throws IOException { + mHdfsConfiguration.set("dfs.name.dir", mTemp.newFolder("nn").getAbsolutePath()); + mHdfsConfiguration.set("dfs.data.dir", mTemp.newFolder("dn").getAbsolutePath()); + // 1MB block size for testing to save memory + mHdfsConfiguration.setInt("dfs.block.size", BLOCK_SIZE); + + mCluster = new MiniDFSCluster.Builder(mHdfsConfiguration) + .enableManagedDfsDirsRedundancy(false) + .manageDataDfsDirs(false) + .manageNameDfsDirs(false) + .numDataNodes(1).build(); + + UnderFileSystemConfiguration ufsConf = + UnderFileSystemConfiguration.defaults(alluxio.conf.Configuration.global()); + + setConfiguration(); + + mUfs = createUfs(ufsConf); + } + + @After + public void after() { + if (mCluster != null) { + mCluster.close(); + } + if (mUfs != null) { + mUfs.close(); + } + } + + protected void writeMultiBlockFileTest(String testFilePath) throws IOException { + // 16MB + 1 byte, 17 blocks + int fileLength = 1024 * 1024 * 16 + 1; + int numHdfsBlocks = (fileLength - 1) / BLOCK_SIZE + 1; + + RandomStringGenerator randomStringGenerator = + new RandomStringGenerator.Builder() + .withinRange('0', 'z') + .filteredBy(CharacterPredicates.LETTERS, CharacterPredicates.DIGITS) + .build(); + String fileContentToWrite = randomStringGenerator.generate(fileLength); + + OutputStream os = mUfs.create(testFilePath, getCreateOption()); + os.write(fileContentToWrite.getBytes()); + os.close(); + + InputStream is = mUfs.open(testFilePath); + String readFileContent = IOUtils.toString(is); + Assert.assertEquals(fileContentToWrite, readFileContent); + + assertEquals(fileLength, mUfs.getStatus(testFilePath).asUfsFileStatus().getContentLength()); + FileStatus status = mUfs.getFs().getFileStatus(new Path(testFilePath)); + assertEquals(numHdfsBlocks, + mUfs.getFs().getFileBlockLocations(status, 0, status.getLen()).length); + } + + protected void writeEmptyFileTest() throws IOException { + String testFilePath = "/empty_file"; + OutputStream os = mUfs.create(testFilePath, getCreateOption()); + os.close(); + assertEquals(0, mUfs.getStatus(testFilePath).asUfsFileStatus().getContentLength()); + } + + protected void hdfsDownDuringUploadTest() throws Exception { + String testFilePath = "/test_file"; + + RandomStringGenerator randomStringGenerator = + new RandomStringGenerator.Builder() + .withinRange('0', 'z') + .filteredBy(CharacterPredicates.LETTERS, CharacterPredicates.DIGITS) + .build(); + String oneBlockFileContent = randomStringGenerator.generate(1024 * 1024); + + OutputStream os = mUfs.create(testFilePath, getCreateOption()); + os.write(oneBlockFileContent.getBytes()); + os.write(oneBlockFileContent.getBytes()); + // Stop the data node in the middle of the write process. + mCluster.stopDataNode(0); + os.write(oneBlockFileContent.getBytes()); + os.close(); + } + + protected HdfsUnderFileSystem createUfs(UnderFileSystemConfiguration ufsConf) { + return new HdfsUnderFileSystem(new AlluxioURI("/"), ufsConf, mHdfsConfiguration) { + @Override + public FileSystem getFs() throws IOException { + // Hookup HDFS mini cluster to HDFS UFS + return mCluster.getFileSystem(); + } + }; + } + + protected void setConfiguration() {} + + protected CreateOptions getCreateOption() { + return CreateOptions.defaults(alluxio.conf.Configuration.global()); + } +} From b1e18e3e6982e1bb0e909b19c814208c68ac8325 Mon Sep 17 00:00:00 2001 From: Huang Hua Date: Tue, 1 Aug 2023 19:03:08 +0800 Subject: [PATCH 08/12] rename variables for better reading Signed-off-by: Huang Hua --- .../alluxio/client/file/DoraCacheFileSystem.java | 6 +++--- .../src/main/java/alluxio/client/file/URIStatus.java | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dora/core/client/fs/src/main/java/alluxio/client/file/DoraCacheFileSystem.java b/dora/core/client/fs/src/main/java/alluxio/client/file/DoraCacheFileSystem.java index cbca547a0679..4fec84039b7f 100644 --- a/dora/core/client/fs/src/main/java/alluxio/client/file/DoraCacheFileSystem.java +++ b/dora/core/client/fs/src/main/java/alluxio/client/file/DoraCacheFileSystem.java @@ -152,7 +152,7 @@ public URIStatus getStatus(AlluxioURI path, GetStatusPOptions options) UFS_FALLBACK_COUNTER.inc(); LOG.debug("Dora client get status error ({} times). Fall back to UFS.", UFS_FALLBACK_COUNTER.getCount(), ex); - return mDelegatedFileSystem.getStatus(ufsFullPath, options).setIsFromUFS(); + return mDelegatedFileSystem.getStatus(ufsFullPath, options).setFromUFSFallBack(); } } @@ -176,8 +176,8 @@ public FileInStream openFile(URIStatus status, OpenFilePOptions options) OpenFilePOptions mergedOptions = FileSystemOptionsUtils.openFileDefaults(conf) .toBuilder().mergeFrom(options).build(); try { - if (status.getIsFromUFS()) { - throw new RuntimeException("Status is retrieved from UFS by falling back"); + if (status.isFromUFSFallBack()) { + throw new RuntimeException("Status is retrieved from UFS by falling back."); } Protocol.OpenUfsBlockOptions openUfsBlockOptions = Protocol.OpenUfsBlockOptions.newBuilder().setUfsPath(status.getUfsPath()) diff --git a/dora/core/common/src/main/java/alluxio/client/file/URIStatus.java b/dora/core/common/src/main/java/alluxio/client/file/URIStatus.java index f4a927396d78..71b7541badf5 100644 --- a/dora/core/common/src/main/java/alluxio/client/file/URIStatus.java +++ b/dora/core/common/src/main/java/alluxio/client/file/URIStatus.java @@ -43,7 +43,7 @@ public class URIStatus { /** Context associated with this URI, possibly set by other external engines (e.g., presto). */ private final CacheContext mCacheContext; - private boolean mIsFromUFS; + private boolean mFromUFSFallBack; /** * Constructs an instance of this class from a {@link FileInfo}. @@ -63,7 +63,7 @@ public URIStatus(FileInfo info) { public URIStatus(FileInfo info, @Nullable CacheContext context) { mInfo = Preconditions.checkNotNull(info, "info"); mCacheContext = context; - mIsFromUFS = false; + mFromUFSFallBack = false; } /** @@ -71,8 +71,8 @@ public URIStatus(FileInfo info, @Nullable CacheContext context) { * * If true, it means this status is retrieved by falling back to UFS. */ - public boolean getIsFromUFS() { - return mIsFromUFS; + public boolean isFromUFSFallBack() { + return mFromUFSFallBack; } /** @@ -86,8 +86,8 @@ public boolean getIsFromUFS() { * * @return this object itself */ - public URIStatus setIsFromUFS() { - mIsFromUFS = true; + public URIStatus setFromUFSFallBack() { + mFromUFSFallBack = true; return this; } From f92b917e3fd7ffc66a25853420941b2f8e75ea32 Mon Sep 17 00:00:00 2001 From: elega <445092967@qq.com> Date: Tue, 1 Aug 2023 22:54:07 +0800 Subject: [PATCH 09/12] Generate test-jar for HDFS UFS ### What changes are proposed in this pull request? There are some other packages need to reference the test file in HDFS UFS package so we create a test jar here. ### Does this PR introduce any user facing changes? N/A pr-link: Alluxio/alluxio#17872 change-id: cid-3483198bbd68e29d116370b7a1dbd7c63b2178b3 --- dora/underfs/hdfs/pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/dora/underfs/hdfs/pom.xml b/dora/underfs/hdfs/pom.xml index 9524e228a2c7..2816b431cd46 100644 --- a/dora/underfs/hdfs/pom.xml +++ b/dora/underfs/hdfs/pom.xml @@ -166,6 +166,18 @@ + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + org.apache.maven.plugins maven-compiler-plugin From a565dae347d98e7a61555c031a5d77a0719c2113 Mon Sep 17 00:00:00 2001 From: ccmao1130 Date: Tue, 1 Aug 2023 12:24:14 -0700 Subject: [PATCH 10/12] [DOCFIX] Reformat Storage Integrations ### What changes are proposed in this pull request? New "Storage Integrations Overview" page that holds all prerequisites and links to each storage system page Each storage page has been updated to have the same language for prerequisites, basic setup, and running locally. Each storage page has a table for prerequisites with useful links for creating bucket and getting keys & all external links open in a new tab. ### Why are the changes needed? Storage Overview page to give an overview of what storage systems can integrate with Alluxio and its necessary properties before set up. Consistency in language, format, and information across all storage integration pages. ### Does this PR introduce any user facing changes? Web UI pr-link: Alluxio/alluxio#17853 change-id: cid-8233b2f9be2433b03764901a3931f923b85ed23e --- docs/_data/menu-en.yml | 26 ++-- docs/en/reference/Environment-List.md | 2 +- docs/en/ufs/Aliyun-OSS.md | 95 ++++++++++++ docs/en/ufs/Azure-Blob-Store.md | 73 ++++------ docs/en/ufs/Azure-Data-Lake-Gen2.md | 107 +++++--------- docs/en/ufs/Azure-Data-Lake.md | 79 +++++----- docs/en/ufs/COS.md | 197 ------------------------- docs/en/ufs/CephFS.md | 88 ++++++----- docs/en/ufs/CephObjectStorage.md | 161 ++++++++++++++------- docs/en/ufs/GCS.md | 80 ++++------ docs/en/ufs/HDFS.md | 63 ++++---- docs/en/ufs/Huawei-OBS.md | 106 ++++++++++++++ docs/en/ufs/Minio.md | 82 ++++++----- docs/en/ufs/NFS.md | 45 ++---- docs/en/ufs/OBS.md | 110 -------------- docs/en/ufs/OSS.md | 102 ------------- docs/en/ufs/Ozone.md | 110 +++++++++----- docs/en/ufs/Qiniu-KODO.md | 82 +++++------ docs/en/ufs/S3.md | 65 ++++----- docs/en/ufs/Storage-Overview.md | 176 ++++++++++++++++++++++ docs/en/ufs/Swift.md | 115 ++++++++------- docs/en/ufs/Tencent-COS.md | 201 ++++++++++++++++++++++++++ 22 files changed, 1173 insertions(+), 992 deletions(-) create mode 100644 docs/en/ufs/Aliyun-OSS.md delete mode 100644 docs/en/ufs/COS.md create mode 100644 docs/en/ufs/Huawei-OBS.md delete mode 100644 docs/en/ufs/OBS.md delete mode 100644 docs/en/ufs/OSS.md create mode 100644 docs/en/ufs/Storage-Overview.md create mode 100644 docs/en/ufs/Tencent-COS.md diff --git a/docs/_data/menu-en.yml b/docs/_data/menu-en.yml index f143959988f1..5f032e9c7c00 100644 --- a/docs/_data/menu-en.yml +++ b/docs/_data/menu-en.yml @@ -64,38 +64,40 @@ - title: Storage Integrations buttonTitle: Storage_Integrations subfiles: + - title: Storage Integrations Overview + url: /en/ufs/Storage-Overview.html - title: Amazon AWS S3 url: /en/ufs/S3.html - title: HDFS url: /en/ufs/HDFS.html + - title: Aliyun OSS + url: /en/ufs/Aliyun-OSS.html - title: Azure Blob Store url: /en/ufs/Azure-Blob-Store.html - title: Azure Data Lake Storage url: /en/ufs/Azure-Data-Lake.html - title: Azure Data Lake Storage Gen2 url: /en/ufs/Azure-Data-Lake-Gen2.html - - title: Google Cloud Storage - url: /en/ufs/GCS.html - - title: Qiniu Kodo - url: /en/ufs/Qiniu-KODO.html - - title: Tencent COS - url: /en/ufs/COS.html + - title: CephFS + url: /en/ufs/CephFS.html - title: CephObjectStorage url: /en/ufs/CephObjectStorage.html + - title: Google Cloud Storage + url: /en/ufs/GCS.html + - title: Huawei OBS + url: /en/ufs/Huawei-OBS.html - title: MinIO url: /en/ufs/Minio.html - title: NFS url: /en/ufs/NFS.html - - title: Aliyun Object Storage Service - url: /en/ufs/OSS.html - title: Ozone url: /en/ufs/Ozone.html + - title: Qiniu Kodo + url: /en/ufs/Qiniu-KODO.html - title: Swift url: /en/ufs/Swift.html - - title: CephFS - url: /en/ufs/CephFS.html - - title: Huawei OBS - url: /en/ufs/OBS.html + - title: Tencent COS + url: /en/ufs/Tencent-COS.html - title: FUSE SDK buttonTitle: FUSE_SDK diff --git a/docs/en/reference/Environment-List.md b/docs/en/reference/Environment-List.md index ec4afc5ed350..c2d2eef88046 100644 --- a/docs/en/reference/Environment-List.md +++ b/docs/en/reference/Environment-List.md @@ -16,7 +16,7 @@ variables. {% endfor %} -For example, the following example will set up: +The following example will set up: - an Alluxio master at `localhost` - the root mount point as an HDFS cluster with a namenode also running at `localhost` - defines the maximum heap space of the VM to be 30g diff --git a/docs/en/ufs/Aliyun-OSS.md b/docs/en/ufs/Aliyun-OSS.md new file mode 100644 index 000000000000..ad8b01e731e8 --- /dev/null +++ b/docs/en/ufs/Aliyun-OSS.md @@ -0,0 +1,95 @@ +--- +layout: global +title: Aliyun Object Storage Service +--- + + +This guide describes how to configure [Aliyun OSS](https://intl.aliyun.com/product/oss){:target="_blank"} as Alluxio's under storage system. + +Aliyun Object Storage Service (OSS) is a massive, secure and highly reliable cloud storage service provided by Alibaba Cloud. OSS provides multiple storage classes to help you manage and reduce storage costs. + +For more information about Aliyun OSS, please read its [documentation](https://www.alibabacloud.com/help/en/oss/){:target="_blank"} + +## Prerequisites + +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using OSS with Alluxio: + + + + + + + + + + + + + + + + + + + + + +
``[Create a a new bucket in the OSS console](https://www.alibabacloud.com/help/en/oss/getting-started/create-buckets-6#task-2013189){:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``ID used to identify a user. See [How to Obtain AccessKey Pair](https://www.alibabacloud.com/help/en/sls/developer-reference/accesskey-pair){:target="_blank"}
``Secret is used to verify the identify of the user. See [How to Obtain AccessKey Pair](https://www.alibabacloud.com/help/en/sls/developer-reference/accesskey-pair){:target="_blank"}
``Endpoints are the domain names that other services can use to access OSS. See [Regions and OSS Endpoints in the Public Cloud](https://www.alibabacloud.com/help/en/oss/user-guide/regions-and-endpoints){:target="_blank"}
+ +## Basic Setup + +To use Aliyun OSS as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. + +```shell +$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties +``` +Specify an OSS bucket and directory as the underfs address by modifying `conf/alluxio-site.properties`. + +For example, the under storage address can be `oss://alluxio-bucket/` if +you want to mount the whole bucket to Alluxio, or `oss://alluxio-bucket/alluxio/data` if only the directory `/alluxio/data` +inside the oss bucket `alluxio-bucket` is mapped to Alluxio. + +```properties +alluxio.dora.client.ufs.root=oss:/// +``` + +Specify credentials for Aliyun OSS access by adding the following properties in `conf/alluxio-site.properties`: + +```properties +fs.oss.accessKeyId= +fs.oss.accessKeySecret= +fs.oss.endpoint= +``` + +## Running Alluxio Locally with Aliyun OSS + +Once you have configured Alluxio to Aliyun OSS, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. + +## Advanced Setup + +### [Experimental] OSS multipart upload + +The default upload method uploads one file completely from start to end in one go. We use multipart-upload method to upload one file by multiple parts, every part will be uploaded in one thread. It won't generate any temporary files while uploading. + +To enable OSS multipart upload, you need to modify `conf/alluxio-site.properties` to include: + +```properties +alluxio.underfs.oss.multipart.upload.enabled=true +``` + +There are other parameters you can specify in `conf/alluxio-site.properties` to make the process faster and better. + +```properties +# Timeout for uploading part when using multipart upload. +alluxio.underfs.object.store.multipart.upload.timeout +``` +```properties +# Thread pool size for OSS multipart upload. +alluxio.underfs.oss.multipart.upload.threads +``` +```properties +# Multipart upload partition size for OSS. The default partition size is 64MB. +alluxio.underfs.oss.multipart.upload.partition.size +``` diff --git a/docs/en/ufs/Azure-Blob-Store.md b/docs/en/ufs/Azure-Blob-Store.md index 42e6b7175528..face20bd9626 100644 --- a/docs/en/ufs/Azure-Blob-Store.md +++ b/docs/en/ufs/Azure-Blob-Store.md @@ -4,28 +4,40 @@ title: Azure Blob Store --- -This guide describes how to configure Alluxio with [Azure Blob Store](https://azure.microsoft.com/en-in/services/storage/blobs/) as the under storage system. +This guide describes how to configure Alluxio with [Azure Blob Store](https://azure.microsoft.com/en-us/products/storage/blobs/){:target="_blank"} as the under storage system. Azure Blob Storage is Microsoft's object storage solution for the cloud. Blob Storage is optimized for storing massive amounts of unstructured data. +For more information about Azure Blob Store, please read its +[documentation](https://learn.microsoft.com/en-us/azure/storage/blobs/){:target="_blank"}. ## Prerequisites -In preparation for using Azure Blob Store with Alluxio, create a new container in your Azure -storage account or use an existing container. You should also note that the directory you want to -use in that container, either by creating a new directory in the container, or using an existing -one. For the purposes of this guide, the Azure storage account name is called ``, the -container in that storage account is called `` and the directory in that container is -called ``. For more information about Azure storage account, please see -[here](https://docs.microsoft.com/en-us/azure/storage/storage-create-storage-account). - +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using Azure Blob Store with Alluxio: + + + + + + + + + + + + + + + + + +
``[Create a new container in your Azure storage account](https://docs.microsoft.com/en-us/azure/storage/storage-create-storage-account){:target="_blank"} or use an existing container.
``The directory you want to use in that container, either by creating a new directory or using an existing one
``Your Azure storage account
``Your Azure account credientials
## Basic Setup -To use Azure blob store as the UFS of Alluxio root mount point, -you need to configure Alluxio to use under storage systems by modifying -`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the -template. +To use Azure Blob Store as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. ```shell $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties @@ -37,44 +49,17 @@ Specify the underfs address by modifying `conf/alluxio-site.properties` to inclu alluxio.dora.client.ufs.root=wasbs://@.blob.core.windows.net// ``` -Specify credentials for the Azure account of the root mount point by adding the following -properties in `conf/alluxio-site.properties`: +Specify credentials for the Azure account of the root mount point by adding the following properties in `conf/alluxio-site.properties`: ```properties -fs.azure.account.key..blob.core.windows.net= +fs.azure.account.key..blob.core.windows.net= ``` ## Running Alluxio Locally with Azure Blob Store -Start up Alluxio locally to see that everything works. - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -This should start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` - -Visit your container `` to verify the files and directories created by Alluxio exist. For this test, you should see files named like: - -``` -//default_tests_files/BASIC_CACHE_PROMOTE_CACHE_THROUGH -``` - -To stop Alluxio, you can run: - -``` shell -$ ./bin/alluxio-stop.sh local -``` +Once you have configured Alluxio to Azure Blob Store, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. -## FAQ +## Troubleshooting ### What should I do if I get http not support error? If you mount the Blob and configure the Blob path start with `wasb://`, you may see the error as follows: diff --git a/docs/en/ufs/Azure-Data-Lake-Gen2.md b/docs/en/ufs/Azure-Data-Lake-Gen2.md index b96899b21ad6..420f9aeae05f 100644 --- a/docs/en/ufs/Azure-Data-Lake-Gen2.md +++ b/docs/en/ufs/Azure-Data-Lake-Gen2.md @@ -1,26 +1,41 @@ --- layout: global -title: Azure Storage Gen2 +title: Azure Data Lake Storage Gen2 --- -This guide describes how to configure Alluxio with [Azure Data Lake Storage Gen2](https://docs.microsoft.com/en-in/azure/storage/blobs/data-lake-storage-introduction) as the under storage system. +This guide describes how to configure Alluxio with [Azure Data Lake Storage Gen2](https://azure.microsoft.com/en-us/products/storage/data-lake-storage/){:target="_blank"} as the under storage system. Azure Data Lake Storage Gen2 is a set of capabilities dedicated to big data analytics, built on Azure Blob Storage. It converges the capabilities of Azure Data Lake Storage Gen1 with Azure Blob Storage. -## Prerequisites - -In preparation for using Azure Data Lake storage with Alluxio, [create a new Data Lake storage in your Azure -account](https://docs.microsoft.com/en-in/azure/storage/blobs/create-data-lake-storage-account) or use an existing Data Lake storage. -You should also note the directory you want to use, either by creating a new directory, or using an existing one. You also need a -[SharedKey](https://docs.microsoft.com/en-us/rest/api/storageservices/authorize-with-shared-key). -For the purposes of this guide, the Azure storage account name is called ``, -the directory in that storage account is called ``, and the name of the container is called ``. +For more information about Azure Data Lake Storage Gen1, please read its [documentation](https://docs.microsoft.com/en-in/azure/storage/blobs/data-lake-storage-introduction){:target="_blank"}. +## Prerequisites -## Setup with Shared Key - -To use Azure Data Lake Storage as the UFS of Alluxio root mount point, +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using Azure Data Lake Storage Gen2 with Alluxio, [create a new Data Lake storage in your Azure account](https://learn.microsoft.com/en-us/azure/storage/blobs/create-data-lake-storage-account){:target="_blank"} or use an existing Data Lake storage. + + + + + + + + + + + + + +
``The container you want to use
``The directory you want to use in the container, either by creating a new directory or using an existing one
``Your Azure storage account
+ +You also need a +[SharedKey](https://docs.microsoft.com/en-us/rest/api/storageservices/authorize-with-shared-key){:target="_blank"} to authorize requests. + +## Basic Setup + +To use Azure Data Lake Storage Gen2 as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. @@ -35,28 +50,15 @@ Specify the underfs address by modifying `conf/alluxio-site.properties` to inclu alluxio.dora.client.ufs.root=abfs://@.dfs.core.windows.net// ``` +### Setup with Shared Key + Specify the Shared Key by adding the following property in `conf/alluxio-site.properties`: ```properties -fs.azure.account.key..dfs.core.windows.net= +fs.azure.account.key..dfs.core.windows.net= ``` -## Setup with OAuth 2.0 Client Credentials - -To use Azure Data Lake Storage as the UFS of Alluxio root mount point, -you need to configure Alluxio to use under storage systems by modifying -`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the -template. - -```shell -$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties -``` - -Specify the underfs address by modifying `conf/alluxio-site.properties` to include: - -```properties -alluxio.dora.client.ufs.root=abfs://@.dfs.core.windows.net// -``` +### Setup with OAuth 2.0 Client Credentials Specify the OAuth 2.0 Client Credentials by adding the following property in `conf/alluxio-site.properties`: (Please note that for URL Endpoint, use the V1 token endpoint) @@ -69,55 +71,14 @@ fs.azure.account.oauth2.client.secret= ## Setup with Azure Managed Identities -To use Azure Data Lake Storage as the UFS of Alluxio root mount point, -you need to configure Alluxio to use under storage systems by modifying -`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the -template. - -```shell -$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties -``` - -Specify the underfs address by modifying `conf/alluxio-site.properties` to include: - -```properties -alluxio.dora.client.ufs.root=abfs://@.dfs.core.windows.net// -``` - Specify the Azure Managed Identities by adding the following property in `conf/alluxio-site.properties`: ```properties fs.azure.account.oauth2.msi.endpoint= fs.azure.account.oauth2.client.id= -fs.azure.account.oauth2.msi.tenant= +fs.azure.account.oauth2.msi.tenant= ``` ## Running Alluxio Locally with Data Lake Storage -Start up Alluxio locally to see that everything works. - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -This should start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` - -Visit your directory `` to verify the files and directories created by Alluxio exist. For this test, you should see files named like: - -``` -/default_tests_files/BASIC_CACHE_PROMOTE_CACHE_THROUGH -``` - -To stop Alluxio, you can run: - -```shell -$ ./bin/alluxio-stop.sh local -``` +Once you have configured Alluxio to Azure Data Lake Storage Gen2, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. diff --git a/docs/en/ufs/Azure-Data-Lake.md b/docs/en/ufs/Azure-Data-Lake.md index cd0e7660800f..1409a65fc168 100644 --- a/docs/en/ufs/Azure-Data-Lake.md +++ b/docs/en/ufs/Azure-Data-Lake.md @@ -1,29 +1,52 @@ --- layout: global -title: Azure Data Lake Storage +title: Azure Data Lake Storage Gen1 --- -This guide describes how to configure Alluxio with [Azure Data Lake Storage Gen1](https://docs.microsoft.com/en-in/azure/data-lake-store/data-lake-store-overview) as the under storage system. +This guide describes how to configure Alluxio with [Azure Data Lake Storage Gen1](https://azure.microsoft.com/en-us/products/storage/data-lake-storage/){:target="_blank"} as the under storage system. Azure Data Lake Storage is an enterprise-wide hyper-scale repository for big data analytic workloads. Azure Data Lake enables you to capture data of any size, type, and ingestion speed in one single place for operational and exploratory analytics. It is designed to store and analyze large amounts of structured, semi-structured, and unstructured data. +For more information about Azure Data Lake Storage Gen1, please read its [documentation](https://docs.microsoft.com/en-in/azure/data-lake-store/data-lake-store-overview){:target="_blank"}. -## Prerequisites +**Note**: Azure Data Lake Storage Gen1 will be retired on Feb 29, 2024. Be sure to migrate to Azure Data Lake Storage Gen2 prior to that date. See how [here](https://learn.microsoft.com/en-us/azure/storage/blobs/data-lake-storage-migrate-gen1-to-gen2-azure-portal){:target="_blank"}. -In preparation for using Azure Data Lake storage with Alluxio, create a new Data Lake storage in your Azure -account or use an existing Data Lake storage. You should also note the directory you want to -use, either by creating a new directory, or using an existing one. You also need to set up -[Service-to-service authentication](https://docs.microsoft.com/en-us/azure/data-lake-store/data-lake-store-service-to-service-authenticate-using-active-directory) for your storage account. -For the purposes of this guide, the Azure storage account name is called `` -and the directory in that storage account is called ``. For more information -about Azure storage account, please see -[here](https://docs.microsoft.com/en-us/azure/data-lake-store/data-lake-store-get-started-portal). +## Prerequisites +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using Azure Data Lake Storage Gen1 with Alluxio, [create a new Data Lake storage in your Azure +account](https://docs.microsoft.com/en-us/azure/data-lake-store/data-lake-store-get-started-portal){:target="_blank"} or use an existing Data Lake storage. + + + + + + + + + + + + + + + + + + + + + +
``The directory you want to use, either by creating a new directory or using an existing one
``Your Azure storage account
``See [Get application ID and authentication key](https://learn.microsoft.com/en-us/azure/active-directory/develop/howto-create-service-principal-portal#sign-in-to-the-application){:target="_blank"} for instructions on how to retrieve the application (client) ID and authentication key (also called the client secret) for your application
``See [Get application ID and authentication key](https://learn.microsoft.com/en-us/azure/active-directory/develop/howto-create-service-principal-portal#sign-in-to-the-application){:target="_blank"} for instructions on how to retrieve the application (client) ID and authentication key (also called the client secret) for your application
``See [Get tenant ID](https://learn.microsoft.com/en-us/azure/active-directory/develop/howto-create-service-principal-portal#sign-in-to-the-application){:target="_blank"} for instructions on how to retrieve the tenant ID
+ +You also need to set up +[Service-to-service authentication](https://docs.microsoft.com/en-us/azure/data-lake-store/data-lake-store-service-to-service-authenticate-using-active-directory){:target="_blank"} for your storage account. ## Basic Setup -To use Azure Data Lake Storage as the UFS of Alluxio root mount point, +To use Azure Data Lake Storage Gen1 as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. @@ -40,11 +63,9 @@ alluxio.dora.client.ufs.root=adl://.azuredatalakestore.net/.oauth2.client.id= +fs.adl.account..oauth2.client.id= fs.adl.account..oauth2.credential= fs.adl.account..oauth2.refresh.url=https://login.microsoftonline.com//oauth2/token ``` @@ -53,30 +74,4 @@ After these changes, Alluxio should be configured to work with Azure Data Lake s ## Running Alluxio Locally with Data Lake Storage -Start up Alluxio locally to see that everything works. - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -This should start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` - -Visit your directory `` to verify the files and directories created by Alluxio exist. For this test, you should see files named like: - -``` -/default_tests_files/BASIC_CACHE_PROMOTE_CACHE_THROUGH -``` - -To stop Alluxio, you can run: - -```shell -$ ./bin/alluxio-stop.sh local -``` +Once you have configured Alluxio to Azure Data Lake Storage Gen1, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. diff --git a/docs/en/ufs/COS.md b/docs/en/ufs/COS.md deleted file mode 100644 index 5e51cda47247..000000000000 --- a/docs/en/ufs/COS.md +++ /dev/null @@ -1,197 +0,0 @@ ---- -layout: global -title: Tencent COS ---- - -This guide describes the instructions to configure [Tencent COS](https://cloud.tencent.com/product/cos) as Alluxio's -under storage system. - -Alluxio support two different implementations of under storage system for Tencent COS: - -* [COS](https://cloud.tencent.com/product/cos) -: Tencent Cloud Object Storage (COS) is a distributed storage service offered by Tencent Cloud for unstructured data and accessible via HTTP/HTTPS protocols. It can store massive amounts of data and features imperceptible bandwidth and capacity expansion, making it a perfect data pool for big data computation and analytics. - -* [COSN](https://hadoop.apache.org/docs/stable/hadoop-cos/cloud-storage/index.html), also known as Hadoop-COS -: COSN, also known as Hadoop-COS, is a client that makes the upper computing systems based on HDFS be able to use [Tencent Cloud Object Storage (COS)](https://cloud.tencent.com/product/cos) as its underlying storage system. - -## Prerequisites - -In preparation for using COS with Alluxio, create a new bucket or use an existing bucket. -You should also note the directory you want to use in that bucket, either by creating a new directory in the bucket or using an existing one. - -{% navtabs Prerequisites %} -{% navtab COS %} - -For the purposes of this guide, the COS bucket name is called -`COS_BUCKET`, and the directory in that bucket is called `COS_DIRECTORY`. - -You also need to provide APPID and REGION. In this guide, the APPID is called `COS_APP_ID`, and the REGION is called `COS_REGION`. For more information, please refer [here](https://cloud.tencent.com/document/product/436/7751). - -{% endnavtab %} -{% navtab COSN %} - -For the purposes of this guide, the COSN Bucket name is called `COSN_BUCKET`, the directory in that bucket is called `COSN_DIRECTORY`, and COSN Bucket region is called `COSN_REGION` which specifies the region of your bucket. - -{% endnavtab %} -{% endnavtabs %} - -## Basic Setup - -Create `conf/alluxio-site.properties` and `conf/core-site.xml` if they do not exist. - -```shell -$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties -$ cp conf/core-site.xml.template conf/core-site.xml -``` - -{% navtabs Setup %} -{% navtab COS %} - -Configure Alluxio to use COS as its under storage system by modifying `conf/alluxio-site.properties`. -Specify an **existing** COS bucket and directory as the under storage system by modifying -`conf/alluxio-site.properties` to include: - -```properties -alluxio.dora.client.ufs.root=cos://COS_BUCKET/COS_DIRECTORY/ -``` - -Note that if you want to mount the whole cos bucket, add a trailing slash after the bucket name -(e.g. `cos://COS_BUCKET/`). - -Specify the COS credentials for COS access by setting `fs.cos.access.key` and `fs.cos.secret.key` in -`alluxio-site.properties`. - -```properties -fs.cos.access.key= -fs.cos.secret.key= -``` - -Specify the COS region by setting `fs.cos.region` in `alluxio-site.properties` (e.g. ap-beijing) and `fs.cos.app.id`. - -```properties -fs.cos.region= -fs.cos.app.id= -``` - -{% endnavtab %} -{% navtab COSN %} - -Configure Alluxio to use COSN as its under storage system by modifying `conf/alluxio-site.properties` and `conf/core-site.xml`. -Specify an existing COS bucket and directory as the under storage system by modifying -`conf/alluxio-site.properties` to include: - -```properties -alluxio.dora.client.ufs.root=cosn://COSN_BUCKET/COSN_DIRECTORY/ -``` - -Specify COS configuration information in order to access COS by modifying `conf/core-site.xml` to include: - -```xml - - fs.cosn.impl - org.apache.hadoop.fs.CosFileSystem - - - fs.AbstractFileSystem.cosn.impl - org.apache.hadoop.fs.CosN - - - fs.cosn.userinfo.secretKey - xxxx - - - fs.cosn.userinfo.secretId - xxxx - - - fs.cosn.bucket.region - xx - -``` - -The above is the most basic configuration. For more configuration please refer to [here](https://hadoop.apache.org/docs/r3.3.1/hadoop-cos/cloud-storage/index.html). - -{% endnavtab %} -{% endnavtabs %} - -After these changes, Alluxio should be configured to work with COS or COSN as its under storage system. - -## Running Alluxio Locally with COS - -Start the Alluxio servers: - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -This will start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Before running an example program, please make sure the root mount point -set in the `conf/alluxio-site.properties` is a valid path in the ufs. -Make sure the user running the example program has write permissions to the alluxio file system. - -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` - -{% navtabs Test %} -{% navtab COS %} - -Visit your COS directory at `COS_BUCKET/COS_DIRECTORY` to verify the files and directories created by Alluxio exist. -For this test, you should see files named like: - -``` -COS_BUCKET/COS_DIRECTORY/default_tests_files/BASIC_CACHE_THROUGH -``` - -{% endnavtab %} -{% navtab COSN %} - -Visit your COSN directory at `COSN_BUCKET/COSN_DIRECTORY` to verify the files and directories created by Alluxio exist. -For this test, you should see files named like: - -``` -COSN_BUCKET/COSN_DIRECTORY/default_tests_files/BASIC_CACHE_THROUGH -``` - -{% endnavtab %} -{% endnavtabs %} - -To stop Alluxio, you can run: -```shell -$ ./bin/alluxio-stop.sh local -``` - -## Advanced Setup - -### [Experimental] COS multipart upload - -The default upload method uploads one file completely from start to end in one go. We use multipart-upload method to upload one file by multiple parts, every part will be uploaded in one thread. It won't generate any temporary files while uploading. - -To enable COS multipart upload, you need to modify `conf/alluxio-site.properties` to include: - -```properties -alluxio.underfs.cos.multipart.upload.enabled=true -``` - -There are other parameters you can specify in `conf/alluxio-site.properties` to make the process faster and better. - -```properties -# Timeout for uploading part when using multipart upload. -alluxio.underfs.object.store.multipart.upload.timeout -``` -```properties -# Thread pool size for COS multipart upload. -alluxio.underfs.cos.multipart.upload.threads -``` -```properties -# Multipart upload partition size for COS. The default partition size is 64MB. -alluxio.underfs.cos.multipart.upload.partition.size -``` - - - diff --git a/docs/en/ufs/CephFS.md b/docs/en/ufs/CephFS.md index 8555d0573f02..3a9c3eb03c82 100644 --- a/docs/en/ufs/CephFS.md +++ b/docs/en/ufs/CephFS.md @@ -3,18 +3,44 @@ layout: global title: CephFS --- -This guide describes how to configure Alluxio with [CephFS](https://docs.ceph.com/en/latest/cephfs/) as the under storage system. +This guide describes how to configure Alluxio with [CephFS](https://ceph.io/en/discover/technology/#file){:target="_blank"} as the under storage system. The Ceph File System (CephFS) is a POSIX-compliant file system built on top of Ceph’s distributed object store, RADOS. CephFS endeavors to provide a state-of-the-art, multi-use, highly available, and performant file store for a variety of applications, including traditional use-cases like shared home directories, HPC scratch space, and distributed workflow shared storage. -Alluxio supports two different implementations of under storage system for CephFS: -- [cephfs](https://docs.ceph.com/en/latest/cephfs/api/libcephfs-java/) -- [cephfs-hadoop](https://docs.ceph.com/en/nautilus/cephfs/hadoop/) +Alluxio supports two different implementations of under storage system for CephFS. Fore more information, please read its documentation: +- [cephfs](https://docs.ceph.com/en/latest/cephfs/api/libcephfs-java/){:target="_blank"} +- [cephfs-hadoop](https://docs.ceph.com/en/nautilus/cephfs/hadoop/){:target="_blank"} ## Prerequisites +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using CephFS with Alluxio: + + + + + + + + + + + + + + + + + + + + + +
``Local path to Ceph configuration file ceph.conf
``Ceph URI that is used to identify dameon instances in the ceph.conf
``The directory you want to use, either by creating a new directory or using an existing one
``Ceph user id
``Ceph keyring file that stores one or more Ceph authentication keys
+ ### Install Dependencies -According to [ceph packages install](https://docs.ceph.com/en/latest/install/get-packages/) to install below packages: +Follow [Ceph packages install](https://docs.ceph.com/en/latest/install/get-packages/){:target="_blank"} to install below packages: ``` cephfs-java @@ -39,9 +65,7 @@ $ curl -o $java_path/jre/lib/ext/hadoop-cephfs.jar -s https://download.ceph.com/ ## Basic Setup -Configure Alluxio to use under storage systems by modifying -`conf/alluxio-site.properties` and `conf/core-site.xml`. If they do not exist, -create the configuration files from the templates +To use CephFS as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. ```shell $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties @@ -54,11 +78,11 @@ $ cp conf/core-site.xml.template conf/core-site.xml Modify `conf/alluxio-site.properties` to include: ```properties -alluxio.underfs.cephfs.conf.file= -alluxio.underfs.cephfs.mds.namespace= -alluxio.underfs.cephfs.mount.point= -alluxio.underfs.cephfs.auth.id= -alluxio.underfs.cephfs.auth.keyring= +alluxio.underfs.cephfs.conf.file= +alluxio.underfs.cephfs.mds.namespace= +alluxio.underfs.cephfs.mount.point= +alluxio.underfs.cephfs.auth.id= +alluxio.underfs.cephfs.auth.keyring= ``` {% endnavtab %} @@ -122,22 +146,11 @@ Modify `conf/core-site.xml` to include: ## Running Alluxio Locally with CephFS -Start up Alluxio locally to see that everything works. - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -This should start an Alluxio master and Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). +Once you have configured Alluxio to CephFS, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. {% navtabs Test %} {% navtab cephfs %} -An CephFS location can be mounted at a nested directory in the Alluxio namespace to have unified access -to multiple under storage systems. Alluxio's [Command Line Interface]({{ '/en/operation/User-CLI.html' | relativize_url }}) can be used for this purpose. - Issue the following command to use the ufs cephfs: ```shell @@ -151,14 +164,13 @@ Run a simple example program: $ ./bin/alluxio runTests --path cephfs://mon1\;mon2\;mon3/ ``` -Visit your cephfs to verify the files and directories created by Alluxio exist. - -You should see files named like: -In cephfs, you can visit cephfs with ceph-fuse or mount by POSIX APIs. [Mounting CephFS](https://docs.ceph.com/en/latest/cephfs/#mounting-cephfs) +Visit your cephfs to verify the files and directories created by Alluxio exist. You should see files named like: ``` -${ceph-fs-dir}/default_tests_files/Basic_CACHE_THROUGH +${cephfs-dir}/default_tests_files/Basic_CACHE_THROUGH ``` +In cephfs, you can visit cephfs with ceph-fuse or mount by POSIX APIs. [Mounting CephFS](https://docs.ceph.com/en/latest/cephfs/#mounting-cephfs){:target="_blank"} + In Alluxio, you can visit the nested directory in the Alluxio. Alluxio's [Command Line Interface]({{ '/en/operation/User-CLI.html' | relativize_url }}) can be used for this purpose. ``` @@ -168,14 +180,11 @@ In Alluxio, you can visit the nested directory in the Alluxio. Alluxio's [Comman {% endnavtab %} {% navtab cephfs-hadoop %} -An CephFS location can be mounted at a nested directory in the Alluxio namespace to have unified access -to multiple under storage systems. Alluxio's [Command Line Interface]({{ '/en/operation/User-CLI.html' | relativize_url }}) can be used for this purpose. - -Issue the following command to use the ufs cephfs: +Issue the following command to use the ufs cephfs-hadoop: ```shell $ ./bin/alluxio fs mkdir /mnt/cephfs-hadoop -$ ./bin/alluxio fs mount /mnt/cephfs-hadoop ceph://mon1\;mon2\;mon3/ +$ ./bin/alluxio fs mount /mnt/cephfs-hadoop cephfs://mon1\;mon2\;mon3/ ``` Run a simple example program: @@ -184,14 +193,13 @@ Run a simple example program: $ ./bin/alluxio runTests --path cephfs://mon1\;mon2\;mon3/ ``` -Visit your cephfs to verify the files and directories created by Alluxio exist. - -You should see files named like: -In cephfs, you can visit cephfs with ceph-fuse or mount by POSIX APIs. [Mounting CephFS](https://docs.ceph.com/en/latest/cephfs/#mounting-cephfs) +Visit your cephfs-hadoop to verify the files and directories created by Alluxio exist. You should see files named like: ``` -${ceph-fs-dir}/default_tests_files/Basic_CACHE_THROUGH +${cephfs-hadoop-dir}/default_tests_files/Basic_CACHE_THROUGH ``` +In cephfs, you can visit cephfs with ceph-fuse or mount by POSIX APIs. [Mounting CephFS](https://docs.ceph.com/en/latest/cephfs/#mounting-cephfs){:target="_blank"} + In Alluxio, you can visit the nested directory in the Alluxio. Alluxio's [Command Line Interface]({{ '/en/operation/User-CLI.html' | relativize_url }}) can be used for this purpose. ``` diff --git a/docs/en/ufs/CephObjectStorage.md b/docs/en/ufs/CephObjectStorage.md index b80b75e82c5c..f40f60c702d1 100644 --- a/docs/en/ufs/CephObjectStorage.md +++ b/docs/en/ufs/CephObjectStorage.md @@ -4,88 +4,141 @@ title: CephObjectStorage --- -This guide describes how to configure Alluxio with [Ceph Object Storage](http://ceph.com/ceph-storage/object-storage/) as the under storage system. +This guide describes how to configure Alluxio with [Ceph Object Storage](https://ceph.com/en/discover/technology/#object){:target="_blank"} as the under storage system. Ceph Object Storage is a distributed, open-source storage system designed for storing and retrieving large amounts of unstructured data. It provides a scalable and highly available storage solution that can be deployed on commodity hardware. -Alluxio supports two different clients APIs to connect to Ceph Object Storage using [Rados Gateway](http://docs.ceph.com/docs/master/radosgw/): -- [S3](http://docs.aws.amazon.com/AmazonS3/latest/API/Welcome.html) (preferred) -- [Swift](http://docs.openstack.org/developer/swift/) +Alluxio supports two different clients APIs to connect to Ceph Object Storage using [Rados Gateway](http://docs.ceph.com/docs/master/radosgw/){:target="_blank"}. For more information, please read its documentation: +- [S3](https://docs.ceph.com/en/latest/radosgw/s3/){:target="_blank"} (preferred) +- [Swift](https://docs.ceph.com/en/latest/radosgw/swift/){:target="_blank"} + +## Prerequisites + +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using Ceph Object Storage with Alluxio: +{% navtabs Prerequisites %} +{% navtab S3 %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
``[Create a new S3 bucket](https://docs.ceph.com/en/quincy/radosgw/s3/bucketops/){:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``Used to sign programmatic requests made to AWS. See [How to Obtain Access Key ID and Secret Access Key](https://docs.aws.amazon.com/powershell/latest/userguide/pstools-appendix-sign-up.html){:target="_blank"}
``Used to sign programmatic requests made to AWS. See [How to Obtain Access Key ID and Secret Access Key](https://docs.aws.amazon.com/powershell/latest/userguide/pstools-appendix-sign-up.html){:target="_blank"}
``The host for the Ceph Object Gateway instance, which can be an IP address or a hostname. Read [more](https://docs.ceph.com/en/mimic/radosgw/config-ref/){:target="_blank"}
``The port the instance listens for requests and if not specified, Ceph Object Gateway runs external FastCGI. Read [more](https://docs.ceph.com/en/mimic/radosgw/config-ref/){:target="_blank"}
``
+ +{% endnavtab %} +{% navtab Swift %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
``[Create a new Swift bucket](https://docs.ceph.com/en/quincy/radosgw/swift/containerops/){:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``
``
``
``The host for the Ceph Object Gateway instance, which can be an IP address or a hostname. Read [more](https://docs.ceph.com/en/mimic/radosgw/config-ref/){:target="_blank"}
``The port the instance listens for requests and if not specified, Ceph Object Gateway runs external FastCGI. Read [more](https://docs.ceph.com/en/mimic/radosgw/config-ref/){:target="_blank"}
``
+ +{% endnavtab %} +{% endnavtabs %} ## Basic Setup -A Ceph bucket can be mounted to Alluxio either at the root of the namespace, or at a nested directory. - -Configure Alluxio to use under storage systems by modifying -`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the -template. +To use Ceph Object Storage as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. ```shell $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties ``` -### Option 1: S3 Interface (preferred) +{% navtabs Setup %} +{% navtab S3 %} Modify `conf/alluxio-site.properties` to include: ```properties -alluxio.dora.client.ufs.root=s3:/// -s3a.accessKeyId= -s3a.secretKey= -alluxio.underfs.s3.endpoint=http://: +alluxio.dora.client.ufs.root=s3:/// +s3a.accessKeyId= +s3a.secretKey= +alluxio.underfs.s3.endpoint=http://: alluxio.underfs.s3.disable.dns.buckets=true -alluxio.underfs.s3.inherit.acl= +alluxio.underfs.s3.inherit.acl= ``` If using a Ceph release such as hammer (or older) specify `alluxio.underfs.s3.signer.algorithm=S3SignerType` to use v2 S3 signatures. To use GET Bucket (List Objects) Version 1 specify `alluxio.underfs.s3.list.objects.v1=true`. -### Option 2: Swift Interface +{% endnavtab %} +{% navtab Swift %} + Modify `conf/alluxio-site.properties` to include: ```properties -alluxio.dora.client.ufs.root=swift:/// -fs.swift.user= -fs.swift.tenant= -fs.swift.password= -fs.swift.auth.url= -fs.swift.auth.method= -``` -Replace `/` with an existing Swift container location. Possible values of `` are -`true`, `false`. Specify `` as `swiftauth` if using native Ceph RGW authentication and `` -as `http://:/auth/1.0`. - -## Running Alluxio Locally with Ceph - -Start up Alluxio locally to see that everything works. - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local +alluxio.dora.client.ufs.root=swift:/// +fs.swift.user= +fs.swift.tenant= +fs.swift.password= +fs.swift.auth.url=http://:/auth/1.0 +fs.swift.auth.method= ``` +Replace `/` with an existing Swift bucket location. -This should start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). +{% endnavtab %} +{% endnavtabs %} -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` - -Visit your bucket to verify the files and directories created by Alluxio exist. - -You should see files named like: -``` -//default_tests_files/Basic_CACHE_THROUGH -``` - -To stop Alluxio, run: +## Running Alluxio Locally with Ceph -```shell -$ ./bin/alluxio-stop.sh local -``` +Once you have configured Alluxio to Ceph Object Storage, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. ## Advanced Setup @@ -93,5 +146,5 @@ $ ./bin/alluxio-stop.sh local If Alluxio security is enabled, Alluxio enforces the access control inherited from underlying Ceph Object Storage. Depending on the interace used, refer to -[S3 Access Control]({{ '/en/ufs/S3.html' | relativize_url }}#s3-access-control) or +[S3 Access Control]({{ '/en/ufs/S3.html' | relativize_url }}#identity-and-access-control-of-s3-objects) or [Swift Access Control]({{ '/en/ufs/Swift.html' | relativize_url }}#swift-access-control) for more information. diff --git a/docs/en/ufs/GCS.md b/docs/en/ufs/GCS.md index f24e412f1a67..d543d6d58107 100644 --- a/docs/en/ufs/GCS.md +++ b/docs/en/ufs/GCS.md @@ -4,50 +4,52 @@ title: Google Cloud Storage --- -This guide describes how to configure Alluxio with [Google Cloud Storage (GCS)](https://cloud.google.com/storage/) +This guide describes how to configure Alluxio with [Google Cloud Storage (GCS)](https://cloud.google.com/storage/){:target="_blank"} as the under storage system. Google Cloud Storage (GCS) is a scalable and durable object storage service offered by Google Cloud Platform (GCP). It allows users to store and retrieve various types of data, including unstructured and structured data. -## Prerequisites +For more information about GCS, please read its [documentation](https://cloud.google.com/storage/docs){:target="_blank"}. -In preparation for using GCS with Alluxio, create a bucket (or use an existing bucket). You -should also note the directory you want to use in that bucket, either by creating a new directory in -the bucket, or using an existing one. For the purposes of this guide, the GCS bucket name is called -`GCS_BUCKET`, and the directory in that bucket is called `GCS_DIRECTORY`. +## Prerequisites -For more information on GCS, please read its -[documentation](https://cloud.google.com/storage/docs/overview). +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. -## Setup +In preparation for using GCS with Alluxio: + + + + + + + + + +
``[Create a new bucket in your Google Cloud account](https://cloud.google.com/storage/docs/creating-buckets){:target="_blank"}{:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
Alluxio provides two ways to access GCS. GCS version 1 is implemented based on -[jets3t](http://www.jets3t.org/) library which is design for AWS S3. +[jets3t](http://www.jets3t.org/){:target="_blank"} library which is design for AWS S3. Thus, it only accepts Google cloud storage interoperability access/secret keypair which allows full access to all Google cloud storages inside a Google cloud project. No permission or access control can be placed on the interoperability keys. The conjunction of Google interoperability API and jets3t library also impact the performance of the default GCS UFS module. The default GCS UFS module (GCS version 2) is implemented based on Google Cloud API -which accepts [Google application credentials](https://cloud.google.com/docs/authentication/getting-started). +which accepts [Google application credentials](https://cloud.google.com/docs/authentication/getting-started){:target="_blank"}. Based on the application credentials, Google cloud can determine what permissions an authenticated client has for its target Google cloud storage bucket. Besides, GCS with Google cloud API has much better performance than the default one in metadata and read/write operations. ## Basic Setup -A GCS bucket can be mounted to the Alluxio either at the root of the namespace, or at a nested directory. - -Configure Alluxio to use under storage systems by modifying -`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the -template. +To use Google Cloud Storage as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying +`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. ```shell $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties ``` -Configure Alluxio to use GCS as its root under storage system. The first modification is to -specify an **existing** GCS bucket and directory as the under storage system by modifying +Specify an **existing** GCS bucket and directory as the underfs address by modifying `conf/alluxio-site.properties` to include: ```properties @@ -72,56 +74,26 @@ but it is always recommended to set this property explicitly. {% navtab GCS version 1 %} -In`conf/alluxio-site.properties`, add: +In `conf/alluxio-site.properties`, add: ```properties alluxio.underfs.gcs.version=1 fs.gcs.accessKeyId= fs.gcs.secretAccessKey= ``` -- The first property key tells Alluxio to load the Version 1 GCS UFS module which uses the [jets3t](http://www.jets3t.org/) library. +- The first property key tells Alluxio to load the Version 1 GCS UFS module which uses the [jets3t](http://www.jets3t.org/){:target="_blank"} library. - Replace `` and `` with actual -[GCS interoperable storage access keys](https://console.cloud.google.com/storage/settings), +[GCS interoperable storage access keys](https://console.cloud.google.com/storage/settings){:target="_blank"}, or other environment variables that contain your credentials. Note: GCS interoperability is disabled by default. Please click on the Interoperability tab -in [GCS setting](https://console.cloud.google.com/storage/settings) and enable this feature. +in [GCS setting](https://console.cloud.google.com/storage/settings){:target="_blank"} and enable this feature. Click on `Create a new key` to get the Access Key and Secret pair. {% endnavtab %} {% endnavtabs %} -After these changes, Alluxio should be configured to work with GCS as its under storage system, and -you can [Run Alluxio Locally with GCS](#running-alluxio-locally-with-gcs). - ## Running Alluxio Locally with GCS -Start up Alluxio locally to see that everything works. - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local SudoMount -``` - -This should start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` - -Visit your GCS directory `GCS_BUCKET/GCS_DIRECTORY` to verify the files -and directories created by Alluxio exist. For this test, you should see files named like: - -``` -GCS_BUCKET/GCS_DIRECTORY/default_tests_files/BASIC_CACHE_THROUGH -``` - -To stop Alluxio, you can run: - -```shell -$ ./bin/alluxio-stop.sh local -``` +Once you have configured Alluxio to GCS, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. ## Advanced Setup @@ -164,7 +136,7 @@ GCS buckets nor objects. By default, Alluxio tries to extract the GCS user id from the credentials. Optionally, `alluxio.underfs.gcs.owner.id.to.username.mapping` can be used to specify a preset gcs owner id to Alluxio username static mapping in the format `id1=user1;id2=user2`. The Google Cloud Storage IDs -can be found at the console [address](https://console.cloud.google.com/storage/settings). Please use +can be found at the console [address](https://console.cloud.google.com/storage/settings){:target="_blank"}. Please use the "Owners" one. ### Accessing GCS through Proxy (GCS Version 2 only) diff --git a/docs/en/ufs/HDFS.md b/docs/en/ufs/HDFS.md index 4e02517fc4cd..8b5f754ea066 100644 --- a/docs/en/ufs/HDFS.md +++ b/docs/en/ufs/HDFS.md @@ -4,10 +4,32 @@ title: HDFS --- -This guide describes the instructions to configure [HDFS](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsUserGuide.html) as Alluxio's under storage system. +This guide describes the instructions to configure [HDFS](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html){:target="_blank"} as Alluxio's under storage system. HDFS, or Hadoop Distributed File System, is the primary distributed storage used by Hadoop applications, providing reliable and scalable storage for big data processing in Hadoop ecosystems. +For more information about HDFS, please read its [documentation](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsUserGuide.html){:target="_blank"}. + +## Prerequisites + +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using HDFS with Alluxio: + + + + + + + + + + + + + +
``The IP address of the NameNode that processes client connections to the cluster. NameNode is the master node in the Apache Hadoop HDFS Architecture that maintains and manages the blocks present on the DataNodes (slave nodes).
``The port at which the NameNode accepts client connections.
``
+ ## Basic Setup To configure Alluxio to use HDFS as under storage, you will need to modify the configuration @@ -18,41 +40,28 @@ If the file does not exist, create the configuration file from the template. $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties ``` -Edit `conf/alluxio-site.properties` file to set the under storage address to the HDFS namenode -address and the HDFS directory you want to mount to Alluxio. For example, the under storage address -can be `hdfs://localhost:8020` if you are running the HDFS namenode locally with default port and +Specify the HDFS namenode and the HDFS port as the underfs address by modifying `conf/alluxio-site.properties`. + +For example, the under storage address can be `hdfs://localhost:8020` if you are running the HDFS namenode locally with default port and mapping HDFS root directory to Alluxio, or `hdfs://localhost:8020/alluxio/data` if only the HDFS directory `/alluxio/data` is mapped to Alluxio. -To find out where HDFS is running, use `hdfs getconf -confKey fs.defaultFS` to get the default hostname -and port HDFS is listening on. ```properties -alluxio.dora.client.ufs.root=hdfs://: +alluxio.dora.client.ufs.root=hdfs://: ``` +To find out where HDFS is running, use `hdfs getconf -confKey fs.defaultFS` to get the default hostname +and port HDFS is listening on. + Additionally, you may need to specify the following property to be your HDFS version. See [mounting HDFS with specific versions]({{ '/en/ufs/HDFS.html' | relativize_url }}#mount-hdfs-with-specific-versions). ```properties -alluxio.underfs.version= +alluxio.underfs.version= ``` ## Running Alluxio Locally with HDFS -Before this step, make sure your HDFS cluster is running and the directory mapped to Alluxio -exists. Start the Alluxio servers: - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh master -$ ./bin/alluxio-start.sh worker -``` - -Stop Alluxio by running: - -```shell -$ ./bin/alluxio-stop.sh master -$ ./bin/alluxio-stop.sh worker -``` +Once you have configured Alluxio to HDFS, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. ## Advanced Setup @@ -83,7 +92,7 @@ alluxio.underfs.hdfs.configuration=/path/to/hdfs/conf/core-site.xml:/path/to/hdf To configure Alluxio to work with HDFS namenodes in HA mode, first configure Alluxio servers to [access HDFS with the proper configuration files](#specify-hdfs-configuration-location). In addition, set the under storage address to `hdfs://nameservice/` (`nameservice` is -the [HDFS nameservice](https://hadoop.apache.org/docs/r3.3.1/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithQJM.html#Configuration_details) +the [HDFS nameservice](https://hadoop.apache.org/docs/r3.3.1/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithQJM.html#Configuration_details){:target="_blank"} already configured in `hdfs-site.xml`). To mount an HDFS subdirectory to Alluxio instead of the whole HDFS namespace, change the under storage address to something like `hdfs://nameservice/alluxio/data`. @@ -99,11 +108,11 @@ HDFS is consistent with Alluxio (e.g., a file created by user Foo in Alluxio is HDFS also with owner as user Foo), the user to start Alluxio master and worker processes **is required** to be either: -1. [HDFS super user](http://hadoop.apache.org/docs/r3.3.1/hadoop-project-dist/hadoop-hdfs/HdfsPermissionsGuide.html#The_Super-User). +1. [HDFS super user](http://hadoop.apache.org/docs/r3.3.1/hadoop-project-dist/hadoop-hdfs/HdfsPermissionsGuide.html#The_Super-User){:target="_blank"}. Namely, use the same user that starts HDFS namenode process to also start Alluxio master and worker processes. -2. A member of [HDFS superuser group](http://hadoop.apache.org/docs/r3.3.1/hadoop-project-dist/hadoop-hdfs/HdfsPermissionsGuide.html#Configuration_Parameters). +2. A member of [HDFS superuser group](http://hadoop.apache.org/docs/r3.3.1/hadoop-project-dist/hadoop-hdfs/HdfsPermissionsGuide.html#Configuration_Parameters){:target="_blank"}. Edit HDFS configuration file `hdfs-site.xml` and check the value of configuration property `dfs.permissions.superusergroup`. If this property is set with a group (e.g., "hdfs"), add the user to start Alluxio process (e.g., "alluxio") to this group ("hdfs"); if this property is not @@ -190,7 +199,7 @@ To build this module yourself, build the shaded hadoop client and then the UFS m Hadoop comes with a native library that provides better performance and additional features compared to its Java implementation. For example, when the native library is used, the HDFS client can use native checksum function which is more efficient than the default Java implementation. To use the Hadoop native library with Alluxio HDFS under filesystem, first install the native library on Alluxio nodes by following the -instructions on [this page](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/NativeLibraries.html). +instructions on [this page](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/NativeLibraries.html){:target="_blank"}. Once the hadoop native library is installed on the machine, update Alluxio startup Java parameters in `conf/alluxio-env.sh` by adding the following line: ```sh diff --git a/docs/en/ufs/Huawei-OBS.md b/docs/en/ufs/Huawei-OBS.md new file mode 100644 index 000000000000..bfec1681460a --- /dev/null +++ b/docs/en/ufs/Huawei-OBS.md @@ -0,0 +1,106 @@ +--- +layout: global +title: Huawei Object Storage Service +--- + + +This guide describes the instructions to configure [Huawei OBS](https://www.huaweicloud.com/product/obs){:target="_blank"} as Alluxio's +under storage system. + +Huawei Object Storage Service (OBS) is a scalable service that provides secure, reliable, and cost-effective cloud storage for massive amounts of data. OBS provides unlimited storage capacity for objects of any format, catering to the needs of common users, websites, enterprises, and developers. + +For more information about Huawei OBS, please read its [documentation](https://support.huaweicloud.com/intl/en-us/obs/index.html){:target="_blank"}. + +## Prerequisites + +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using Huawei OBS with Alluxio: + + + + + + + + + + + + + + + + + + + + + +
``[Create a new bucket](https://support.huaweicloud.com/intl/en-us/qs-obs/obs_qs_0007.html){:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``Used to authenticate the identity of a requester. See [Obtaining Access Keys (AK and SK)](https://support.huaweicloud.com/intl/en-us/qs-obs/obs_qs_0005.html){:target="_blank"}
``Used to authenticate the identity of a requester. See [Obtaining Access Keys (AK and SK)](https://support.huaweicloud.com/intl/en-us/qs-obs/obs_qs_0005.html){:target="_blank"}
``Domain name to access OBS in a region and is used to process requests of that region. See [Regions and Endpoints](https://developer.huaweicloud.com/intl/en-us/endpoint){:target="_blank"}
+ +## Basic Setup + +To use Huawei OBS as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. + +```shell +$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties +``` + +Specify an **existing** OBS bucket and directory as the underfs addresss system by modifying +`conf/alluxio-site.properties` to include: + +```properties +alluxio.dora.client.ufs.root=obs:/// +``` + +Note that if you want to mount the whole obs bucket, add a trailing slash after the bucket name +(e.g. `obs://OBS_BUCKET/`). + +Specify credentials for OBS access by setting `fs.obs.accessKey` and `fs.obs.secretKey` in +`alluxio-site.properties`. + +```properties +fs.obs.accessKey= +fs.obs.secretKey= +``` + +Specify the OBS region by setting `fs.obs.endpoint` in `alluxio-site.properties` (e.g. obs.cn-north-4.myhuaweicloud.com). + +```properties +fs.obs.endpoint= +``` + +## Running Alluxio Locally with OBS + +Once you have configured Alluxio to Azure Blob Store, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. + +## Advanced Setup + +### [Experimental] OBS multipart upload + +The default upload method uploads one file completely from start to end in one go. We use multipart-upload method to upload one file by multiple parts, every part will be uploaded in one thread. It won't generate any temporary files while uploading. + +To enable OBS multipart upload, you need to modify `conf/alluxio-site.properties` to include: + +```properties +alluxio.underfs.obs.multipart.upload.enabled=true +``` + +There are other parameters you can specify in `conf/alluxio-site.properties` to make the process faster and better. + +```properties +# Timeout for uploading part when using multipart upload. +alluxio.underfs.object.store.multipart.upload.timeout +``` +```properties +# Thread pool size for OBS multipart upload. +alluxio.underfs.obs.multipart.upload.threads +``` +```properties +# Multipart upload partition size for OBS. The default partition size is 64MB. +alluxio.underfs.obs.multipart.upload.partition.size +``` + + + diff --git a/docs/en/ufs/Minio.md b/docs/en/ufs/Minio.md index 450c8c8a5832..fd47a83cf871 100644 --- a/docs/en/ufs/Minio.md +++ b/docs/en/ufs/Minio.md @@ -4,45 +4,65 @@ title: MinIO --- -This guide describes how to configure Alluxio with [MinIO](https://min.io/) as the +This guide describes how to configure Alluxio with [MinIO](https://min.io/){:target="_blank"} as the under storage system. -MinIO is a high-performance, S3 compatible object store. It is built for large scale AI/ML, data lake and database workloads. It runs on-prem and on any cloud (public or private) and from the data center to the edge. +MinIO is a high-performance, S3 compatible object store. It is built for large scale AI/ML, data lake and database workloads. It runs on-prem and on any cloud (public or private) and from the data center to the edge. MinIO provides an open source alternative to AWS S3. Alluxio natively provides the `s3://` scheme (recommended for better performance). You can use this scheme to connect Alluxio with a MinIO server. -## Setup MinIO +For more information about MinIO, please read its [documentation](https://min.io/docs/minio/linux/index.html){:target="_blank"} -MinIO is an object storage server built for cloud applications and DevOps. MinIO provides an open -source alternative to AWS S3. +## Prerequisites -Launch a MinIO server instance using the steps mentioned -[here](http://docs.min.io/docs/minio-quickstart-guide). -Then, either create a new bucket or use an existing one. -Once the MinIO server is launched, keep a note of the server endpoint, accessKey and secretKey. +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. -You should also note the directory you want to use in that bucket, either by creating -a new directory in the bucket or using an existing one. -For the purposes of this guide, the MinIO bucket name is called `MINIO_BUCKET`, and the directory in -that bucket is called `MINIO_DIRECTORY`. +Launch a MinIO server instance using the steps mentioned +[here](https://min.io/docs/minio/linux/index.html){:target="_blank"}. Once the MinIO server is launched, keep a note of the server endpoint, accessKey and secretKey. + +In preparation for using MinIO with Alluxio: + + + + + + + + + + + + + + + + + + + + + +
``[Create a new bucket](https://min.io/docs/minio/linux/reference/minio-mc/mc-mb.html#examples){:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``
``Used to sign programmatic requests made to AWS. See [How to Obtain Access Key ID and Secret Access Key](https://docs.aws.amazon.com/powershell/latest/userguide/pstools-appendix-sign-up.html){:target="_blank"}
``Used to sign programmatic requests made to AWS. See [How to Obtain Access Key ID and Secret Access Key](https://docs.aws.amazon.com/powershell/latest/userguide/pstools-appendix-sign-up.html){:target="_blank"}
+ +## Basic Setup + +To use MinIO as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. -## Configuring Alluxio +```shell +$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties +``` -You need to configure Alluxio to use MinIO as its under storage system by modifying -`conf/alluxio-site.properties`. The first modification is to specify an **existing** MinIO -bucket and directory as the under storage system. -Because Minio supports the `s3` protocol, it is possible to configure Alluxio as if it were -pointing to an AWS S3 endpoint. +Specify an **existing** MinIO bucket and directory as the underfs address. Because Minio supports the `s3` protocol, it is possible to configure Alluxio as if it were pointing to an AWS S3 endpoint. -All the fields to be modified in `conf/alluxio-site.properties` file are listed here: +Modify `conf/alluxio-site.properties` to include: ```properties alluxio.dora.client.ufs.root=s3:/// alluxio.underfs.s3.endpoint=http:/// alluxio.underfs.s3.disable.dns.buckets=true alluxio.underfs.s3.inherit.acl=false -s3a.accessKeyId= -s3a.secretKey= +s3a.accessKeyId= +s3a.secretKey= ``` For these parameters, replace `` with the hostname and port of your MinIO service, @@ -51,23 +71,7 @@ If the port value is left unset, it defaults to port 80 for `http` and 443 for ` ## Running Alluxio Locally with MinIO -Format and start Alluxio with - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -Verify Alluxio is running by navigating to [http://localhost:19999](http://localhost:19999) or by -examining the logs to ensure the process is running. - -Then, to run tests using some basic Alluxio operations execute the following command: - -```shell -$ ./bin/alluxio runTests -``` - -If there are no errors then MinIO is configured properly! +Once you have configured Alluxio to MinIO, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. ## Troubleshooting diff --git a/docs/en/ufs/NFS.md b/docs/en/ufs/NFS.md index 5d420f4517f0..de67008ee4b4 100644 --- a/docs/en/ufs/NFS.md +++ b/docs/en/ufs/NFS.md @@ -4,25 +4,25 @@ title: NFS --- -This guide describes the instructions to configure [NFS](http://nfs.sourceforge.net) as Alluxio's under +This guide describes the instructions to configure [NFS](http://nfs.sourceforge.net){:target="_blank"} as Alluxio's under storage system. Network File System (NFS) is a distributed file system protocol that allows a client computer to access files over a network as if they were located on its local storage. NFS enables file sharing and remote file access between systems in a networked environment. You'll need to have a configured and running installation of NFS for the rest of this guide. If you need to get your own NFS installation up and running, we recommend taking a look at the -[NFS-HOW TO](http://nfs.sourceforge.net/nfs-howto/) +[NFS-HOW TO](http://nfs.sourceforge.net/nfs-howto/){:target="_blank"}. ## Prerequisites -The prerequisite for this part is that you have a version of -[Java 8](https://adoptopenjdk.net/releases.html?variant=openjdk8&jvmVariant=hotspot) -installed. +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +Make sure that you have a version of +[Java 8](https://adoptopenjdk.net/releases.html?variant=openjdk8&jvmVariant=hotspot){:target="_blank"} installed. Turn on remote login service so that `ssh localhost` can succeed. To avoid the need to repeatedly input the password, you can add the public SSH key for the host into -`~/.ssh/authorized_keys`. See [this tutorial](http://www.linuxproblem.org/art_9.html) for more -details. +`~/.ssh/authorized_keys`. See [this tutorial](http://www.linuxproblem.org/art_9.html){:target="_blank"} for more details. ## Creating NFS mount point @@ -36,17 +36,15 @@ Please mount your nfs share like this. $ sudo mount -o actimeo=0 nfshost:/nfsdir /mnt/nfs ``` -## Configuring Alluxio +## Basic Setup -Configure Alluxio to use under storage systems by modifying -`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the -template. +To use NFS as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. ```shell $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties ``` -Assume we have mounted NFS share at `/mnt/nfs` on all Alluxio masters and workers, the following lines should be exist within the `conf/alluxio-site.properties` file. +Assume we have mounted NFS share at `/mnt/nfs` on all Alluxio masters and workers, modify `conf/alluxio-site.properties` to include: ```properties alluxio.master.hostname=localhost @@ -55,22 +53,7 @@ alluxio.dora.client.ufs.root=/mnt/nfs ## Running Alluxio with NFS -Run the following command to start Alluxio filesystem. - -```shell -$ ./bin/alluxio-mount.sh SudoMount -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -To verify that Alluxio is running, you can visit -[http://localhost:19999](http://localhost:19999), or see the log in the `logs` folder. - -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` +Once you have configured Alluxio to NFS, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. Visit your NFS volume at `/mnt/nfs` to verify the files and directories created by Alluxio exist. For this test, you should see files named: @@ -78,9 +61,3 @@ For this test, you should see files named: ``` /mnt/nfs/default_tests_files/BASIC_CACHE_THROUGH ``` - -Stop Alluxio by running: - -```shell -$ ./bin/alluxio-stop.sh local -``` diff --git a/docs/en/ufs/OBS.md b/docs/en/ufs/OBS.md deleted file mode 100644 index 01f4f58edc32..000000000000 --- a/docs/en/ufs/OBS.md +++ /dev/null @@ -1,110 +0,0 @@ ---- -layout: global -title: Huawei OBS ---- - - -This guide describes the instructions to configure [Huawei OBS](https://www.huaweicloud.com/product/obs) as Alluxio's -under storage system. - -Huawei Object Storage Service (OBS) is a scalable service that provides secure, reliable, and cost-effective cloud storage for massive amounts of data. OBS provides unlimited storage capacity for objects of any format, catering to the needs of common users, websites, enterprises, and developers. - -## Prerequisites - -In preparation for using OBS with Alluxio, create a new bucket or use an existing bucket. You -should also note the directory you want to use in that bucket, either by creating a new directory in -the bucket, or using an existing one. For the purposes of this guide, the OBS bucket name is called -`OBS_BUCKET`, and the directory in that bucket is called `OBS_DIRECTORY`. - -## Basic Setup - -Create `conf/alluxio-site.properties` if it does not exist. - -```shell -$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties -``` - -Configure Alluxio to use OBS as its under storage system by modifying `conf/alluxio-site.properties`. -Specify an **existing** OBS bucket and directory as the under storage system by modifying -`conf/alluxio-site.properties` to include: - -```properties -alluxio.dora.client.ufs.root=obs:/// -``` - -Note that if you want to mount the whole obs bucket, add a trailing slash after the bucket name -(e.g. `obs://OBS_BUCKET/`). - -Specify the OBS credentials for OBS access by setting `fs.obs.accessKey` and `fs.obs.secretKey` in -`alluxio-site.properties`. - -```properties -fs.obs.accessKey= -fs.obs.secretKey= -``` - -Specify the OBS region by setting `fs.obs.endpoint` in `alluxio-site.properties` (e.g. obs.cn-north-4.myhuaweicloud.com). - -```properties -fs.obs.endpoint= -``` - -After these changes, Alluxio should be configured to work with OBS as its under storage system, and -you can try [Running Alluxio Locally with OBS](#running-alluxio-locally-with-obs). - - -## Running Alluxio Locally with OBS - -Start the Alluxio servers: - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -This will start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` - -Before running an example program, please make sure the root mount point -set in the `conf/alluxio-site.properties` is a valid path in the ufs. -Make sure the user running the example program has write permissions to the alluxio file system. - -```shell -$ ./bin/alluxio-stop.sh local -``` - -## Advanced Setup - -### [Experimental] OBS multipart upload - -The default upload method uploads one file completely from start to end in one go. We use multipart-upload method to upload one file by multiple parts, every part will be uploaded in one thread. It won't generate any temporary files while uploading. - -To enable OBS multipart upload, you need to modify `conf/alluxio-site.properties` to include: - -```properties -alluxio.underfs.obs.multipart.upload.enabled=true -``` - -There are other parameters you can specify in `conf/alluxio-site.properties` to make the process faster and better. - -```properties -# Timeout for uploading part when using multipart upload. -alluxio.underfs.object.store.multipart.upload.timeout -``` -```properties -# Thread pool size for OBS multipart upload. -alluxio.underfs.obs.multipart.upload.threads -``` -```properties -# Multipart upload partition size for OBS. The default partition size is 64MB. -alluxio.underfs.obs.multipart.upload.partition.size -``` - - - diff --git a/docs/en/ufs/OSS.md b/docs/en/ufs/OSS.md deleted file mode 100644 index 77baa60a74a4..000000000000 --- a/docs/en/ufs/OSS.md +++ /dev/null @@ -1,102 +0,0 @@ ---- -layout: global -title: Aliyun Object Storage Service ---- - - -This guide describes how to configure [Aliyun OSS](https://intl.aliyun.com/product/oss) as Alluxio's under storage system. - -Object Storage Service (OSS) is a massive, secure and highly reliable cloud storage service provided by Aliyun. - -## Prerequisites - -In preparation for using OSS with Alluxio, follow the [OSS quick start guide](https://www.alibabacloud.com/help/doc-detail/31883.htm) -to sign up for OSS and create an OSS bucket. - -## Basic Setup - -To configure Alluxio to use OSS as under storage, you will need to modify the configuration file -`conf/alluxio-site.properties`. If the file does not exist, create the configuration file from the template. - -```shell -$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties -``` - -Edit `conf/alluxio-site.properties` file to set the under storage address to the OSS bucket and -the OSS directory you want to mount to Alluxio. For example, the under storage address can be `oss://alluxio-bucket/` if -you want to mount the whole bucket to Alluxio, or `oss://alluxio-bucket/alluxio/data` if only the directory `/alluxio/data` -inside the oss bucket `alluxio-bucket` is mapped to Alluxio. - -```properties -alluxio.dora.client.ufs.root=oss:/// -``` - -Specify the Aliyun credentials for OSS access. In `conf/alluxio-site.properties`, add: - -```properties -fs.oss.accessKeyId= -fs.oss.accessKeySecret= -fs.oss.endpoint= -``` - -`fs.oss.accessKeyId` and `fs.oss.accessKeySecret` is the [AccessKey](https://www.alibabacloud.com/help/doc-detail/29009.htm) for OSS, -which are created and managed in [Aliyun AccessKey management console](https://ak-console.aliyun.com). - -`fs.oss.endpoint` is the internet endpoint of this bucket, which can be found in the bucket overview page with -values like `oss-us-west-1.aliyuncs.com` and `oss-cn-shanghai.aliyuncs.com`. Available endpoints are listed in the -[OSS Internet Endpoints documentation](https://intl.aliyun.com/help/doc-detail/31837.htm). - -## Running Alluxio Locally with Aliyun OSS - -Start the Alluxio servers: - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -This will start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` - -Visit your OSS directory `oss:///` to verify the files -and directories created by Alluxio exist. For this test, you should see files named like -`//default_tests_files/BasicFile_CACHE_PROMOTE_MUST_CACHE`. - -Stop Alluxio by running: - -```shell -$ ./bin/alluxio-stop.sh local -``` - -## Advanced Setup - -### [Experimental] OSS multipart upload - -The default upload method uploads one file completely from start to end in one go. We use multipart-upload method to upload one file by multiple parts, every part will be uploaded in one thread. It won't generate any temporary files while uploading. - -To enable OSS multipart upload, you need to modify `conf/alluxio-site.properties` to include: - -```properties -alluxio.underfs.oss.multipart.upload.enabled=true -``` - -There are other parameters you can specify in `conf/alluxio-site.properties` to make the process faster and better. - -```properties -# Timeout for uploading part when using multipart upload. -alluxio.underfs.object.store.multipart.upload.timeout -``` -```properties -# Thread pool size for OSS multipart upload. -alluxio.underfs.oss.multipart.upload.threads -``` -```properties -# Multipart upload partition size for OSS. The default partition size is 64MB. -alluxio.underfs.oss.multipart.upload.partition.size -``` diff --git a/docs/en/ufs/Ozone.md b/docs/en/ufs/Ozone.md index b6c2e962684d..3f393f73fee6 100644 --- a/docs/en/ufs/Ozone.md +++ b/docs/en/ufs/Ozone.md @@ -4,45 +4,106 @@ title: Ozone --- -This guide describes how to configure [Ozone](https://ozone.apache.org/) as Alluxio's under storage system. +This guide describes how to configure [Ozone](https://ozone.apache.org/){:target="_blank"} as Alluxio's under storage system. Ozone is a scalable, redundant, and distributed object store for Hadoop. Apart from scaling to billions of objects of varying sizes, Ozone can function effectively in containerized environments such as Kubernetes and YARN. +Ozone supports two different schemas. The biggest difference between `o3fs` and `ofs` is that `o3fs` suports operations only at a **single bucket**, while `ofs` supports operations across all volumes and buckets and provides a full view of all the volume/buckets. For more information, please read its documentation: +- [o3fs](https://ozone.apache.org/docs/1.0.0/interface/o3fs.html){:target="_blank"} +- [ofs](https://ozone.apache.org/docs/1.0.0/interface/ofs.html){:target="_blank"} + ## Prerequisites -In preparation for using Ozone with Alluxio, follow the [Ozone On Premise Installation](https://ozone.apache.org/docs/1.2.1/start/onprem.html) -to install a Ozone cluster, and follow the [Cli Commands](https://ozone.apache.org/docs/1.2.1/interface/cli.html) to create volume and bucket for Ozone cluster. +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using Ozone with Alluxio: + +{% navtabs Prerequisites %} +{% navtab o3fs %} + + + + + + + + + + + + + + + + + + +
``[Create a new volume](https://ozone.apache.org/docs/1.2.1/interface/o3fs.html){:target="_blank"} or use an existing volume
``[Create a new bucket](https://ozone.apache.org/docs/1.2.1/interface/o3fs.html){:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``To select between the available HA clusters, a logical named called a serviceID is required for each of the cluseters. Read [more](https://ozone.apache.org/docs/1.0.0/feature/ha.html){:target="_blank"}
+ +{% endnavtab %} +{% navtab ofs %} + + + + + + + + + + + + + + + + + +
``The namespace manager for Ozone. See [Ozone Manager](https://ozone.apache.org/docs/1.2.1/concept/ozonemanager.html){:target="_blank"}
``[Create a new volume](https://ozone.apache.org/docs/1.2.1/interface/ofs.html){:target="_blank"} or use an existing volume
``[Create a new bucket](https://ozone.apache.org/docs/1.2.1/interface/ofs.html){:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
+ +{% endnavtab %} +{% endnavtabs %} + +Follow the [Ozone On Premise Installation](https://ozone.apache.org/docs/1.2.1/start/onprem.html){:target="_blank"} to install a Ozone cluster. ## Basic Setup -To configure Alluxio to use Ozone as under storage, you will need to modify the configuration file -`conf/alluxio-site.properties`. If the file does not exist, create the configuration file from the template. +To use Ozone as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. ```shell $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties ``` -Edit `conf/alluxio-site.properties` file to set the under storage address to the Ozone bucket and the Ozone directory you want to mount to Alluxio. -Ozone supports two different schemas `o3fs` and `ofs` -### o3fs +Specify an Ozone bucket and directory as the underfs address by modifying `conf/alluxio-site.properties`. + +{% navtabs Setup %} +{% navtab o3fs %} + For example, the under storage address can be `o3fs://./` if you want to mount the whole bucket to Alluxio, or `o3fs://./alluxio/data` if only the directory `/alluxio/data` inside the ozone bucket `` of `` is mapped to Alluxio. -set the property `alluxio.underfs.hdfs.configuration` in `conf/alluxio-site.properties` to point to your `ozone-site.xml`. Make sure this configuration is set on all servers running Alluxio. - ```properties alluxio.dora.client.ufs.root=o3fs://./ +``` + +Set the property `alluxio.underfs.hdfs.configuration` in `conf/alluxio-site.properties` to point to your `ozone-site.xml`. Make sure this configuration is set on all servers running Alluxio. + +```properties alluxio.underfs.hdfs.configuration=/path/to/hdfs/conf/ozone-site.xml ``` -### ofs +{% endnavtab %} +{% navtab ofs %} + For example, the under storage address can be `ofs://///` if you want to mount the whole bucket to Alluxio, or `ofs://///alluxio/data` if only the directory `/alluxio/data` inside the ozone bucket `` of `` is mapped to Alluxio. ```properties alluxio.dora.client.ufs.root=ofs:///// ``` +{% endnavtab %} +{% endnavtabs %} ## Ozone HA Mode ### o3fs @@ -84,30 +145,13 @@ alluxio.underfs.version= ## Running Alluxio Locally with Ozone -Start the Alluxio servers: - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -This will start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). +Once you have configured Alluxio to Ozone, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. -Run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` - -Use the HDFS shell or Ozone shell to Visit your Ozone directory `o3fs://./` +Use the HDFS shell or Ozone shell to visit your Ozone directory `o3fs://./` to verify the files and directories created by Alluxio exist. For this test, you should see files named like -`.//default_tests_files/BasicFile_CACHE_PROMOTE_MUST_CACHE`. - -Stop Alluxio by running: ```shell -$ ./bin/alluxio-stop.sh local +.//default_tests_files/BasicFile_CACHE_PROMOTE_MUST_CACHE ``` ## Advanced Setup @@ -119,7 +163,7 @@ Currently, the only tested Ozone version with Alluxio is `1.0.0`, `1.1.0`, `1.2. ## Contributed by the Alluxio Community Ozone UFS integration is contributed and maintained by the Alluxio community. -The source code is located [here](https://github.com/Alluxio/alluxio/tree/master/underfs/ozone). +The source code is located [here](https://github.com/Alluxio/alluxio/tree/master/underfs/ozone){:target="_blank"}. Feel free submit pull requests to improve the integration and update -the documentation [here](https://github.com/Alluxio/alluxio/edit/master/docs/en/ufs/Ozone.md) +the documentation [here](https://github.com/Alluxio/alluxio/edit/master/docs/en/ufs/Ozone.md){:target="_blank"} if any information is missing or out of date. diff --git a/docs/en/ufs/Qiniu-KODO.md b/docs/en/ufs/Qiniu-KODO.md index f2e7db255bfc..94117bd15dbb 100644 --- a/docs/en/ufs/Qiniu-KODO.md +++ b/docs/en/ufs/Qiniu-KODO.md @@ -4,45 +4,64 @@ title: Qiniu Kodo --- -This guide describes how to configure Alluxio with [Qiniu Kodo](https://www.qiniu.com/products/kodo) as the under storage system. +This guide describes how to configure Alluxio with [Qiniu Kodo](https://www.qiniu.com/products/kodo){:target="_blank"} as the under storage system. -Qiniu Object Storage Service (Kodo) is a cloud-based object storage service provided by Qiniu Cloud, a Chinese cloud service provider. Kodo is a massive, secure and highly reliable cloud storge service that is designed to store, manage, and serve large amounts of unstructured data. +Object Storage (Kodo) is a cloud-based object storage service provided by Qiniu Cloud, a Chinese cloud service provider. Kodo is a massive, secure and highly reliable cloud storge service that is designed to store, manage, and serve large amounts of unstructured data. ## Prerequisites -A Qiniu Kodo bucket is necessary before using Kodo with Alluxio. In this guide, the Qiniu Kodo bucket -is called `KODO_BUCKET`, and the directory in the bucket is called `KODO_DIRECTORY`. -In addition, you should provide a domain to identify the specified bucket, which is called `KODO_DOWNLOAD_HOST`. -Through the `KODO_DOWNLOAD_HOST` you can get objects from the bucket. +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. -## Mounting Kodo +In preparation for using Qiniu Kodo with Alluxio: + + + + + + + + + + + + + + + + + + + + + +
``Create a new bucket or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``
``
``Provide a domain to identify the specificed bucket through which you can get objects from the bucket
-Alluxio unifies access to different storage systems through the -unified namespace feature. -The root of Alluxio namespace or its subdirectories are all available for the mount point of Kodo. +In preparation for using Qiniu Kodo with Alluxio, create a new bucket or use an existing bucket. You should also note the directory you want to use in that bucket, either by creating a new directory in the bucket or using an existing one. -If you want to use Qiniu Kodo as its under storage system in Alluxio, you need to configure Alluxio to use under storage systems by modifying -`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the -template. +## Basic Setup + +To use Qiniu Kodo as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying +`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. ```shell $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties ``` -In the beginning, an existing Kodo bucket and its directory should be specified for storage: +Specify the underfs address by modifying `conf/alluxio-site.properties` to include: + ```properties alluxio.dora.client.ufs.root=kodo://// ``` -Next, some settings must be added to `conf/alluxio-site.properties`: +Specify credentials by adding the following properties in `conf/alluxio-site.properties`: ```properties fs.kodo.accesskey= -fs.kodo.secretkey= +fs.kodo.secretkey= alluxio.underfs.kodo.downloadhost= alluxio.underfs.kodo.endpoint= ``` -`AccessKey/SecretKey` can be found in [Qiniu Console - Key Management](https://portal.qiniu.com/user/key) +`accessKey/secretKey` can be found in [Qiniu Console - Key Management](https://portal.qiniu.com/user/key){:target="_blank"} -`alluxio.underfs.kodo.downloadhost` can be found in [Qiniu Console - Kodo](https://portal.qiniu.com/bucket) +`alluxio.underfs.kodo.downloadhost` can be found in [Qiniu Console - Kodo](https://portal.qiniu.com/bucket){:target="_blank"} `alluxio.underfs.kodo.endpoint` is the endpoint of this bucket, which can be found in the bucket in this table: @@ -81,33 +100,12 @@ alluxio.underfs.kodo.endpoint= ## Running Alluxio Locally with Kodo -After everything is configured, you can start up Alluxio locally to see that everything works. - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` -This should start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Next, you can run a simple example program: - -```shell -$ ./bin/alluxio runTests -``` -After this succeeds, you can visit your Kodo directory `kodo:///` to verify the files -and directories mounted by Alluxio exist. For this test, you should see files named like -`KODO_BUCKET/KODO_DIRECTORY/default_tests_files/BasicFile_CACHE_PROMOTE_MUST_CACHE`. - -To stop Alluxio, you can run: -```shell -$ ./bin/alluxio-stop.sh local -``` +Once you have configured Alluxio to Qiniu Kodo, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. ## Contributed by the Alluxio Community Qiniu KODO UFS integration is contributed and maintained by the Alluxio community. -The source code is located [here](https://github.com/Alluxio/alluxio/tree/master/underfs/kodo). +The source code is located [here](https://github.com/Alluxio/alluxio/tree/master/underfs/kodo){:target="_blank"}. Feel free submit pull requests to improve the integration and update -the documentation [here](https://github.com/Alluxio/alluxio/edit/master/docs/en/ufs/Qiniu-KODO.md) +the documentation [here](https://github.com/Alluxio/alluxio/edit/master/docs/en/ufs/Qiniu-KODO.md){:target="_blank"} if any information is missing or out of date. diff --git a/docs/en/ufs/S3.md b/docs/en/ufs/S3.md index 18d0015c0f6e..32164c44be86 100644 --- a/docs/en/ufs/S3.md +++ b/docs/en/ufs/S3.md @@ -4,28 +4,45 @@ title: Amazon AWS S3 --- -This guide describes the instructions to configure [Amazon S3](https://aws.amazon.com/s3/) as Alluxio's under storage system. +This guide describes the instructions to configure [Amazon AWS S3](https://aws.amazon.com/s3/) as Alluxio's under storage system. -Amazon S3, or Amazon Simple Storage Service, is an object storage service offering industry-leading scalability, data availability, security, and performance. +Amazon AWS S3, or Amazon Simple Storage Service, is an object storage service offering industry-leading scalability, data availability, security, and performance. +For more information about Amazon AWS S3, please read its [documentation](https://docs.aws.amazon.com/s3/index.html){:target="_blank"}. ## Prerequisites -In preparation for using S3 with Alluxio, create a new bucket or use an existing bucket. You -should also note the directory you want to use in that bucket, either by creating a new directory in -the bucket, or using an existing one. For the purposes of this guide, the S3 bucket name is called -`S3_BUCKET`, and the directory in that bucket is called `S3_DIRECTORY`. +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using Amazon AWS S3 with Alluxio: + + + + + + + + + + + + + + + + + +
``[Create a new S3 bucket](https://docs.ceph.com/en/quincy/radosgw/s3/bucketops/){:target="_blank"} or use an existing bucket
``The directory you want to use in that container, either by creating a new directory or using an existing one.
``Used to sign programmatic requests made to AWS. See [How to Obtain Access Key ID and Secret Access Key](https://docs.aws.amazon.com/powershell/latest/userguide/pstools-appendix-sign-up.html){:target="_blank"}
``Used to sign programmatic requests made to AWS. See [How to Obtain Access Key ID and Secret Access Key](https://docs.aws.amazon.com/powershell/latest/userguide/pstools-appendix-sign-up.html){:target="_blank"}
## Basic Setup -Create `conf/alluxio-site.properties` if it does not exist. +To use Amazon AWS S3 as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. ```shell $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties ``` -Configure Alluxio to use S3 as its under storage system by modifying `conf/alluxio-site.properties`. -Specify an **existing** S3 bucket and directory as the under storage system by modifying +Specify an **existing** S3 bucket and directory as the underfs address by modifying `conf/alluxio-site.properties` to include: ```properties @@ -35,41 +52,19 @@ alluxio.dora.client.ufs.root=s3:/// Note that if you want to mount the whole s3 bucket, add a trailing slash after the bucket name (e.g. `s3://S3_BUCKET/`). -Specify the AWS credentials for S3 access by setting s3a.accessKeyId and s3a.secretKey in +Specify the AWS credentials for S3 access by setting `s3a.accessKeyId` and `s3a.secretKey` in `alluxio-site.properties`. ```properties -s3a.accessKeyId= -s3a.secretKey= +s3a.accessKeyId= +s3a.secretKey= ``` For other methods of setting AWS credentials, see the credentials section in [Advanced Setup](#advanced-credentials-setup). -After these changes, Alluxio should be configured to work with S3 as its under storage system, and -you can try [Running Alluxio Locally with S3](#running-alluxio-locally-with-s3). - - ## Running Alluxio Locally with S3 -Start up Alluxio locally to see that everything works. - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh master -$ ./bin/alluxio-start.sh worker -``` - -This should start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Before running an example program, please make sure the root mount point -set in the `alluxio-site.properties` is a valid path in the ufs. -Make sure the user running the example program has write permissions to the alluxio file system. - -```shell -$ ./bin/alluxio-stop.sh master -$ ./bin/alluxio-stop.sh worker -``` +Once you have configured Alluxio to Amazon AWS S3, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. ## Advanced Setup diff --git a/docs/en/ufs/Storage-Overview.md b/docs/en/ufs/Storage-Overview.md new file mode 100644 index 000000000000..4cb5b67cc08b --- /dev/null +++ b/docs/en/ufs/Storage-Overview.md @@ -0,0 +1,176 @@ +--- +layout: global +title: Storage Integrations Overview +--- + +This guide will cover general prerequisites and running Alluxio locally with your desired under storage system. To learn how to configure Alluxio with each individual storage system, please look at their respective pages. + +## Prerequisites + +In preparation for using your chosen storage system with Alluxio, please be sure you have all the required location, credentials, and additional properties before you begin configuring Alluxio to your under storage system. + +For the purposes of this guide, the following are placeholders. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Storage SystemLocationCredentialsAdditional Properties
[Amazon AWS S3]({{ '/en/ufs/S3.html' | relativize_url }})`S3_BUCKET`, `S3_DIRECTORY``S3_ACCESS_KEY_ID`, `S3_SECRET_KEY`
[HDFS]({{ '/en/ufs/HDFS.html' | relativize_url }})`HDFS_NAMENODE`, `HDFS_PORT` + Specify Hadoop version:
+ `HADOOP_VERSION`
[Aliyun Object Storage Service (OSS)]({{ '/en/ufs/Aliyun-OSS.html' | relativize_url }})`OSS_BUCKET`, `OSS_DIRECTORY``OSS_ACCESS_KEY_ID`, `OSS_ACCESS_KEY_SECRET`, `OSS_ENDPOINT`
[Azure Blob Store]({{ '/en/ufs/Azure-Blob-Store.html' | relativize_url }})`AZURE_CONTAINER`, `AZURE_DIRECTORY``AZURE_ACCOUNT`, `AZURE_ACCOUNT_KEY`
[Azure Data Lake Storage Gen1]({{ '/en/ufs/Azure-Data-Lake.html' | relativize_url }})`AZURE_DIRECTORY``AZURE_ACCOUNT`OAuth credentials:
+ `CLIENT_ID`, `AUTHENTICATION_KEY`, `TENANT_ID`
[Azure Data Lake Storage Gen2]({{ '/en/ufs/Azure-Data-Lake-Gen2.html' | relativize_url }})`AZURE_CONTAINER`, `AZURE_DIRECTORY``AZURE_ACCOUNT`, `AZURE_SHARED_KEY` + OAuth credentials:
+ `OAUTH_ENDPOINT`, `CLIENT_ID`, `CLIENT_SECRET`, `MSI_ENDPOINT`, `MSI_TENANT`
[CephFS]({{ '/en/ufs/CephFS.html' | relativize_url }})`CEPHFS_CONF_FILE`, `CEPHFS_NAME`, `CEPHFS_DIRECTORY`, `CEPHFS_AUTH_ID`, `CEPHFS_KEYRING_FILE`
[CephObjectStorage]({{ '/en/ufs/CephObjectStorage.html' | relativize_url }})`CEPH_BUCKET`, `CEPH_DIRECTORY` + [S3](http://docs.aws.amazon.com/AmazonS3/latest/API/Welcome.html) (preferred): `S3_ACCESS_KEY_ID`, `S3_SECRET_KEY_ID`
+ [Swift](http://docs.openstack.org/developer/swift/): `SWIFT_USER`, `SWIFT_TENANT`, `SWIFT_PASSWORD`, `SWIFT_AUTH_URL`, `SWIFT_AUTH_METHOD` +
+ Specify S3 properties: (preferred)
+ `RGW_HOSTNAME`, `RGW_PORT`, `INHERIT_ACL`

+
[Google Cloud Storage (GCS)]({{ '/en/ufs/GCS.html' | relativize_url }})`GCS_BUCKET`, `GCS_DIRECTORY`For GCS Version 1: `GCS_ACCESS_KEY_ID`, `GCS_SECRET_ACCESS_KEY`
[Huawei Object Storage Service (OBS)]({{ '/en/ufs/Huawei-OBS.html' | relativize_url }})`OBS_BUCKET`, `OBS_DIRECTORY``OBS_ACCESS_KEY`, `OBS_SECRET_KEY`, `OBS_ENDPOINT`
[MinIO]({{ '/en/ufs/Minio.html' | relativize_url }})`MINIO_BUCKET`, `MINIO_DIRECTORY``S3_ACCESS_KEY_ID`, `S3_SECRET_KEY`, `MINIO_ENDPOINT`
[NFS]({{ '/en/ufs/NFS.html' | relativize_url }})
[Ozone]({{ '/en/ufs/Ozone.html' | relativize_url }}) + [o3fs](https://ozone.apache.org/docs/1.0.0/interface/ofs.html): `OZONE_BUCKET`, `OZONE_VOLUME`
+ [ofs](https://ozone.apache.org/docs/1.0.0/interface/o3fs.html): `OZONE_MANAGER`, `OZONE_BUCKET`, `OZONE_DIRECTORY`, `OZONE_VOLUME`
+ `OM_SERVICE_IDS` + Mount specific version:
+ `OZONE_VERSION`
[Qiniu Kodo]({{ '/en/ufs/Qiniu-KODO.html' | relativize_url }})`KODO_BUCKET`, `KODO_DIRECTORY``KODO_ACCESS_KEY`, `KODO_SECRET_KEY` + Specify domain to identify bucket:
+ `KODO_DOWNLOAD_HOST`, `KODO_ENDPOINT` +
[Swift]({{ '/en/ufs/Swift.html' | relativize_url }})`SWIFT_BUCKET`, `SWIFT_DIRECTORY``SWIFT_USER`, `SWIFT_TENANT`, `SWIFT_PASSWORD`, `SWIFT_AUTH_URL`, `SWIFT_AUTH_METHOD` + Specify Swift Region:
+ `SWIFT_REGION`
[Tencent Cloud Object Storage (COS)]({{ '/en/ufs/Tencent-COS.html' | relativize_url }})`COS_BUCKET`, `COS_DIRECTORY``COS_ACCESS_KEY`, `COS_SECRET_KEY` + Specify COS region:
+ `COS_REGION`, `COS_APPID` +
[Tencent Cloud Object Storage in Hadoop (COSN)]({{ '/en/ufs/Tencent-COS.html' | relativize_url }})`COSN_BUCKET`, `COSN_DIRECTORY``COSN_SECRET_ID`, `COSN_SECRET_KEY` + Specify COSN region:
+ `COSN_REGION` +
+ +## Running Alluxio Locally + +Once you have configured Alluxio to your desired under storage system, start up Alluxio locally to see that everything works. + +```shell +$ ./bin/alluxio format +$ ./bin/alluxio-start.sh local +``` + +This should start an Alluxio master and an Alluxio worker. You can see the master UI at +[http://localhost:19999](http://localhost:19999). + +Run a simple example program: + +```shell +$ ./bin/alluxio runTests +``` + +Visit your container `/` or bucket `/` to verify the files and directories created by Alluxio exist. If there are no errors, then you have successfully configured your storage system! + +To stop Alluxio, you can run: + +``` shell +$ ./bin/alluxio-stop.sh local +``` \ No newline at end of file diff --git a/docs/en/ufs/Swift.md b/docs/en/ufs/Swift.md index 479925e42bdd..1af3de9db2bf 100644 --- a/docs/en/ufs/Swift.md +++ b/docs/en/ufs/Swift.md @@ -5,17 +5,55 @@ title: Swift This guide describes how to configure Alluxio with an under storage system supporting the -[Swift API](http://docs.openstack.org/developer/swift/). +[Swift API](https://wiki.openstack.org/wiki/Swift){:target="_blank"}. Swift is a highly available, distributed, eventually consistent object/blob store. Organizations can use Swift to store lots of data efficiently, safely, and cheaply. -## Basic Setup +For more information about Swift, please read its [documentation](http://docs.openstack.org/developer/swift/){:target="_blank"}. + +## Prerequisites + +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using Swift with Alluxio: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
``[Create a new Swift bucket](https://docs.openstack.org/newton/user-guide/cli-swift-create-containers.html){:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``
``
``
``
``
``
+ -A Swift bucket can be mounted to the Alluxio either at the root of the namespace, or at a nested directory. +## Basic Setup -Configure Alluxio to use under storage systems by modifying -`conf/alluxio-site.properties`. If it does not exist, create the configuration file from the -template. +To use Swift as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. ```shell $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties @@ -24,79 +62,50 @@ $ cp conf/alluxio-site.properties.template conf/alluxio-site.properties Modify `conf/alluxio-site.properties` to include: ```properties -alluxio.dora.client.ufs.root=swift:/// -fs.swift.user= -fs.swift.tenant= -fs.swift.password= -fs.swift.auth.url= -fs.swift.auth.method= +alluxio.dora.client.ufs.root=swift:/// +fs.swift.user= +fs.swift.tenant= +fs.swift.password= +fs.swift.auth.url= +fs.swift.auth.method= ``` -Replace `/` with an existing Swift bucket location. Possible values of -`` are `true`, `false`. Possible values of `` are `keystonev3`, -`keystone`, `tempauth`, `swiftauth`. - When using either keystone authentication, the following parameter can optionally be set: ```properties -fs.swift.region= +fs.swift.region= ``` On the successful authentication, Keystone will return two access URLs: public and private. If Alluxio is used inside company network and Swift is located on the same network it is advised to set value of `` to `false`. -## Options for Swift Object Storage +### Options for Swift Object Storage -Using the Swift module makes [Ceph Object Storage](https://ceph.com/ceph-storage/object-storage/) -and [IBM SoftLayer](https://www.ibm.com/cloud/object-storage) Object Storage as under storage options -for Alluxio. To use Ceph, the [Rados Gateway](http://docs.ceph.com/docs/master/radosgw/) module must +Using the Swift module makes [Ceph Object Storage](https://ceph.com/ceph-storage/object-storage/){:target="_blank"} +and [IBM SoftLayer Object Storage](https://www.ibm.com/cloud/object-storage){:target="_blank"} as under storage options +for Alluxio. To use Ceph, the [Rados Gateway](http://docs.ceph.com/docs/master/radosgw/){:target="_blank"} module must be deployed. -## Running Alluxio Locally with Swift - -Start an Alluxio cluster: - -```shell -$ ./bin/alluxio format -$ ./bin/alluxio-start.sh local -``` - -This should start an Alluxio master and an Alluxio worker. You can see the master UI at -[http://localhost:19999](http://localhost:19999). - -Run a simple example program: +See [CephObjectStorage Integration with Alluxio]({{ '/en/ufs/CephObjectStorage.html' | relativize_url }}). -```shell -$ ./bin/alluxio runTests -``` - -Visit your Swift bucket to verify the files and directories created -by Alluxio exist. For this test, you should see files named like: - -``` -//default_tests_files/BASIC_CACHE_THROUGH -``` - -To stop Alluxio, you can run: +## Running Alluxio Locally with Swift -```shell -$ ./bin/alluxio-stop.sh local -``` +Once you have configured Alluxio to Swift, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. -## Running functional tests +## Running Functional Tests The following command can be used to test if the given Swift credentials are valid. Developers can also use it to run functional tests against a Swift endpoint to validate the contract between Alluxio and Swift. ```shell -$ ./bin/alluxio runUfsTests --path swift:// \ +$ ./bin/alluxio runUfsTests --path swift:// \ -Dfs.swift.user= \ -Dfs.swift.tenant= \ -Dfs.swift.password= \ - -Dfs.swift.auth.url= \ - -Dfs.swift.auth.method= + -Dfs.swift.auth.url= \ + -Dfs.swift.auth.method= ``` ## Advanced Setup diff --git a/docs/en/ufs/Tencent-COS.md b/docs/en/ufs/Tencent-COS.md new file mode 100644 index 000000000000..64d81252c5e9 --- /dev/null +++ b/docs/en/ufs/Tencent-COS.md @@ -0,0 +1,201 @@ +--- +layout: global +title: Tencent COS +--- + +This guide describes the instructions to configure [Tencent COS](https://cloud.tencent.com/product/cos){:target="_blank"} as Alluxio's +under storage system. + +Alluxio support two different implementations of under storage system for Tencent COS: + +* [COS](https://cloud.tencent.com/product/cos){:target="_blank"} +: Tencent Cloud Object Storage (COS) is a distributed storage service offered by Tencent Cloud for unstructured data and accessible via HTTP/HTTPS protocols. It can store massive amounts of data and features imperceptible bandwidth and capacity expansion, making it a perfect data pool for big data computation and analytics. +: For more information about Tencent COS, please read its [documentation](https://www.tencentcloud.com/document/product/436){:target="_blank"}. + +* [COSN](https://hadoop.apache.org/docs/stable/hadoop-cos/cloud-storage/index.html){:target="_blank"}, also known as Hadoop-COS +: COSN is a client that makes the upper computing systems based on HDFS be able to use Tencent COS as its underlying storage system. +: For more information about COSN, please read its [documentation](https://www.tencentcloud.com/document/product/436/6884){:target="_blank"}. + + +## Prerequisites + +If you haven't already, please see [Prerequisites]({{ '/en/ufs/Storage-Overview.html#prerequisites' | relativize_url }}) before you get started. + +In preparation for using COS or COSN with Alluxio: + +{% navtabs Prerequisites %} +{% navtab COS %} + + + + + + + + + + + + + + + + + + + + + + + + + + +
``[Create a new bucket](https://www.tencentcloud.com/document/product/436/32955#step-4.-create-a-bucket){:target="_blank"} or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``A developer-owned access key used for the project. It can be obtained at [Manage API Key](https://console.tencentcloud.com/capi){:target="_blank"}
``A developer-owned secret key used for the project. It can be obtained at [Manage API Key](https://console.tencentcloud.com/capi){:target="_blank"}
``Region information. For more information about the enumerated values, please see [Regions and Access Endpoints](https://www.tencentcloud.com/document/product/436/6224){:target="_blank"}
``A unique user-level resource identifier for COS access. It can be obtained at [Manage API Key](https://console.tencentcloud.com/capi){:target="_blank"}
+ +{% endnavtab %} +{% navtab COSN %} + + + + + + + + + + + + + + + + + + + + + + +
``Create a new bucket or use an existing bucket
``The directory you want to use in the bucket, either by creating a new directory or using an existing one
``ID used to authenticate user
``Key used to authenticate user
``Region information, for more information about the enumerated values please see [Regions and Access Endpoints](https://www.tencentcloud.com/document/product/436/6224){:target="_blank"}
+ +{% endnavtab %} +{% endnavtabs %} + +## Basic Setup + +{% navtabs Setup %} +{% navtab COS %} + +To use Tencent COS as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties`. If it does not exist, create the configuration file from the template. + +```shell +$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties +``` + +Specify an **existing** COS bucket and directory as the underfs address by modifying `conf/alluxio-site.properties` to include: + +```properties +alluxio.dora.client.ufs.root=cos://COS_BUCKET/COS_DIRECTORY/ +``` + +Note that if you want to mount the whole cos bucket, add a trailing slash after the bucket name +(e.g. `cos://COS_BUCKET/`). + +Specify the COS credentials for COS access by setting `fs.cos.access.key` and `fs.cos.secret.key` in +`alluxio-site.properties`. + +```properties +fs.cos.access.key= +fs.cos.secret.key= +``` + +Specify the COS region by setting `fs.cos.region` in `alluxio-site.properties` (e.g. ap-beijing) and `fs.cos.app.id`. + +```properties +fs.cos.region= +fs.cos.app.id= +``` + +{% endnavtab %} +{% navtab COSN %} + +To use Tencent COSN as the UFS of Alluxio root mount point, you need to configure Alluxio to use under storage systems by modifying `conf/alluxio-site.properties` and `conf/core-site.xml`. If they do not exist, create the files from the template. + +```shell +$ cp conf/alluxio-site.properties.template conf/alluxio-site.properties +$ cp conf/core-site.xml.template conf/core-site.xml +``` + +Specify an **existing** COSN bucket and directory as the underfs address by modifying +`conf/alluxio-site.properties` to include: + +```properties +alluxio.dora.client.ufs.root=cosn://COSN_BUCKET/COSN_DIRECTORY/ +``` + +Specify COSN configuration information in order to access COSN by modifying `conf/core-site.xml` to include `COSN_SECRET_KEY`, `COSN_SECRET_ID`, AND `COSN_REGION`: + +```xml + + fs.cosn.impl + org.apache.hadoop.fs.CosFileSystem + + + fs.AbstractFileSystem.cosn.impl + org.apache.hadoop.fs.CosN + + + fs.cosn.userinfo.secretKey + COSN_SECRET_KEY + + + fs.cosn.userinfo.secretId + COSN_SECRET_ID + + + fs.cosn.bucket.region + COSN_REGION + +``` + +The above is the most basic configuration. For more configuration please refer to [here](https://hadoop.apache.org/docs/r3.3.1/hadoop-cos/cloud-storage/index.html){:target="_blank"}. + +{% endnavtab %} +{% endnavtabs %} + +After these changes, Alluxio should be configured to work with COS or COSN as its under storage system. + +## Running Alluxio Locally with COS/COSN + +Once you have configured Alluxio to Tencent COS or COSN, try [running Alluxio locally]({{ '/en/ufs/Storage-Overview.html#running-alluxio-locally' | relativize_url}}) to see that everything works. + +## Advanced Setup + +### [Experimental] COS multipart upload + +The default upload method uploads one file completely from start to end in one go. We use multipart-upload method to upload one file by multiple parts, every part will be uploaded in one thread. It won't generate any temporary files while uploading. + +To enable COS multipart upload, you need to modify `conf/alluxio-site.properties` to include: + +```properties +alluxio.underfs.cos.multipart.upload.enabled=true +``` + +There are other parameters you can specify in `conf/alluxio-site.properties` to make the process faster and better. + +```properties +# Timeout for uploading part when using multipart upload. +alluxio.underfs.object.store.multipart.upload.timeout +``` +```properties +# Thread pool size for COS multipart upload. +alluxio.underfs.cos.multipart.upload.threads +``` +```properties +# Multipart upload partition size for COS. The default partition size is 64MB. +alluxio.underfs.cos.multipart.upload.partition.size +``` + + + From 20eac74409a8256b617e6307a37516d50da0a849 Mon Sep 17 00:00:00 2001 From: lucyge2022 <111789461+lucyge2022@users.noreply.github.com> Date: Tue, 1 Aug 2023 14:53:14 -0700 Subject: [PATCH 11/12] Add Etcd membership module ### What changes are proposed in this pull request? Add membership module for dora membership management. ### Why are the changes needed? Allow usage of etcd cluster or static configuration instead of Master for membership management ### Does this PR introduce any user facing changes? No pr-link: Alluxio/alluxio#17736 change-id: cid-e48bce79049aeddae88183bb885b1acf3e440c66 --- conf/etcd/etcd.conf.template | 69 +++ conf/etcd/etcd.service.template | 11 + .../client/file/FileSystemContext.java | 26 + dora/core/common/pom.xml | 4 + .../main/java/alluxio/conf/PropertyKey.java | 41 ++ .../alluxio/membership/AlluxioEtcdClient.java | 581 ++++++++++++++++++ .../membership/EtcdMembershipManager.java | 209 +++++++ .../alluxio/membership/MembershipManager.java | 132 ++++ .../alluxio/membership/MembershipType.java | 21 + .../membership/NoOpMembershipManager.java | 74 +++ .../membership/ServiceDiscoveryRecipe.java | 361 +++++++++++ .../alluxio/membership/ServiceEntity.java | 103 ++++ .../alluxio/membership/StateListener.java | 30 + .../membership/StaticMembershipManager.java | 175 ++++++ .../membership/WorkerServiceEntity.java | 122 ++++ .../src/main/java/alluxio/util/HashUtils.java | 58 ++ .../scheduler/DefaultWorkerProvider.java | 5 + .../MembershipManagerWorkerProvider.java | 70 +++ .../alluxio/worker/dora/PagedDoraWorker.java | 67 +- .../worker/modules/DoraWorkerModule.java | 9 + .../worker/dora/PagedDoraWorkerTest.java | 7 +- .../alluxio/scheduler/job/WorkerProvider.java | 6 + .../multi/process/MultiProcessCluster.java | 3 + .../cli/fsadmin/command/ReportCommand.java | 14 +- .../cli/fsadmin/report/NodeStatusCommand.java | 50 ++ .../command/DoctorCommandIntegrationTest.java | 2 + .../command/QuorumCommandIntegrationTest.java | 2 + .../ConfigCheckerIntegrationTest.java | 1 + .../WorkerMetadataSyncIntegrationTest.java | 5 + pom.xml | 5 + 30 files changed, 2249 insertions(+), 14 deletions(-) create mode 100644 conf/etcd/etcd.conf.template create mode 100644 conf/etcd/etcd.service.template create mode 100644 dora/core/common/src/main/java/alluxio/membership/AlluxioEtcdClient.java create mode 100644 dora/core/common/src/main/java/alluxio/membership/EtcdMembershipManager.java create mode 100644 dora/core/common/src/main/java/alluxio/membership/MembershipManager.java create mode 100644 dora/core/common/src/main/java/alluxio/membership/MembershipType.java create mode 100644 dora/core/common/src/main/java/alluxio/membership/NoOpMembershipManager.java create mode 100644 dora/core/common/src/main/java/alluxio/membership/ServiceDiscoveryRecipe.java create mode 100644 dora/core/common/src/main/java/alluxio/membership/ServiceEntity.java create mode 100644 dora/core/common/src/main/java/alluxio/membership/StateListener.java create mode 100644 dora/core/common/src/main/java/alluxio/membership/StaticMembershipManager.java create mode 100644 dora/core/common/src/main/java/alluxio/membership/WorkerServiceEntity.java create mode 100644 dora/core/common/src/main/java/alluxio/util/HashUtils.java create mode 100644 dora/core/server/master/src/main/java/alluxio/master/scheduler/MembershipManagerWorkerProvider.java create mode 100644 dora/shell/src/main/java/alluxio/cli/fsadmin/report/NodeStatusCommand.java diff --git a/conf/etcd/etcd.conf.template b/conf/etcd/etcd.conf.template new file mode 100644 index 000000000000..1b125d74fae2 --- /dev/null +++ b/conf/etcd/etcd.conf.template @@ -0,0 +1,69 @@ +# This is the configuration file to start a etcd instance +# e.g. /usr/local/bin/etcd --config-file /etc/etcd/etcd.conf +# *******README****** +# To make etcd a linux service: +# After installation of etcd, make sure etcd and etcdctl +# are available in /usr/local/bin +# To make etcd a linux service: +# Copy alluxio/conf/etcd/etcd.service.template to /etc/systemd/system/etcd.service +# Copy alluxio/conf/etcd/etcd.conf.template to /etc/etcd/etcd.conf +# For each etcd instance, change the config params in etcd.conf +# accordingly. +# And do: +# #systemctl daemon-reload +# Then etcd could be registered as a linux service +# e.g. +# Check status +# #service etcd status +# Start etcd +# #service etcd start +# Stop etcd +# #service etcd stop + + +# Human-readable name for this member. +#name: 'etcd1' + +# Path to the data directory. +data-dir: /etcd-data-dir/data + +# Path to the dedicated wal directory. +wal-dir: /etcd-data-dir/wal + + +# List of comma separated URLs to listen on for peer traffic. +#give ip/hostname of this etcd instance +listen-peer-urls: http://:2380 + +# List of comma separated URLs to listen on for client traffic. +#give ip/hostname of this etcd instance +listen-client-urls: http://:2379,http://127.0.0.1:2379 + +# List of this member's peer URLs to advertise to the rest of the cluster. +# The URLs needed to be a comma-separated list. +#give ip/hostname of this etcd instance for remote etcd members communication +initial-advertise-peer-urls: http://:2380 + +# List of this member's client URLs to advertise to the public. +# The URLs needed to be a comma-separated list. +#give ip/hostname of this etcd instance for etcd client communication +advertise-client-urls: http://:2379 + +# Initial cluster configuration for bootstrapping. +#give all ip/hostnames of members of initial etcd cluster +initial-cluster: etcd0=http://:2380,etcd1=http://:2380,etcd2=http://:2380 + +# Initial cluster token for the etcd cluster during bootstrap. +#initial-cluster-token: 'etcd-cluster-1' + +# Initial cluster state ('new' or 'existing'). +initial-cluster-state: 'new' + +# Enable debug-level logging for etcd. +#log-level: debug + +#logger: zap + +# Specify 'stdout' or 'stderr' to skip journald logging even when running under systemd. +# log-outputs: [stderr] + diff --git a/conf/etcd/etcd.service.template b/conf/etcd/etcd.service.template new file mode 100644 index 000000000000..70a51c67475c --- /dev/null +++ b/conf/etcd/etcd.service.template @@ -0,0 +1,11 @@ +[Unit] +Description=Etcd Service + +[Service] +ExecStart=/usr/local/bin/etcd --config-file /etc/etcd/etcd.conf +KillSignal=SIGTERM +StandardOutput=append:/var/log/etcd.log +StandardError=append:/var/log/etcd.err + +[Install] +WantedBy=default.target diff --git a/dora/core/client/fs/src/main/java/alluxio/client/file/FileSystemContext.java b/dora/core/client/fs/src/main/java/alluxio/client/file/FileSystemContext.java index 1783a0b86c09..3a49c4ef45e3 100644 --- a/dora/core/client/fs/src/main/java/alluxio/client/file/FileSystemContext.java +++ b/dora/core/client/fs/src/main/java/alluxio/client/file/FileSystemContext.java @@ -34,6 +34,8 @@ import alluxio.grpc.GrpcServerAddress; import alluxio.master.MasterClientContext; import alluxio.master.MasterInquireClient; +import alluxio.membership.MembershipManager; +import alluxio.membership.NoOpMembershipManager; import alluxio.metrics.MetricsSystem; import alluxio.network.netty.NettyChannelPool; import alluxio.network.netty.NettyClient; @@ -154,6 +156,8 @@ public class FileSystemContext implements Closeable { */ private volatile ConcurrentHashMap mBlockWorkerClientPoolMap; + @Nullable + private MembershipManager mMembershipManager; /** * Indicates whether the {@link #mLocalWorker} field has been lazily initialized yet. @@ -443,6 +447,11 @@ protected synchronized void initContext(ClientContext ctx, mBlockMasterClientPool = new BlockMasterClientPool(mMasterClientContext); mBlockWorkerClientPoolMap = new ConcurrentHashMap<>(); mUriValidationEnabled = ctx.getUriValidationEnabled(); + try { + mMembershipManager = MembershipManager.Factory.create(getClusterConf()); + } catch (IOException ex) { + LOG.error("Failed to set membership manager.", ex); + } } /** @@ -490,6 +499,12 @@ private synchronized void closeContext() throws IOException { if (mMetricsEnabled) { MetricsHeartbeatContext.removeHeartbeat(getClientContext()); } + LOG.debug("Closing membership manager."); + try (AutoCloseable ignoredCloser = mMembershipManager) { + // do nothing as we are closing + } catch (Exception e) { + throw new IOException(e); + } } else { LOG.warn("Attempted to close FileSystemContext which has already been closed or not " + "initialized."); @@ -864,6 +879,17 @@ public List getCachedWorkers() throws IOException { * @return the info of all block workers */ protected List getAllWorkers() throws IOException { + // TODO(lucy) once ConfigHashSync reinit is gotten rid of, will remove the blockReinit + // guard altogether + try (ReinitBlockerResource r = blockReinit()) { + // Use membership mgr + if (mMembershipManager != null && !(mMembershipManager instanceof NoOpMembershipManager)) { + return mMembershipManager.getAllMembers().stream() + .map(w -> new BlockWorkerInfo(w.getAddress(), w.getCapacityBytes(), w.getUsedBytes())) + .collect(toList()); + } + } + // Fall back to old way try (CloseableResource masterClientResource = acquireBlockMasterClientResource()) { return masterClientResource.get().getWorkerInfoList().stream() diff --git a/dora/core/common/pom.xml b/dora/core/common/pom.xml index 7ee978e04f1d..e1dcc0805b5b 100644 --- a/dora/core/common/pom.xml +++ b/dora/core/common/pom.xml @@ -87,6 +87,10 @@ io.dropwizard.metrics metrics-jvm + + io.etcd + jetcd-core + io.grpc grpc-core diff --git a/dora/core/common/src/main/java/alluxio/conf/PropertyKey.java b/dora/core/common/src/main/java/alluxio/conf/PropertyKey.java index d8a7d4aae146..86264c96d833 100755 --- a/dora/core/common/src/main/java/alluxio/conf/PropertyKey.java +++ b/dora/core/common/src/main/java/alluxio/conf/PropertyKey.java @@ -48,6 +48,7 @@ import alluxio.master.metastore.MetastoreType; import alluxio.master.metastore.rocks.DataBlockIndexType; import alluxio.master.metastore.rocks.IndexType; +import alluxio.membership.MembershipType; import alluxio.network.ChannelType; import alluxio.network.netty.FileTransferType; import alluxio.security.authentication.AuthType; @@ -5505,6 +5506,25 @@ public String toString() { .setConsistencyCheckLevel(ConsistencyCheckLevel.WARN) .setScope(Scope.WORKER) .build(); + public static final PropertyKey WORKER_MEMBERSHIP_MANAGER_TYPE = + enumBuilder(Name.WORKER_MEMBERSHIP_MANAGER_TYPE, MembershipType.class) + .setDefaultValue(MembershipType.NOOP.name()) + .setDescription("Type of membership manager used for workers." + + "Choose STATIC for pre-configured members." + + "Choose ETCD for using etcd for membership management" + + "Default is NOOP which does not enable membership manager at all") + .setConsistencyCheckLevel(ConsistencyCheckLevel.ENFORCE) + .setScope(Scope.ALL) + .build(); + public static final PropertyKey WORKER_STATIC_MEMBERSHIP_MANAGER_CONFIG_FILE = + stringBuilder(Name.WORKER_STATIC_MEMBERSHIP_MANAGER_CONFIG_FILE) + .setDefaultValue(format("${%s}/workers", Name.CONF_DIR)) + .setDescription("Absolute path of the config file for list" + + "of worker hostnames/IPs for the cluster. " + + Name.WORKER_MEMBERSHIP_MANAGER_TYPE + " needs to be set" + + " to STATIC first.") + .setScope(Scope.ALL) + .build(); // // Proxy related properties @@ -7626,6 +7646,19 @@ public String toString() { stringBuilder(Name.ZOOKEEPER_JOB_LEADER_PATH) .setDefaultValue("/alluxio/job_leader").build(); + // + // Membership related properties + // + public static final PropertyKey ALLUXIO_CLUSTER_NAME = + stringBuilder(Name.ALLUXIO_CLUSTER_NAME) + .setDefaultValue("DefaultAlluxioCluster").build(); + public static final PropertyKey ETCD_ENDPOINTS = + listBuilder(Name.ETCD_ENDPOINTS) + .setDescription("A list of comma-separated http://host:port addresses of " + + "etcd cluster (e.g. http://localhost:2379,http://etcd1:2379)") + .setScope(Scope.ALL) + .build(); + // // JVM Monitor related properties // @@ -8993,6 +9026,10 @@ public static final class Name { public static final String WORKER_UFS_INSTREAM_CACHE_MAX_SIZE = "alluxio.worker.ufs.instream.cache.max.size"; public static final String WORKER_WHITELIST = "alluxio.worker.whitelist"; + public static final String WORKER_MEMBERSHIP_MANAGER_TYPE = + "alluxio.worker.membership.manager.type"; + public static final String WORKER_STATIC_MEMBERSHIP_MANAGER_CONFIG_FILE = + "alluxio.worker.static.membership.manager.config.file"; // // Proxy related properties @@ -9480,6 +9517,10 @@ public static final class Name { public static final String ZOOKEEPER_JOB_ELECTION_PATH = "alluxio.zookeeper.job.election.path"; public static final String ZOOKEEPER_JOB_LEADER_PATH = "alluxio.zookeeper.job.leader.path"; + // Membership related properties + public static final String ALLUXIO_CLUSTER_NAME = "alluxio.cluster.name"; + public static final String ETCD_ENDPOINTS = "alluxio.etcd.endpoints"; + // // JVM Monitor related properties // diff --git a/dora/core/common/src/main/java/alluxio/membership/AlluxioEtcdClient.java b/dora/core/common/src/main/java/alluxio/membership/AlluxioEtcdClient.java new file mode 100644 index 000000000000..13e44bb6b3ab --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/AlluxioEtcdClient.java @@ -0,0 +1,581 @@ +/* + * 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.membership; + +import alluxio.conf.AlluxioConfiguration; +import alluxio.conf.PropertyKey; +import alluxio.exception.runtime.AlluxioRuntimeException; +import alluxio.resource.LockResource; +import alluxio.retry.ExponentialBackoffRetry; +import alluxio.retry.RetryUtils; +import alluxio.util.io.PathUtils; + +import com.google.common.base.MoreObjects; +import com.google.common.base.Preconditions; +import com.google.common.io.Closer; +import io.etcd.jetcd.ByteSequence; +import io.etcd.jetcd.Client; +import io.etcd.jetcd.KeyValue; +import io.etcd.jetcd.Watch; +import io.etcd.jetcd.kv.GetResponse; +import io.etcd.jetcd.lease.LeaseGrantResponse; +import io.etcd.jetcd.lease.LeaseRevokeResponse; +import io.etcd.jetcd.lease.LeaseTimeToLiveResponse; +import io.etcd.jetcd.options.DeleteOption; +import io.etcd.jetcd.options.GetOption; +import io.etcd.jetcd.options.LeaseOption; +import io.etcd.jetcd.options.WatchOption; +import io.etcd.jetcd.watch.WatchEvent; +import io.etcd.jetcd.watch.WatchResponse; +import io.netty.util.internal.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +/** + * Wrapper class around jetcd client to achieve utilities API to talk with ETCD. + * This class is supposed to be used as a singleton fashion. It wraps around + * one jetcd Client instance for all sorts of utility functions to interact with etcd. + * Only state it's keeping is the jetcd Client and registered Watcher list + * For kv operations such as Put(createForPath, deleteForPath, addChildren, etc.) + * its atomicity/consistency semantics goes with what ETCD has to offer, this class + * does not add upon any semantics itself. + * + * AlluxioEtcdClient should only be used as singleton wrapping one jetcd Client object, + * currently only resource - jetcd client will be closed as part of close() which is + * called during: + * 1) Worker shutdown or close as part of EtcdMembershipManager close + * 2) FileSystemContext closeContext as part of EtcdMembershipManager close + * As we never set mClient to be null after connect, also jetcd client can be closed idempotently + * so it's ok to ignore thread safety for close() + * + * As for jetcd Client, it's managing its own connect/reconnect/loadbalance to other etcd + * instances, will leave these logic to jetcd client itself for now unless we need to + * handle it in our layer. + */ +public class AlluxioEtcdClient implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(AlluxioEtcdClient.class); + private static final Lock INSTANCE_LOCK = new ReentrantLock(); + public static final long DEFAULT_LEASE_TTL_IN_SEC = 2L; + public static final long DEFAULT_TIMEOUT_IN_SEC = 2L; + public static final int RETRY_TIMES = 3; + private static final int RETRY_SLEEP_IN_MS = 100; + private static final int MAX_RETRY_SLEEP_IN_MS = 500; + @GuardedBy("INSTANCE_LOCK") + @Nullable + private static volatile AlluxioEtcdClient sAlluxioEtcdClient; + public final ServiceDiscoveryRecipe mServiceDiscovery; + private final AtomicBoolean mConnected = new AtomicBoolean(false); + private final Closer mCloser = Closer.create(); + // only watch for children change(add/remove) for given parent path + private final ConcurrentHashMap mRegisteredWatchers = + new ConcurrentHashMap<>(); + private Client mClient; + private final String[] mEndpoints; + + /** + * CTOR for AlluxioEtcdClient. + * @param conf + */ + public AlluxioEtcdClient(AlluxioConfiguration conf) { + String clusterName = conf.getString(PropertyKey.ALLUXIO_CLUSTER_NAME); + List endpointsList = conf.getList(PropertyKey.ETCD_ENDPOINTS); + mEndpoints = endpointsList.toArray(new String[0]); + mServiceDiscovery = new ServiceDiscoveryRecipe(this, clusterName); + } + + /** + * Get the singleton instance of AlluxioEtcdClient. + * @param conf + * @return AlluxioEtcdClient + */ + public static AlluxioEtcdClient getInstance(AlluxioConfiguration conf) { + if (sAlluxioEtcdClient == null) { + try (LockResource lockResource = new LockResource(INSTANCE_LOCK)) { + if (sAlluxioEtcdClient == null) { + sAlluxioEtcdClient = new AlluxioEtcdClient(conf); + } + } + } + return sAlluxioEtcdClient; + } + + /** + * Create jetcd grpc client no forcing. + */ + public void connect() { + connect(false); + } + + /** + * Create jetcd grpc client and force(or not) connection. + * @param force + */ + public void connect(boolean force) { + if (mConnected.get() && !force) { + return; + } + mConnected.set(false); + // create client using endpoints + Client client = Client.builder().endpoints(mEndpoints) + .build(); + if (mConnected.compareAndSet(false, true)) { + mClient = client; + } + } + + /** + * Disconnect. + * @throws IOException + */ + public void disconnect() throws IOException { + close(); + } + + /** + * Watch for a single path or the change among all children of this path. + */ + enum WatchType { + CHILDREN, + SINGLE_PATH + } + + /** + * Lease structure to keep the info about a lease in etcd. + */ + public static class Lease { + public long mLeaseId = -1; + public long mTtlInSec = -1; + + /** + * CTOR for Lease. + * @param leaseId + * @param ttlInSec + */ + public Lease(long leaseId, long ttlInSec) { + mLeaseId = leaseId; + mTtlInSec = ttlInSec; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("leaseId", mLeaseId) + .add("ttl", mTtlInSec) + .toString(); + } + } + + /** + * Create a lease with timeout and ttl. + * @param ttlInSec + * @param timeout + * @param timeUnit + * @return Lease + * @throws IOException + */ + public Lease createLease(long ttlInSec, long timeout, TimeUnit timeUnit) + throws IOException { + try { + return RetryUtils.retryCallable(String.format("Creating Lease with ttl:%s", ttlInSec), () -> { + CompletableFuture leaseGrantFut = + getEtcdClient().getLeaseClient().grant(ttlInSec, timeout, timeUnit); + long leaseId; + LeaseGrantResponse resp = leaseGrantFut.get(timeout, timeUnit); + leaseId = resp.getID(); + Lease lease = new Lease(leaseId, ttlInSec); + return lease; + }, new ExponentialBackoffRetry(RETRY_SLEEP_IN_MS, MAX_RETRY_SLEEP_IN_MS, RETRY_TIMES)); + } catch (AlluxioRuntimeException ex) { + throw new IOException(ex.getMessage(), ex.getCause()); + } + } + + /** + * Create lease with default ttl and timeout. + * @return Lease + * @throws IOException + */ + public Lease createLease() throws IOException { + return createLease(DEFAULT_LEASE_TTL_IN_SEC, DEFAULT_TIMEOUT_IN_SEC, TimeUnit.SECONDS); + } + + /** + * Revoke given lease. + * @param lease + * @throws IOException + */ + public void revokeLease(Lease lease) throws IOException { + RetryUtils.retry(String.format("Revoking Lease:%s", lease.toString()), () -> { + try { + CompletableFuture leaseRevokeFut = + getEtcdClient().getLeaseClient().revoke(lease.mLeaseId); + long leaseId; + LeaseRevokeResponse resp = leaseRevokeFut.get(DEFAULT_TIMEOUT_IN_SEC, TimeUnit.SECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException ex) { + throw new IOException("Error revoking lease:" + lease.toString(), ex); + } + }, new ExponentialBackoffRetry(RETRY_SLEEP_IN_MS, MAX_RETRY_SLEEP_IN_MS, RETRY_TIMES)); + } + + /** + * Check with etcd if a lease is already expired. + * @param lease + * @return lease expired + */ + public boolean isLeaseExpired(Lease lease) throws IOException { + try { + return RetryUtils.retryCallable( + String.format("Checking IsLeaseExpired, lease:%s", lease.toString()), () -> { + LeaseTimeToLiveResponse leaseResp = mClient.getLeaseClient() + .timeToLive(lease.mLeaseId, LeaseOption.DEFAULT) + .get(DEFAULT_TIMEOUT_IN_SEC, TimeUnit.SECONDS); + // if no such lease, lease resp will still be returned with a negative ttl + return leaseResp.getTTl() <= 0; + }, new ExponentialBackoffRetry(RETRY_SLEEP_IN_MS, MAX_RETRY_SLEEP_IN_MS, RETRY_TIMES)); + } catch (AlluxioRuntimeException e) { + throw new IOException("Failed to check if lease expired:" + lease.toString(), e.getCause()); + } + } + + /** + * Create a childPath with value to a parentPath. + * e.g. create "lower_path" under path /upper_path/ to form a + * kv pair of /upper_path/lower_path with a given value. + * @param parentPath + * @param childPath + * @param value + */ + public void addChildren(String parentPath, String childPath, byte[] value) + throws IOException { + Preconditions.checkArgument(!StringUtil.isNullOrEmpty(parentPath)); + Preconditions.checkArgument(!StringUtil.isNullOrEmpty(childPath)); + String fullPath = PathUtils.concatPath(parentPath, childPath); + Preconditions.checkArgument(!StringUtil.isNullOrEmpty(fullPath)); + RetryUtils.retry( + String.format("Adding child, parentPath:%s, childPath:%s", + parentPath, childPath), () -> { + try { + mClient.getKVClient().put( + ByteSequence.from(fullPath, StandardCharsets.UTF_8), + ByteSequence.from(value)) + .get(DEFAULT_TIMEOUT_IN_SEC, TimeUnit.SECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException e) { + throw new IOException("Failed to addChildren, parentPath:" + parentPath + + " child:" + childPath, e); + } + }, + new ExponentialBackoffRetry(RETRY_SLEEP_IN_MS, MAX_RETRY_SLEEP_IN_MS, 0)); + } + + /** + * Get list of children path kv pairs from a given parentPath + * e.g. get [/upper/lower1 - val1, /upper/lower2 - val2] + * under parent path /upper/ + * @param parentPath parentPath ends with / + * @return list of children KeyValues + */ + public List getChildren(String parentPath) throws IOException { + try { + Preconditions.checkArgument(!StringUtil.isNullOrEmpty(parentPath)); + return RetryUtils.retryCallable( + String.format("Getting children for path:%s", parentPath), () -> { + GetResponse getResponse = mClient.getKVClient().get( + ByteSequence.from(parentPath, StandardCharsets.UTF_8), + GetOption.newBuilder().isPrefix(true).build()) + .get(DEFAULT_TIMEOUT_IN_SEC, TimeUnit.SECONDS); + return getResponse.getKvs(); + }, new ExponentialBackoffRetry(RETRY_SLEEP_IN_MS, MAX_RETRY_SLEEP_IN_MS, RETRY_TIMES)); + } catch (AlluxioRuntimeException e) { + throw new IOException("Failed to getChildren for parentPath:" + parentPath, e.getCause()); + } + } + + /** + * Add listener to a path internal function. + * @param parentPath + * @param listener + * @param watchType + */ + private void addListenerInternal( + String parentPath, StateListener listener, WatchType watchType) { + if (mRegisteredWatchers.containsKey(getRegisterWatcherKey(parentPath, watchType))) { + LOG.warn("Watcher already there for path:{} for children.", parentPath); + return; + } + WatchOption.Builder watchOptBuilder = WatchOption.newBuilder(); + switch (watchType) { + /* e.g. Given the parentPath '/parent/', + give query-like syntax equivalent to: + select * with value < '/parent0' ('0' the char after '/' in ASCII) + since everything prefixed with '/parent/' is strictly smaller than '/parent0' + Example: with list of keys ['/parent-1', '/parent/k1','/parent/~'] + this query with keyRangeEnd = '/parent0' will result with ['/parent/k1', '/parent/~'] + since '/parent-1' is not prefixed with '/parent/' + and '/parent/~' is the largest below '/parent0' + */ + case CHILDREN: + String keyRangeEnd = parentPath.substring(0, parentPath.length() - 1) + + (char) (parentPath.charAt(parentPath.length() - 1) + 1); + watchOptBuilder.isPrefix(true) + .withRange(ByteSequence.from(keyRangeEnd, StandardCharsets.UTF_8)); + break; + case SINGLE_PATH: // no need to add anything to watchoption, fall through. + default: + break; + } + + Watch.Watcher watcher = mClient.getWatchClient().watch( + ByteSequence.from(parentPath, StandardCharsets.UTF_8), + watchOptBuilder.build(), + new Watch.Listener() { + @Override + public void onNext(WatchResponse response) { + for (WatchEvent event : response.getEvents()) { + switch (event.getEventType()) { + case PUT: + listener.onNewPut( + event.getKeyValue().getKey().toString(StandardCharsets.UTF_8), + event.getKeyValue().getValue().getBytes()); + break; + case DELETE: + listener.onNewDelete( + event.getKeyValue().getKey().toString(StandardCharsets.UTF_8)); + break; + case UNRECOGNIZED: // Fall through + default: + LOG.info("Unrecognized event:{} on watch path of:{}", + event.getEventType(), parentPath); + break; + } + } + } + + @Override + public void onError(Throwable throwable) { + LOG.warn("Error occurred on children watch for path:{}, removing the watch.", + parentPath, throwable); + removeChildrenListener(parentPath); + } + + @Override + public void onCompleted() { + LOG.warn("Watch for path onCompleted:{}, removing the watch.", parentPath); + removeChildrenListener(parentPath); + } + }); + Watch.Watcher prevWatcher = mRegisteredWatchers.putIfAbsent( + getRegisterWatcherKey(parentPath, watchType), watcher); + // another same watcher already added in a race, close current one + if (prevWatcher != null) { + watcher.close(); + } else { + mCloser.register(watcher); + } + } + + /** + * Get the registered watch key in the map. + * @param path + * @param type + * @return key for registered watcher + */ + private static String getRegisterWatcherKey(String path, WatchType type) { + return path + "$$@@$$" + type.toString(); + } + + /** + * Add state listener to given path. + * @param path + * @param listener + */ + public void addStateListener(String path, StateListener listener) { + addListenerInternal(path, listener, WatchType.SINGLE_PATH); + } + + /** + * Remove state listener for give path. + * @param path + */ + public void removeStateListener(String path) { + removeListenerInternal(path, WatchType.SINGLE_PATH); + } + + /** + * Add state listener to watch children for given path. + * @param parentPath + * @param listener + */ + public void addChildrenListener(String parentPath, StateListener listener) { + addListenerInternal(parentPath, listener, WatchType.CHILDREN); + } + + /** + * Remove state listener for children on a given parentPath. + * @param parentPath + */ + public void removeChildrenListener(String parentPath) { + removeListenerInternal(parentPath, WatchType.CHILDREN); + } + + /** + * Get latest value attached to the path. + * @param path + * @return byte[] value + * @throws IOException + */ + public byte[] getForPath(String path) throws IOException { + try { + return RetryUtils.retryCallable(String.format("Get for path:%s", path), () -> { + byte[] ret = null; + CompletableFuture getResponse = + getEtcdClient().getKVClient().get(ByteSequence.from(path, StandardCharsets.UTF_8)); + List kvs = getResponse.get(DEFAULT_TIMEOUT_IN_SEC, TimeUnit.SECONDS).getKvs(); + if (!kvs.isEmpty()) { + KeyValue latestKv = Collections.max( + kvs, Comparator.comparing(KeyValue::getModRevision)); + return latestKv.getValue().getBytes(); + } + return ret; + }, new ExponentialBackoffRetry(RETRY_SLEEP_IN_MS, MAX_RETRY_SLEEP_IN_MS, RETRY_TIMES)); + } catch (AlluxioRuntimeException ex) { + throw new IOException(ex.getMessage()); + } + } + + /** + * Check existence of a single given path. + * @param path + * @return if the path exists or not + * @throws IOException + */ + public boolean checkExistsForPath(String path) throws IOException { + try { + return RetryUtils.retryCallable(String.format("Get for path:%s", path), () -> { + boolean exist = false; + try { + CompletableFuture getResponse = + getEtcdClient().getKVClient().get( + ByteSequence.from(path, StandardCharsets.UTF_8)); + List kvs = getResponse.get( + DEFAULT_TIMEOUT_IN_SEC, TimeUnit.SECONDS).getKvs(); + exist = !kvs.isEmpty(); + } catch (ExecutionException | InterruptedException | TimeoutException ex) { + throw new IOException("Error getting path:" + path, ex); + } + return exist; + }, new ExponentialBackoffRetry(RETRY_SLEEP_IN_MS, MAX_RETRY_SLEEP_IN_MS, RETRY_TIMES)); + } catch (AlluxioRuntimeException ex) { + throw new IOException(ex.getMessage()); + } + } + + /** + * Create a path with given value in non-transactional way. + * @param path + * @param value + * @throws IOException + */ + public void createForPath(String path, Optional value) throws IOException { + RetryUtils.retry(String.format("Get for path:%s, value size:%s", + path, (!value.isPresent() ? "null" : value.get().length)), () -> { + try { + mClient.getKVClient().put( + ByteSequence.from(path, StandardCharsets.UTF_8), + ByteSequence.from(value.get())) + .get(DEFAULT_TIMEOUT_IN_SEC, TimeUnit.SECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException ex) { + String errMsg = String.format("Error createForPath:%s", path); + throw new IOException(errMsg, ex); + } + }, new ExponentialBackoffRetry(RETRY_SLEEP_IN_MS, MAX_RETRY_SLEEP_IN_MS, RETRY_TIMES)); + } + + /** + * Delete a path or recursively all paths with given path as prefix. + * @param path + * @param recursive + * @throws IOException + */ + public void deleteForPath(String path, boolean recursive) throws IOException { + RetryUtils.retry(String.format("Delete for path:%s", path), () -> { + try { + mClient.getKVClient().delete( + ByteSequence.from(path, StandardCharsets.UTF_8), + DeleteOption.newBuilder().isPrefix(recursive).build()) + .get(DEFAULT_TIMEOUT_IN_SEC, TimeUnit.SECONDS); + } catch (ExecutionException | InterruptedException | TimeoutException ex) { + String errMsg = String.format("Error deleteForPath:%s", path); + throw new IOException(errMsg, ex); + } + }, new ExponentialBackoffRetry(RETRY_SLEEP_IN_MS, MAX_RETRY_SLEEP_IN_MS, RETRY_TIMES)); + } + + /** + * Remove listener on given path. + * @param path + * @param watchType + */ + public void removeListenerInternal(String path, WatchType watchType) { + Watch.Watcher watcher = mRegisteredWatchers.remove(getRegisterWatcherKey(path, watchType)); + if (watcher == null) { + return; + } + watcher.close(); + } + + /** + * Check if it's connected. + * @return true if this client is connected + */ + public boolean isConnected() { + return mConnected.get(); + } + + /** + * Get the jetcd client instance. + * @return jetcd client + */ + public Client getEtcdClient() { + if (mConnected.get()) { + return mClient; + } + connect(); + return mClient; + } + + @Override + public void close() throws IOException { + if (mClient != null) { + mClient.close(); + } + mCloser.close(); + } +} diff --git a/dora/core/common/src/main/java/alluxio/membership/EtcdMembershipManager.java b/dora/core/common/src/main/java/alluxio/membership/EtcdMembershipManager.java new file mode 100644 index 000000000000..c397500c459d --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/EtcdMembershipManager.java @@ -0,0 +1,209 @@ +/* + * 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.membership; + +import alluxio.conf.AlluxioConfiguration; +import alluxio.conf.PropertyKey; +import alluxio.exception.status.AlreadyExistsException; +import alluxio.wire.WorkerInfo; + +import com.google.common.annotations.VisibleForTesting; +import io.etcd.jetcd.KeyValue; +import org.apache.zookeeper.server.ByteBufferInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * MembershipManager backed by configured etcd cluster. + */ +public class EtcdMembershipManager implements MembershipManager { + private static final Logger LOG = LoggerFactory.getLogger(EtcdMembershipManager.class); + private static final String RING_PATH_FORMAT = "/DHT/%s/AUTHORIZED/"; + private final AlluxioConfiguration mConf; + private AlluxioEtcdClient mAlluxioEtcdClient; + private String mClusterName; + private String mRingPathPrefix = ""; + + /** + * @param conf + * @return EtcdMembershipManager + */ + public static EtcdMembershipManager create(AlluxioConfiguration conf) { + return new EtcdMembershipManager(conf); + } + + /** + * CTOR for EtcdMembershipManager. + * @param conf + */ + public EtcdMembershipManager(AlluxioConfiguration conf) { + this(conf, AlluxioEtcdClient.getInstance(conf)); + } + + /** + * CTOR for EtcdMembershipManager with given AlluxioEtcdClient client. + * @param conf + * @param alluxioEtcdClient + */ + public EtcdMembershipManager(AlluxioConfiguration conf, AlluxioEtcdClient alluxioEtcdClient) { + mConf = conf; + mClusterName = conf.getString(PropertyKey.ALLUXIO_CLUSTER_NAME); + mRingPathPrefix = String.format(RING_PATH_FORMAT, mClusterName); + mAlluxioEtcdClient = alluxioEtcdClient; + } + + @Override + public void join(WorkerInfo wkrAddr) throws IOException { + WorkerServiceEntity entity = new WorkerServiceEntity(wkrAddr.getAddress()); + // 1) register to the ring + String pathOnRing = new StringBuffer() + .append(mRingPathPrefix) + .append(entity.getServiceEntityName()).toString(); + byte[] ret = mAlluxioEtcdClient.getForPath(pathOnRing); + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos)) { + entity.serialize(dos); + byte[] serializedEntity = baos.toByteArray(); + // If there's existing entry, check if it's me. + if (ret != null) { + // It's not me, something is wrong. + if (!Arrays.equals(serializedEntity, ret)) { + throw new AlreadyExistsException( + "Some other member with same id registered on the ring, bail."); + } + // It's me, go ahead to start heartbeating. + } else { + // If haven't created myself onto the ring before, create now. + mAlluxioEtcdClient.createForPath(pathOnRing, Optional.of(serializedEntity)); + } + // 2) start heartbeat + mAlluxioEtcdClient.mServiceDiscovery.registerAndStartSync(entity); + } + } + + @Override + public List getAllMembers() throws IOException { + List registeredWorkers = retrieveFullMembers(); + return registeredWorkers.stream() + .map(e -> new WorkerInfo().setAddress(e.getWorkerNetAddress())) + .collect(Collectors.toList()); + } + + private List retrieveFullMembers() throws IOException { + List fullMembers = new ArrayList<>(); + List childrenKvs = mAlluxioEtcdClient.getChildren(mRingPathPrefix); + for (KeyValue kv : childrenKvs) { + try (ByteArrayInputStream bais = + new ByteArrayInputStream(kv.getValue().getBytes()); + DataInputStream dis = new DataInputStream(bais)) { + WorkerServiceEntity entity = new WorkerServiceEntity(); + entity.deserialize(dis); + fullMembers.add(entity); + } catch (IOException ex) { + // Ignore + } + } + return fullMembers; + } + + private List retrieveLiveMembers() throws IOException { + List liveMembers = new ArrayList<>(); + for (Map.Entry entry : mAlluxioEtcdClient.mServiceDiscovery + .getAllLiveServices().entrySet()) { + try (ByteBufferInputStream bbis = + new ByteBufferInputStream(entry.getValue()); + DataInputStream dis = new DataInputStream(bbis)) { + WorkerServiceEntity entity = new WorkerServiceEntity(); + entity.deserialize(dis); + liveMembers.add(entity); + } catch (IOException ex) { + // Ignore + } + } + return liveMembers; + } + + @Override + @VisibleForTesting + public List getLiveMembers() throws IOException { + List liveWorkers = retrieveLiveMembers(); + return liveWorkers.stream() + .map(e -> new WorkerInfo().setAddress(e.getWorkerNetAddress())) + .collect(Collectors.toList()); + } + + @Override + @VisibleForTesting + public List getFailedMembers() throws IOException { + List registeredWorkers = retrieveFullMembers(); + List liveWorkers = retrieveLiveMembers() + .stream().map(e -> e.getServiceEntityName()) + .collect(Collectors.toList()); + registeredWorkers.removeIf(e -> liveWorkers.contains(e.getServiceEntityName())); + return registeredWorkers.stream() + .map(e -> new WorkerInfo().setAddress(e.getWorkerNetAddress())) + .collect(Collectors.toList()); + } + + @Override + @VisibleForTesting + public String showAllMembers() { + try { + List registeredWorkers = retrieveFullMembers(); + List liveWorkers = retrieveLiveMembers().stream().map( + w -> w.getServiceEntityName()).collect(Collectors.toList()); + String printFormat = "%s\t%s\t%s%n"; + StringBuilder sb = new StringBuilder( + String.format(printFormat, "WorkerId", "Address", "Status")); + for (WorkerServiceEntity entity : registeredWorkers) { + String entryLine = String.format(printFormat, + entity.getServiceEntityName(), + entity.getWorkerNetAddress().getHost() + ":" + + entity.getWorkerNetAddress().getRpcPort(), + liveWorkers.contains(entity.getServiceEntityName()) ? "ONLINE" : "OFFLINE"); + sb.append(entryLine); + } + return sb.toString(); + } catch (IOException ex) { + return String.format("Exception happened:%s", ex.getMessage()); + } + } + + @Override + public void stopHeartBeat(WorkerInfo worker) throws IOException { + WorkerServiceEntity entity = new WorkerServiceEntity(worker.getAddress()); + mAlluxioEtcdClient.mServiceDiscovery.unregisterService(entity.getServiceEntityName()); + } + + @Override + public void decommission(WorkerInfo worker) throws IOException { + // TO BE IMPLEMENTED + } + + @Override + public void close() throws Exception { + mAlluxioEtcdClient.close(); + } +} diff --git a/dora/core/common/src/main/java/alluxio/membership/MembershipManager.java b/dora/core/common/src/main/java/alluxio/membership/MembershipManager.java new file mode 100644 index 000000000000..6ee3cd2b72c7 --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/MembershipManager.java @@ -0,0 +1,132 @@ +/* + * 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.membership; + +import alluxio.conf.AlluxioConfiguration; +import alluxio.conf.PropertyKey; +import alluxio.resource.LockResource; +import alluxio.wire.WorkerInfo; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import javax.annotation.concurrent.GuardedBy; + +/** + * Interface for worker membership management module. + */ +public interface MembershipManager extends AutoCloseable { + + public static final String PATH_SEPARATOR = "/"; + + /** + * An idempotent call to register to join the membership. + * @param worker + * @throws IOException + */ + public void join(WorkerInfo worker) throws IOException; + + /** + * Get all registered worker members. + * @return all registered workers + * @throws IOException + */ + public List getAllMembers() throws IOException; + + /** + * Get healthy workers. + * @return healthy worker list + * @throws IOException + */ + public List getLiveMembers() throws IOException; + + /** + * Get all failed workers. + * @return failed worker list + * @throws IOException + */ + public List getFailedMembers() throws IOException; + + /** + * Pretty printed members and its liveness status. + * @return pretty-printed status string + */ + public String showAllMembers(); + + /** + * Stop heartbeating for liveness for current worker. + * @param worker WorkerInfo + * @throws IOException + */ + @VisibleForTesting + public void stopHeartBeat(WorkerInfo worker) throws IOException; + + /** + * Decommision a worker. + * @param worker WorkerInfo + * @throws IOException + */ + public void decommission(WorkerInfo worker) throws IOException; + + /** + * Factory class to get or create a MembershipManager. + */ + class Factory { + private static final Logger LOG = LoggerFactory.getLogger(Factory.class); + private static final Lock INIT_LOCK = new ReentrantLock(); + @GuardedBy("INIT_LOCK") + private static final AtomicReference MEMBERSHIP_MANAGER = + new AtomicReference<>(); + + /** + * Get or create a MembershipManager instance. + * @param conf + * @return MembershipManager + * @throws IOException + */ + public static MembershipManager get(AlluxioConfiguration conf) throws IOException { + if (MEMBERSHIP_MANAGER.get() == null) { + try (LockResource lockResource = new LockResource(INIT_LOCK)) { + if (MEMBERSHIP_MANAGER.get() == null) { + MEMBERSHIP_MANAGER.set(create(conf)); + } + } catch (IOException e) { + throw e; + } + } + return MEMBERSHIP_MANAGER.get(); + } + + /** + * @param conf the Alluxio configuration + * @return an instance of {@link MembershipManager} + */ + public static MembershipManager create(AlluxioConfiguration conf) throws IOException { + switch (conf.getEnum(PropertyKey.WORKER_MEMBERSHIP_MANAGER_TYPE, MembershipType.class)) { + case STATIC: + return StaticMembershipManager.create(conf); + case ETCD: + return EtcdMembershipManager.create(conf); + case NOOP: + return NoOpMembershipManager.create(); + default: + throw new IllegalStateException("Unrecognized Membership Type"); + } + } + } +} diff --git a/dora/core/common/src/main/java/alluxio/membership/MembershipType.java b/dora/core/common/src/main/java/alluxio/membership/MembershipType.java new file mode 100644 index 000000000000..17e61a817416 --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/MembershipType.java @@ -0,0 +1,21 @@ +/* + * 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.membership; + +/** + * MembershipManager type. + */ +public enum MembershipType { + STATIC, // Use a static file to configure a static member list for MembershipManager + ETCD, // Use etcd for MembershipManager + NOOP // For regression purpose, still leverage Master for worker registration +} diff --git a/dora/core/common/src/main/java/alluxio/membership/NoOpMembershipManager.java b/dora/core/common/src/main/java/alluxio/membership/NoOpMembershipManager.java new file mode 100644 index 000000000000..4dcfeec79ba3 --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/NoOpMembershipManager.java @@ -0,0 +1,74 @@ +/* + * 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.membership; + +import alluxio.wire.WorkerInfo; + +import org.apache.commons.lang3.StringUtils; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** + * No-op membership manager to disable MembershipManager module + * as default for regression purpose. + */ +public class NoOpMembershipManager implements MembershipManager { + + /** + * @return NoOpMembershipManager + */ + public static NoOpMembershipManager create() { + return new NoOpMembershipManager(); + } + + @Override + public void join(WorkerInfo worker) throws IOException { + // NO-OP + } + + @Override + public List getAllMembers() throws IOException { + return Collections.emptyList(); + } + + @Override + public List getLiveMembers() throws IOException { + return Collections.emptyList(); + } + + @Override + public List getFailedMembers() throws IOException { + return Collections.emptyList(); + } + + @Override + public String showAllMembers() { + return StringUtils.EMPTY; + } + + @Override + public void stopHeartBeat(WorkerInfo worker) throws IOException { + // NO OP + } + + @Override + public void decommission(WorkerInfo worker) throws IOException { + // NO OP + } + + @Override + public void close() throws Exception { + // NO OP + } +} diff --git a/dora/core/common/src/main/java/alluxio/membership/ServiceDiscoveryRecipe.java b/dora/core/common/src/main/java/alluxio/membership/ServiceDiscoveryRecipe.java new file mode 100644 index 000000000000..d1e4e9746df9 --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/ServiceDiscoveryRecipe.java @@ -0,0 +1,361 @@ +/* + * 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.membership; + +import alluxio.exception.status.AlreadyExistsException; +import alluxio.exception.status.NotFoundException; +import alluxio.resource.LockResource; +import alluxio.util.ThreadFactoryUtils; + +import com.google.common.base.Preconditions; +import io.etcd.jetcd.ByteSequence; +import io.etcd.jetcd.KeyValue; +import io.etcd.jetcd.Txn; +import io.etcd.jetcd.kv.TxnResponse; +import io.etcd.jetcd.lease.LeaseKeepAliveResponse; +import io.etcd.jetcd.op.Cmp; +import io.etcd.jetcd.op.CmpTarget; +import io.etcd.jetcd.op.Op; +import io.etcd.jetcd.options.GetOption; +import io.etcd.jetcd.options.PutOption; +import io.etcd.jetcd.support.CloseableClient; +import io.grpc.stub.StreamObserver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * ServiceDiscoveryRecipe for etcd, to track health status + * of all registered services. + */ +public class ServiceDiscoveryRecipe { + private static final Logger LOG = LoggerFactory.getLogger(ServiceDiscoveryRecipe.class); + private static final String BASE_PATH = "/ServiceDiscovery"; + final AlluxioEtcdClient mAlluxioEtcdClient; + private final ScheduledExecutorService mExecutor; + private final String mClusterIdentifier; + // Will look like /ServiceDiscovery/ + private final String mRegisterPathPrefix; + private final ConcurrentHashMap mRegisteredServices + = new ConcurrentHashMap<>(); + + /** + * CTOR for ServiceDiscoveryRecipe. + * @param client + * @param clusterIdentifier + */ + public ServiceDiscoveryRecipe(AlluxioEtcdClient client, String clusterIdentifier) { + client.connect(); + mAlluxioEtcdClient = client; + mClusterIdentifier = clusterIdentifier; + mRegisterPathPrefix = String.format("%s%s%s", BASE_PATH, + MembershipManager.PATH_SEPARATOR, mClusterIdentifier); + mExecutor = Executors.newSingleThreadScheduledExecutor( + ThreadFactoryUtils.build("service-discovery-checker", false)); + mExecutor.scheduleWithFixedDelay(this::checkAllForReconnect, + AlluxioEtcdClient.DEFAULT_LEASE_TTL_IN_SEC, AlluxioEtcdClient.DEFAULT_LEASE_TTL_IN_SEC, + TimeUnit.SECONDS); + } + + /** + * Apply for a new lease or extend expired lease for + * given ServiceEntity in atomic fashion. + * Atomicity: + * creation of given ServiceEntity entry on etcd is handled by etcd transaction + * iff the version = 0 which means when there's no such key present. + * (expired lease will automatically delete the kv attached with it on etcd) + * update of the ServiceEntity fields(lease,revision num) is guarded by + * lock within ServiceEntity instance. + * @param service + * @throws IOException + */ + private void newLeaseInternal(ServiceEntity service) throws IOException { + try (LockResource lockResource = new LockResource(service.mLock)) { + if (service.mLease != null && !mAlluxioEtcdClient.isLeaseExpired(service.mLease)) { + LOG.info("Lease attached with service:{} is not expired, bail from here."); + return; + } + String path = service.mServiceEntityName; + String fullPath = new StringBuffer().append(mRegisterPathPrefix) + .append(MembershipManager.PATH_SEPARATOR) + .append(path).toString(); + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + AlluxioEtcdClient.Lease lease = mAlluxioEtcdClient.createLease(); + Txn txn = mAlluxioEtcdClient.getEtcdClient().getKVClient().txn(); + ByteSequence keyToPut = ByteSequence.from(fullPath, StandardCharsets.UTF_8); + DataOutputStream dos = new DataOutputStream(baos); + service.serialize(dos); + ByteSequence valToPut = ByteSequence.from(baos.toByteArray()); + CompletableFuture txnResponseFut = txn + .If(new Cmp(keyToPut, Cmp.Op.EQUAL, CmpTarget.version(0L))) + .Then(Op.put(keyToPut, valToPut, PutOption.newBuilder() + .withLeaseId(lease.mLeaseId).build())) + .Then(Op.get(keyToPut, GetOption.DEFAULT)) + .Else(Op.get(keyToPut, GetOption.DEFAULT)) + .commit(); + TxnResponse txnResponse = txnResponseFut.get(); + List kvs = new ArrayList<>(); + txnResponse.getGetResponses().stream().map( + r -> kvs.addAll(r.getKvs())).collect(Collectors.toList()); + if (!txnResponse.isSucceeded()) { + if (!kvs.isEmpty()) { + throw new AlreadyExistsException("Same service kv pair is there but " + + "attached lease is expired, this should not happen"); + } + throw new IOException("Failed to new a lease for service:" + service.toString()); + } + Preconditions.checkState(!kvs.isEmpty(), "No such service entry found."); + long latestRevision = kvs.stream().mapToLong(kv -> kv.getModRevision()) + .max().getAsLong(); + service.mRevision = latestRevision; + service.mLease = lease; + startHeartBeat(service); + } catch (ExecutionException | InterruptedException ex) { + throw new IOException("Exception in new-ing lease for service:" + service, ex); + } + } + } + + /** + * Register service and start keeping-alive. + * Atomicity: + * So the same-named ServiceEntity registration atomicity on etcd is guaranteed + * in {@link ServiceDiscoveryRecipe#newLeaseInternal(ServiceEntity)}, + * by etcd transaction semantics. We ensure that + * if #newLeaseInternal succeeded, it's safe to track in mRegisteredServices map. + * Other threads within same process or other processes trying to + * register same named service will fail in #newLeaseInternal already. + * @param service + * @throws IOException + */ + public void registerAndStartSync(ServiceEntity service) throws IOException { + LOG.info("registering service : {}", service); + if (mRegisteredServices.containsKey(service.getServiceEntityName())) { + throw new AlreadyExistsException("Service " + service.mServiceEntityName + + " already registered."); + } + newLeaseInternal(service); + ServiceEntity existEntity = mRegisteredServices.putIfAbsent( + service.getServiceEntityName(), service); + if (existEntity != null) { + // We should never reach here as if concurrent new lease creation for service + // on etcd will not succeed for both race parties. + try (ServiceEntity entity = service) { + // someone is already in register service map, close myself before throw exception. + } + throw new AlreadyExistsException("Service " + service.mServiceEntityName + + " already registered."); + } + } + + /** + * Unregister service and close corresponding keepalive client if any. + * @param serviceIdentifier + * @throws IOException + */ + public void unregisterService(String serviceIdentifier) throws IOException { + ServiceEntity entity = mRegisteredServices.remove(serviceIdentifier); + if (entity != null) { + // It is ok to ignore the declared IOException from closing + // removed ServiceEntity from the map. As internal resource + // closing doesn't throw IOException at all. + try (ServiceEntity service = entity) { + LOG.info("Service unregistered:{}", service); + } + } else { + LOG.info("Service already unregistered:{}", serviceIdentifier); + } + } + + /** + * Unregister all services registered from this ServiceDiscoveryRecipe instance. + * [It won't register services registered through other instances(other processes)] + */ + public void unregisterAll() { + for (Map.Entry entry : mRegisteredServices.entrySet()) { + try { + unregisterService(entry.getKey()); + } catch (IOException ex) { + LOG.error("Unregister all services failed unregistering for:{}.", entry.getKey(), ex); + } + } + } + + /** + * Get the registered service value as ByteBuffer. + * @param serviceEntityName + * @return ByteBuffer container serialized content + * @throws IOException + */ + public ByteBuffer getRegisteredServiceDetail(String serviceEntityName) + throws IOException { + String fullPath = new StringBuffer().append(mRegisterPathPrefix) + .append(MembershipManager.PATH_SEPARATOR) + .append(serviceEntityName).toString(); + byte[] val = mAlluxioEtcdClient.getForPath(fullPath); + return ByteBuffer.wrap(val); + } + + /** + * Update the service value with new value. + * TODO(lucy) we need to handle the cases where txn failed bcos of + * lease expiration. + * Atomicity: + * update of given ServiceEntity on etcd is handled by etcd transaction + * on comparing the revision number for a CAS semantic update. + * update of the ServiceEntity fields is guarded by update lock within + * ServiceEntity instance. + * @param service + * @throws IOException + */ + public void updateService(ServiceEntity service) throws IOException { + LOG.info("Updating service : {}", service); + if (!mRegisteredServices.containsKey(service.mServiceEntityName)) { + Preconditions.checkNotNull(service.mLease, "Service not attach with lease"); + throw new NoSuchElementException("Service " + service.mServiceEntityName + + " not registered, please register first."); + } + String fullPath = new StringBuffer().append(mRegisterPathPrefix) + .append(MembershipManager.PATH_SEPARATOR) + .append(service.mServiceEntityName).toString(); + try (LockResource lockResource = new LockResource(service.mLock); + ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + Txn txn = mAlluxioEtcdClient.getEtcdClient().getKVClient().txn(); + ByteSequence keyToPut = ByteSequence.from(fullPath, StandardCharsets.UTF_8); + DataOutputStream dos = new DataOutputStream(baos); + service.serialize(dos); + ByteSequence valToPut = ByteSequence.from(baos.toByteArray()); + CompletableFuture txnResponseFut = txn + .If(new Cmp(keyToPut, Cmp.Op.EQUAL, CmpTarget.modRevision(service.mRevision))) + .Then(Op.put(keyToPut, valToPut, PutOption.newBuilder() + .withLeaseId(service.mLease.mLeaseId).build())) + .Then(Op.get(keyToPut, GetOption.DEFAULT)) + .Else(Op.get(keyToPut, GetOption.DEFAULT)) + .commit(); + TxnResponse txnResponse = txnResponseFut.get(); + List kvs = new ArrayList<>(); + txnResponse.getGetResponses().stream().map( + r -> kvs.addAll(r.getKvs())).collect(Collectors.toList()); + // return if Cmp returns true + if (!txnResponse.isSucceeded()) { + if (kvs.isEmpty()) { + throw new NotFoundException("Such service kv pair is not in etcd anymore."); + } + throw new IOException("Failed to update service:" + service.toString()); + } + // update the service with + long latestRevision = kvs.stream().mapToLong(kv -> kv.getModRevision()) + .max().getAsLong(); + service.mRevision = latestRevision; + if (service.getKeepAliveClient() == null) { + startHeartBeat(service); + } + // This should be a no-op, as the we should not overwrite any other values. + mRegisteredServices.put(service.getServiceEntityName(), service); + } catch (ExecutionException ex) { + throw new IOException("ExecutionException in registering service:" + service, ex); + } catch (InterruptedException ex) { + LOG.info("InterruptedException caught, bail."); + } + } + + /** + * Start heartbeating(keepalive) for the given service. + * @param service + */ + private void startHeartBeat(ServiceEntity service) { + CloseableClient keepAliveClient = mAlluxioEtcdClient.getEtcdClient().getLeaseClient() + .keepAlive(service.mLease.mLeaseId, new RetryKeepAliveObserver(service)); + service.setKeepAliveClient(keepAliveClient); + } + + class RetryKeepAliveObserver implements StreamObserver { + public ServiceEntity mService; + + public RetryKeepAliveObserver(ServiceEntity service) { + mService = service; + } + + @Override + public void onNext(LeaseKeepAliveResponse value) { + // NO-OP + LOG.debug("onNext keepalive response:id:{}:ttl:{}", value.getID(), value.getTTL()); + } + + @Override + public void onError(Throwable t) { + LOG.error("onError for Lease for service:{}, leaseId:{}. Setting status to reconnect", + mService, mService.mLease.mLeaseId, t); + mService.mNeedReconnect.compareAndSet(false, true); + } + + @Override + public void onCompleted() { + LOG.warn("onCompleted for Lease for service:{}, leaseId:{}. Setting status to reconnect", + mService, mService.mLease.mLeaseId); + mService.mNeedReconnect.compareAndSet(false, true); + } + } + + /** + * Get all healthy service list. + * @return return service name to service entity serialized value + */ + public Map getAllLiveServices() throws IOException { + Map ret = new HashMap<>(); + List children = mAlluxioEtcdClient.getChildren(mRegisterPathPrefix); + for (KeyValue kv : children) { + ret.put(kv.getKey().toString(StandardCharsets.UTF_8), + ByteBuffer.wrap(kv.getValue().getBytes())); + } + return ret; + } + + /** + * Periodically check if any ServiceEntity's lease got expired and needs + * to renew the lease with new keepalive client. + */ + private void checkAllForReconnect() { + // No need for lock over all services, just individual ServiceEntity is enough + for (Map.Entry entry : mRegisteredServices.entrySet()) { + ServiceEntity entity = entry.getValue(); + try (LockResource lockResource = new LockResource(entry.getValue().mLock)) { + if (entity.mNeedReconnect.get()) { + try { + LOG.info("Start reconnect for service:{}", entity.getServiceEntityName()); + newLeaseInternal(entity); + entity.mNeedReconnect.set(false); + } catch (IOException e) { + LOG.info("Failed trying to new the lease for service:{}", entity, e); + } + } + } + } + } +} diff --git a/dora/core/common/src/main/java/alluxio/membership/ServiceEntity.java b/dora/core/common/src/main/java/alluxio/membership/ServiceEntity.java new file mode 100644 index 000000000000..2aef2825d5f4 --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/ServiceEntity.java @@ -0,0 +1,103 @@ +/* + * 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.membership; + +import io.etcd.jetcd.support.CloseableClient; + +import java.io.Closeable; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantLock; +import javax.annotation.concurrent.ThreadSafe; + +/** + * Base Entity class including information to register to Etcd + * when using EtcdMembershipManager. + */ +@ThreadSafe +public class ServiceEntity implements Closeable { + private CloseableClient mKeepAliveClient; + // (package visibility) to do keep alive(heartbeating), + // initialized at time of service registration + AlluxioEtcdClient.Lease mLease; + protected String mServiceEntityName; // unique service alias + // revision number of kv pair of registered entity on etcd, used for CASupdate + protected long mRevision; + public final ReentrantLock mLock = new ReentrantLock(); + public AtomicBoolean mNeedReconnect = new AtomicBoolean(false); + + /** + * CTOR for ServiceEntity. + */ + public ServiceEntity() {} + + /** + * CTOR for ServiceEntity with given ServiceEntity name. + * @param serviceEntityName + */ + public ServiceEntity(String serviceEntityName) { + mServiceEntityName = serviceEntityName; + } + + /** + * Get service entity name. + * @return service entity name + */ + public String getServiceEntityName() { + return mServiceEntityName; + } + + /** + * Set keep alive client. + * @param keepAliveClient + */ + public void setKeepAliveClient(CloseableClient keepAliveClient) { + mKeepAliveClient = keepAliveClient; + } + + /** + * Get the keepalive client instance. + * @return jetcd keepalive client + */ + public CloseableClient getKeepAliveClient() { + return mKeepAliveClient; + } + + /** + * Serialize the ServiceEntity to output stream. + * @param dos + * @throws IOException + */ + public void serialize(DataOutputStream dos) throws IOException { + dos.writeUTF(mServiceEntityName); + dos.writeLong(mRevision); + } + + /** + * Deserialize the ServiceEntity from input stream. + * @param dis + * @throws IOException + */ + public void deserialize(DataInputStream dis) throws IOException { + mServiceEntityName = dis.readUTF(); + mRevision = dis.readLong(); + } + + @Override + public void close() throws IOException { + if (mKeepAliveClient != null) { + mKeepAliveClient.close(); + } + } +} diff --git a/dora/core/common/src/main/java/alluxio/membership/StateListener.java b/dora/core/common/src/main/java/alluxio/membership/StateListener.java new file mode 100644 index 000000000000..6c47beffed07 --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/StateListener.java @@ -0,0 +1,30 @@ +/* + * 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.membership; + +/** + * Interface for getting callback on watch event from etcd. + */ +public interface StateListener { + /** + * Act on detecting new put on the key. + * @param newPutKey + * @param newPutValue + */ + public void onNewPut(String newPutKey, byte[] newPutValue); + + /** + * Act on detecting new delete on the key. + * @param newDeleteKey + */ + public void onNewDelete(String newDeleteKey); +} diff --git a/dora/core/common/src/main/java/alluxio/membership/StaticMembershipManager.java b/dora/core/common/src/main/java/alluxio/membership/StaticMembershipManager.java new file mode 100644 index 000000000000..274b1561bcb5 --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/StaticMembershipManager.java @@ -0,0 +1,175 @@ +/* + * 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.membership; + +import alluxio.annotation.SuppressFBWarnings; +import alluxio.cli.CommandUtils; +import alluxio.conf.AlluxioConfiguration; +import alluxio.conf.Configuration; +import alluxio.conf.PropertyKey; +import alluxio.util.HashUtils; +import alluxio.util.network.NetworkAddressUtils; +import alluxio.wire.WorkerInfo; +import alluxio.wire.WorkerNetAddress; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * MembershipManager configured by a static file. + */ +public class StaticMembershipManager implements MembershipManager { + private final List mMembers; + + private final AlluxioConfiguration mConf; + + /** + * @param conf + * @return StaticMembershipManager + * @throws IOException + */ + public static StaticMembershipManager create(AlluxioConfiguration conf) throws IOException { + // user conf/workers, use default port + String workerListFile = conf.getString( + PropertyKey.WORKER_STATIC_MEMBERSHIP_MANAGER_CONFIG_FILE); + List workers = parseWorkerAddresses(workerListFile, conf); + return new StaticMembershipManager(conf, workers); + } + + /** + * CTOR for StaticMembershipManager. + * @param conf + * @throws IOException + */ + @SuppressFBWarnings({"URF_UNREAD_FIELD"}) + StaticMembershipManager(AlluxioConfiguration conf, List members) { + mConf = conf; + mMembers = members; + } + + /** + * Parse the worker addresses from given static config file. + * The static file only gives the hostname, the rest config params + * are inherited from given Configuration or default values. + * @param configFile + * @param conf + * @return list of parsed WorkerInfos + * @throws IOException + */ + private static List parseWorkerAddresses( + String configFile, AlluxioConfiguration conf) throws IOException { + List workerAddrs = new ArrayList<>(); + File file = new File(configFile); + if (!file.exists()) { + throw new FileNotFoundException("Not found for static worker config file:" + configFile); + } + Set workerHostnames = CommandUtils.readNodeList("", configFile); + for (String workerHostname : workerHostnames) { + WorkerNetAddress workerNetAddress = new WorkerNetAddress() + .setHost(workerHostname) + .setContainerHost(Configuration.global() + .getOrDefault(PropertyKey.WORKER_CONTAINER_HOSTNAME, "")) + .setRpcPort(conf.getInt(PropertyKey.WORKER_RPC_PORT)) + .setWebPort(conf.getInt(PropertyKey.WORKER_WEB_PORT)); + //data port, these are initialized from configuration for client to deduce the + //workeraddr related info, on worker side, it will be corrected by join(). + InetSocketAddress inetAddr; + if (Configuration.global().getBoolean(PropertyKey.USER_NETTY_DATA_TRANSMISSION_ENABLED)) { + inetAddr = NetworkAddressUtils.getBindAddress( + NetworkAddressUtils.ServiceType.WORKER_DATA, + Configuration.global()); + workerNetAddress.setNettyDataPort(inetAddr.getPort()); + } else { + inetAddr = NetworkAddressUtils.getConnectAddress( + NetworkAddressUtils.ServiceType.WORKER_RPC, + Configuration.global()); + } + workerNetAddress.setDataPort(inetAddr.getPort()); + workerAddrs.add(workerNetAddress); + } + return workerAddrs.stream() + .map(w -> new WorkerInfo().setAddress(w)).collect(Collectors.toList()); + } + + @Override + public void join(WorkerInfo worker) throws IOException { + // correct with the actual worker addr, + // same settings such as ports will be applied to other members + WorkerNetAddress addr = worker.getAddress(); + mMembers.stream().forEach(m -> m.getAddress() + .setRpcPort(addr.getRpcPort()) + .setDataPort(addr.getDataPort()) + .setDomainSocketPath(addr.getDomainSocketPath()) + .setTieredIdentity(addr.getTieredIdentity()) + .setNettyDataPort(addr.getNettyDataPort()) + .setWebPort(addr.getWebPort()) + .setSecureRpcPort(addr.getSecureRpcPort())); + } + + @Override + public List getAllMembers() throws IOException { + return mMembers; + } + + @Override + public List getLiveMembers() throws IOException { + // No op for static type membership manager + return mMembers; + } + + @Override + public List getFailedMembers() throws IOException { + // No op for static type membership manager + return Collections.emptyList(); + } + + @Override + public String showAllMembers() { + String printFormat = "%s\t%s\t%s%n"; + StringBuilder sb = new StringBuilder( + String.format(printFormat, "WorkerId", "Address", "Status")); + try { + for (WorkerInfo worker : getAllMembers()) { + String entryLine = String.format(printFormat, + HashUtils.hashAsStringMD5(worker.getAddress().dumpMainInfo()), + worker.getAddress().getHost() + ":" + worker.getAddress().getRpcPort(), + "N/A"); + sb.append(entryLine); + } + } catch (IOException ex) { + // IGNORE + } + return sb.toString(); + } + + @Override + public void stopHeartBeat(WorkerInfo worker) throws IOException { + // NOTHING TO DO + } + + @Override + public void decommission(WorkerInfo worker) throws IOException { + mMembers.remove(worker); + } + + @Override + public void close() throws Exception { + // Nothing to close + } +} diff --git a/dora/core/common/src/main/java/alluxio/membership/WorkerServiceEntity.java b/dora/core/common/src/main/java/alluxio/membership/WorkerServiceEntity.java new file mode 100644 index 000000000000..ee2e456264fc --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/membership/WorkerServiceEntity.java @@ -0,0 +1,122 @@ +/* + * 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.membership; + +import alluxio.annotation.SuppressFBWarnings; +import alluxio.grpc.GrpcUtils; +import alluxio.util.HashUtils; +import alluxio.wire.WorkerNetAddress; + +import com.google.common.base.MoreObjects; +import com.google.common.base.Objects; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** + * Entity class including all the information to register to Etcd + * when using EtcdMembershipManager. + */ +public class WorkerServiceEntity extends ServiceEntity { + /** + * Membership state of the worker. + */ + enum State { + JOINED, + AUTHORIZED, + DECOMMISSIONED + } + + WorkerNetAddress mAddress; + State mState = State.JOINED; + @SuppressFBWarnings({"URF_UNREAD_FIELD"}) + int mGenerationNum = -1; + + /** + * CTOR for WorkerServiceEntity. + */ + public WorkerServiceEntity() { + } + + /** + * CTOR for WorkerServiceEntity with given WorkerNetAddress. + * @param addr + */ + public WorkerServiceEntity(WorkerNetAddress addr) { + super(HashUtils.hashAsStringMD5(addr.dumpMainInfo())); + mAddress = addr; + mState = State.AUTHORIZED; + } + + /** + * Get WorkerNetAddress field. + * @return WorkerNetAddress + */ + public WorkerNetAddress getWorkerNetAddress() { + return mAddress; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("WorkerId", getServiceEntityName()) + .add("WorkerAddr", mAddress.toString()) + .add("State", mState.toString()) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WorkerServiceEntity anotherO = (WorkerServiceEntity) o; + return mAddress.equals(anotherO.mAddress) + && getServiceEntityName().equals(anotherO.getServiceEntityName()); + } + + @Override + public int hashCode() { + return Objects.hashCode(mAddress, mServiceEntityName); + } + + /** + * Serialize the WorkerServiceEntity object. + * @param dos + * @throws IOException + */ + public void serialize(DataOutputStream dos) throws IOException { + super.serialize(dos); + dos.writeInt(mState.ordinal()); + byte[] serializedArr = GrpcUtils.toProto(mAddress).toByteArray(); + dos.writeInt(serializedArr.length); + dos.write(serializedArr); + } + + /** + * Deserialize to WorkerServiceEntity object. + * @param dis + * @throws IOException + */ + public void deserialize(DataInputStream dis) throws IOException { + super.deserialize(dis); + mState = State.values()[dis.readInt()]; + int byteArrLen = dis.readInt(); + byte[] byteArr = new byte[byteArrLen]; + dis.read(byteArr, 0, byteArrLen); + mAddress = GrpcUtils.fromProto(alluxio.grpc.WorkerNetAddress.parseFrom(byteArr)); + } +} diff --git a/dora/core/common/src/main/java/alluxio/util/HashUtils.java b/dora/core/common/src/main/java/alluxio/util/HashUtils.java new file mode 100644 index 000000000000..e3760f27e4b8 --- /dev/null +++ b/dora/core/common/src/main/java/alluxio/util/HashUtils.java @@ -0,0 +1,58 @@ +/* + * 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.util; + +import static com.google.common.hash.Hashing.murmur3_32_fixed; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.hash.HashFunction; +import org.apache.commons.codec.binary.Hex; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import javax.annotation.concurrent.ThreadSafe; + +/** + * Util class for hashing. + */ +@ThreadSafe +public class HashUtils { + + private static final HashFunction HASH_FUNCTION = murmur3_32_fixed(); + + private HashUtils() {} // prevent instantiation + + /** + * MD5 Hash the given obj as string. + * @param object + * @return hash in string + */ + public static String hashAsStringMD5(String object) { + try { + MessageDigest md = MessageDigest.getInstance("MD5"); + md.update(object.getBytes()); + return Hex.encodeHexString(md.digest()).toLowerCase(); + } catch (NoSuchAlgorithmException e) { + /* No actions. Continue with other hash method. */ + } + return HASH_FUNCTION.hashString(object, UTF_8).toString(); + } + + /** + * Hash the give obj as long with given HASH_FUNCTION. + * @param object + * @return hash in long + */ + public static long hashAsLong(String object) { + return HASH_FUNCTION.hashString(object, UTF_8).padToLong(); + } +} diff --git a/dora/core/server/master/src/main/java/alluxio/master/scheduler/DefaultWorkerProvider.java b/dora/core/server/master/src/main/java/alluxio/master/scheduler/DefaultWorkerProvider.java index 3d7f623999ab..27b2f178968f 100644 --- a/dora/core/server/master/src/main/java/alluxio/master/scheduler/DefaultWorkerProvider.java +++ b/dora/core/server/master/src/main/java/alluxio/master/scheduler/DefaultWorkerProvider.java @@ -54,6 +54,11 @@ public List getWorkerInfos() { } } + @Override + public List getLiveWorkerInfos() { + return getWorkerInfos(); + } + @Override public CloseableResource getWorkerClient(WorkerNetAddress address) { try { diff --git a/dora/core/server/master/src/main/java/alluxio/master/scheduler/MembershipManagerWorkerProvider.java b/dora/core/server/master/src/main/java/alluxio/master/scheduler/MembershipManagerWorkerProvider.java new file mode 100644 index 000000000000..bf09b120a4f1 --- /dev/null +++ b/dora/core/server/master/src/main/java/alluxio/master/scheduler/MembershipManagerWorkerProvider.java @@ -0,0 +1,70 @@ +/* + * 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.master.scheduler; + +import alluxio.client.block.stream.BlockWorkerClient; +import alluxio.client.file.FileSystemContext; +import alluxio.exception.runtime.AlluxioRuntimeException; +import alluxio.membership.MembershipManager; +import alluxio.resource.CloseableResource; +import alluxio.scheduler.job.WorkerProvider; +import alluxio.wire.WorkerInfo; +import alluxio.wire.WorkerNetAddress; + +import java.io.IOException; +import java.util.List; + +/** + * MembershipManager backed WorkerProvider for Scheduler. + */ +public class MembershipManagerWorkerProvider implements WorkerProvider { + private final MembershipManager mMembershipManager; + private final FileSystemContext mContext; + + /** + * CTOR for MembershipManagerWorkerProvider. + * @param membershipMgr + * @param context + */ + public MembershipManagerWorkerProvider(MembershipManager membershipMgr, + FileSystemContext context) { + mMembershipManager = membershipMgr; + mContext = context; + } + + @Override + public List getWorkerInfos() { + try { + return mMembershipManager.getAllMembers(); + } catch (IOException ex) { + throw AlluxioRuntimeException.from(ex); + } + } + + @Override + public List getLiveWorkerInfos() { + try { + return mMembershipManager.getLiveMembers(); + } catch (IOException ex) { + throw AlluxioRuntimeException.from(ex); + } + } + + @Override + public CloseableResource getWorkerClient(WorkerNetAddress address) { + try { + return mContext.acquireBlockWorkerClient(address); + } catch (IOException e) { + throw AlluxioRuntimeException.from(e); + } + } +} diff --git a/dora/core/server/worker/src/main/java/alluxio/worker/dora/PagedDoraWorker.java b/dora/core/server/worker/src/main/java/alluxio/worker/dora/PagedDoraWorker.java index 5ce9f6ca8ae2..40484bee81be 100644 --- a/dora/core/server/worker/src/main/java/alluxio/worker/dora/PagedDoraWorker.java +++ b/dora/core/server/worker/src/main/java/alluxio/worker/dora/PagedDoraWorker.java @@ -55,6 +55,8 @@ import alluxio.heartbeat.HeartbeatContext; import alluxio.heartbeat.HeartbeatExecutor; import alluxio.heartbeat.HeartbeatThread; +import alluxio.membership.MembershipManager; +import alluxio.membership.NoOpMembershipManager; import alluxio.network.protocol.databuffer.PooledDirectNioByteBuf; import alluxio.proto.dataserver.Protocol; import alluxio.proto.meta.DoraMeta; @@ -74,9 +76,11 @@ import alluxio.underfs.options.DeleteOptions; import alluxio.underfs.options.MkdirsOptions; import alluxio.util.CommonUtils; +import alluxio.util.HashUtils; import alluxio.util.ModeUtils; import alluxio.util.executor.ExecutorServiceFactories; import alluxio.wire.FileInfo; +import alluxio.wire.WorkerInfo; import alluxio.wire.WorkerNetAddress; import alluxio.worker.AbstractWorker; import alluxio.worker.block.BlockMasterClient; @@ -123,10 +127,12 @@ public class PagedDoraWorker extends AbstractWorker implements DoraWorker { // and assumes all UFS paths belong to the same UFS. private static final int MOUNT_POINT = 1; private final Closer mResourceCloser = Closer.create(); + // TODO(lucy) change to string typed once membership manager got enabled by default private final AtomicReference mWorkerId; private final CacheManager mCacheManager; private final DoraUfsManager mUfsManager; private final DoraMetaManager mMetaManager; + private final MembershipManager mMembershipManager; private final UfsInputStreamCache mUfsStreamCache; private final long mPageSize; private final AlluxioConfiguration mConf; @@ -150,13 +156,16 @@ public class PagedDoraWorker extends AbstractWorker implements DoraWorker { * @param workerId * @param conf * @param cacheManager + * @param membershipManager */ @Inject public PagedDoraWorker( @Named("workerId") AtomicReference workerId, AlluxioConfiguration conf, - CacheManager cacheManager) { - this(workerId, conf, cacheManager, new BlockMasterClientPool(), + CacheManager cacheManager, + MembershipManager membershipManager + ) { + this(workerId, conf, cacheManager, membershipManager, new BlockMasterClientPool(), FileSystemContext.create(conf)); } @@ -164,6 +173,7 @@ protected PagedDoraWorker( AtomicReference workerId, AlluxioConfiguration conf, CacheManager cacheManager, + MembershipManager membershipManager, BlockMasterClientPool blockMasterClientPool, FileSystemContext fileSystemContext) { super(ExecutorServiceFactories.fixedThreadPool("dora-worker-executor", 5)); @@ -182,6 +192,7 @@ protected PagedDoraWorker( mCacheManager = cacheManager; mMetaManager = mResourceCloser.register( new DoraMetaManager(this, mCacheManager, mUfs)); + mMembershipManager = membershipManager; mOpenFileHandleContainer = new DoraOpenFileHandleContainer(); mMkdirsRecursive = MkdirsOptions.defaults(mConf).setCreateParent(true); @@ -217,16 +228,50 @@ public void start(WorkerNetAddress address) throws IOException { // the heartbeat is only used to notify the aliveness of this worker, so that clients // can get the latest worker list from master. // TODO(bowen): once we set up a worker discovery service in place of master, remove this - getExecutorService() - .submit(new HeartbeatThread(HeartbeatContext.WORKER_BLOCK_SYNC, - mResourceCloser.register(new BlockMasterSync()), - () -> new FixedIntervalSupplier(Configuration.getMs( - PropertyKey.WORKER_BLOCK_HEARTBEAT_INTERVAL_MS)), - mConf, ServerUserState.global())); + // TODO(lucy): temporary fallback logic during transition of removing master dependency + if (mMembershipManager instanceof NoOpMembershipManager) { + getExecutorService() + .submit(new HeartbeatThread(HeartbeatContext.WORKER_BLOCK_SYNC, + mResourceCloser.register(new BlockMasterSync()), + () -> new FixedIntervalSupplier(Configuration.getMs( + PropertyKey.WORKER_BLOCK_HEARTBEAT_INTERVAL_MS)), + mConf, ServerUserState.global())); + } } + /** + * Register to join to the distributed membership. + * @throws IOException + */ private void register() throws IOException { - Preconditions.checkState(mAddress != null, "worker not started"); + Preconditions.checkNotNull(mAddress, "worker not started"); + RetryPolicy retry = RetryUtils.defaultWorkerMasterClientRetry(); + // For regression purpose, use the original way of regsiter + if (mMembershipManager instanceof NoOpMembershipManager) { + registerToMaster(); + return; + } + while (true) { + try (PooledResource bmc = mBlockMasterClientPool.acquireCloseable()) { + // TODO(lucy) this is necessary here for MASTER web to be opened for some reason + bmc.get().connect(); + mMembershipManager.join(new WorkerInfo().setAddress(mAddress)); + mWorkerId.set(HashUtils.hashAsLong(mAddress.dumpMainInfo())); + break; + } catch (IOException ioe) { + if (!retry.attempt()) { + throw ioe; + } + } + } + } + + private void decommission() { + // TO BE IMPLEMENTED + } + + private void registerToMaster() throws IOException { + Preconditions.checkNotNull(mAddress, "worker not started"); RetryPolicy retry = RetryUtils.defaultWorkerMasterClientRetry(); while (true) { try (PooledResource bmc = mBlockMasterClientPool.acquireCloseable()) { @@ -242,7 +287,6 @@ private void register() throws IOException { ImmutableMap.of(), Configuration.getConfiguration(Scope.WORKER)); LOG.info("Worker registered with worker ID: {}", mWorkerId.get()); - break; } catch (IOException ioe) { if (!retry.attempt()) { @@ -261,7 +305,8 @@ public void stop() throws IOException { @Override public void close() throws IOException { try (AutoCloseable ignoredCloser = mResourceCloser; - AutoCloseable ignoredCacheManager = mCacheManager + AutoCloseable ignoredCacheManager = mCacheManager; + AutoCloseable ignoredMembershipManager = mMembershipManager; ) { // do nothing as we are closing } catch (Exception e) { diff --git a/dora/core/server/worker/src/main/java/alluxio/worker/modules/DoraWorkerModule.java b/dora/core/server/worker/src/main/java/alluxio/worker/modules/DoraWorkerModule.java index b38ce8bd7db5..8f9bb7d91993 100644 --- a/dora/core/server/worker/src/main/java/alluxio/worker/modules/DoraWorkerModule.java +++ b/dora/core/server/worker/src/main/java/alluxio/worker/modules/DoraWorkerModule.java @@ -19,6 +19,7 @@ import alluxio.conf.Configuration; import alluxio.conf.PropertyKey; import alluxio.master.MasterClientContext; +import alluxio.membership.MembershipManager; import alluxio.network.TieredIdentityFactory; import alluxio.underfs.UfsManager; import alluxio.wire.TieredIdentity; @@ -72,6 +73,14 @@ protected void configure() { throw new RuntimeException(e); } }).in(Scopes.SINGLETON); + bind(MembershipManager.class).toProvider(() -> + { + try { + return MembershipManager.Factory.create(Configuration.global()); + } catch (IOException e) { + throw new RuntimeException(e); + } + }).in(Scopes.SINGLETON); long pageSize = Configuration.global().getBytes(PropertyKey.WORKER_PAGE_STORE_PAGE_SIZE); bind(new TypeLiteral() { diff --git a/dora/core/server/worker/src/test/java/alluxio/worker/dora/PagedDoraWorkerTest.java b/dora/core/server/worker/src/test/java/alluxio/worker/dora/PagedDoraWorkerTest.java index a830dc868bec..34c0ef24eeac 100644 --- a/dora/core/server/worker/src/test/java/alluxio/worker/dora/PagedDoraWorkerTest.java +++ b/dora/core/server/worker/src/test/java/alluxio/worker/dora/PagedDoraWorkerTest.java @@ -40,6 +40,7 @@ import alluxio.grpc.SetAttributePOptions; import alluxio.grpc.UfsReadOptions; import alluxio.grpc.WriteOptions; +import alluxio.membership.MembershipManager; import alluxio.security.authorization.Mode; import alluxio.underfs.UfsStatus; import alluxio.util.io.BufferUtils; @@ -72,6 +73,7 @@ public class PagedDoraWorkerTest { @Rule public TemporaryFolder mTestFolder = new TemporaryFolder(); private CacheManager mCacheManager; + private MembershipManager mMembershipManager; private final long mPageSize = Configuration.global().getBytes(PropertyKey.WORKER_PAGE_STORE_PAGE_SIZE); private static final GetStatusPOptions GET_STATUS_OPTIONS_MUST_SYNC = @@ -89,7 +91,10 @@ public void before() throws Exception { PageMetaStore.create(CacheManagerOptions.createForWorker(Configuration.global())); mCacheManager = CacheManager.Factory.create(Configuration.global(), cacheManagerOptions, pageMetaStore); - mWorker = new PagedDoraWorker(new AtomicReference<>(1L), Configuration.global(), mCacheManager); + mMembershipManager = + MembershipManager.Factory.create(Configuration.global()); + mWorker = new PagedDoraWorker(new AtomicReference<>(1L), + Configuration.global(), mCacheManager, mMembershipManager); } @After diff --git a/dora/job/common/src/main/java/alluxio/scheduler/job/WorkerProvider.java b/dora/job/common/src/main/java/alluxio/scheduler/job/WorkerProvider.java index cf587462b720..efa3738e6125 100644 --- a/dora/job/common/src/main/java/alluxio/scheduler/job/WorkerProvider.java +++ b/dora/job/common/src/main/java/alluxio/scheduler/job/WorkerProvider.java @@ -32,6 +32,12 @@ public interface WorkerProvider { */ List getWorkerInfos(); + /** + * Get live workerInfo list. + * @return list of WorkerInfos who are alive + */ + List getLiveWorkerInfos(); + /** * Gets a worker client. * diff --git a/dora/minicluster/src/main/java/alluxio/multi/process/MultiProcessCluster.java b/dora/minicluster/src/main/java/alluxio/multi/process/MultiProcessCluster.java index 04a020b334a2..9bf685f2ad72 100644 --- a/dora/minicluster/src/main/java/alluxio/multi/process/MultiProcessCluster.java +++ b/dora/minicluster/src/main/java/alluxio/multi/process/MultiProcessCluster.java @@ -42,6 +42,7 @@ import alluxio.master.SingleMasterInquireClient; import alluxio.master.ZkMasterInquireClient; import alluxio.master.journal.JournalType; +import alluxio.membership.MembershipType; import alluxio.multi.process.PortCoordination.ReservedPort; import alluxio.security.user.ServerUserState; import alluxio.util.CommonUtils; @@ -740,6 +741,8 @@ private synchronized Worker createWorker(int i) throws IOException { conf.put(PropertyKey.MASTER_WORKER_REGISTER_LEASE_ENABLED, false); conf.put(PropertyKey.USER_NETTY_DATA_TRANSMISSION_ENABLED, true); + Configuration.set(PropertyKey.WORKER_MEMBERSHIP_MANAGER_TYPE, MembershipType.NOOP); + Worker worker = mCloser.register(new Worker(logsDir, conf)); mWorkers.add(worker); LOG.info("Created worker with (rpc, data, web) ports ({}, {}, {})", rpcPort, dataPort, diff --git a/dora/shell/src/main/java/alluxio/cli/fsadmin/command/ReportCommand.java b/dora/shell/src/main/java/alluxio/cli/fsadmin/command/ReportCommand.java index 28127f8400bb..11e2d8e48100 100644 --- a/dora/shell/src/main/java/alluxio/cli/fsadmin/command/ReportCommand.java +++ b/dora/shell/src/main/java/alluxio/cli/fsadmin/command/ReportCommand.java @@ -17,6 +17,7 @@ import alluxio.cli.fsadmin.report.CapacityCommand; import alluxio.cli.fsadmin.report.JobServiceMetricsCommand; import alluxio.cli.fsadmin.report.MetricsCommand; +import alluxio.cli.fsadmin.report.NodeStatusCommand; import alluxio.cli.fsadmin.report.ProxyCommand; import alluxio.cli.fsadmin.report.SummaryCommand; import alluxio.cli.fsadmin.report.UfsCommand; @@ -83,7 +84,8 @@ enum Command { SUMMARY, // Report cluster summary UFS, // Report under filesystem information JOBSERVICE, // Report job service metrics information - PROXY // Report proxy information in the cluster + PROXY, // Report proxy information in the cluster + NODESTATUS // Report node status - current for workers } private AlluxioConfiguration mConf; @@ -138,6 +140,9 @@ public int run(CommandLine cl) throws IOException { case "proxy": command = Command.PROXY; break; + case "nodestatus": + command = Command.NODESTATUS; + break; default: System.out.println(getUsage()); System.out.println(getDescription()); @@ -182,6 +187,10 @@ public int run(CommandLine cl) throws IOException { ProxyCommand proxyCommand = new ProxyCommand(mMetaClient, mPrintStream); proxyCommand.run(); break; + case NODESTATUS: + NodeStatusCommand nodeStatusCommand = new NodeStatusCommand(mConf, mPrintStream); + nodeStatusCommand.run(cl); + break; default: break; } @@ -229,7 +238,8 @@ public static String description() { + " metrics metrics information\n" + " summary cluster summary\n" + " ufs under storage system information\n" - + " jobservice job service metrics information\n"; + + " jobservice job service metrics information\n" + + " nodestatus node status [worker as of now]\n"; } @Override diff --git a/dora/shell/src/main/java/alluxio/cli/fsadmin/report/NodeStatusCommand.java b/dora/shell/src/main/java/alluxio/cli/fsadmin/report/NodeStatusCommand.java new file mode 100644 index 000000000000..583d260a6b0d --- /dev/null +++ b/dora/shell/src/main/java/alluxio/cli/fsadmin/report/NodeStatusCommand.java @@ -0,0 +1,50 @@ +/* + * 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.cli.fsadmin.report; + +import alluxio.conf.AlluxioConfiguration; +import alluxio.membership.MembershipManager; + +import org.apache.commons.cli.CommandLine; + +import java.io.IOException; +import java.io.PrintStream; + +/** + * Command to get node status. + */ +public class NodeStatusCommand { + + private AlluxioConfiguration mConf; + private PrintStream mPrintStream; + + /** + * CTOR for NodeStatusCommand. + * @param conf + * @param printStream + */ + public NodeStatusCommand(AlluxioConfiguration conf, PrintStream printStream) { + mConf = conf; + mPrintStream = printStream; + } + + /** + * Runs a proxy report command. + * @param cl + * @return 0 on success, 1 otherwise + */ + public int run(CommandLine cl) throws IOException { + MembershipManager memberMgr = MembershipManager.Factory.create(mConf); + mPrintStream.println(memberMgr.showAllMembers()); + return 0; + } +} diff --git a/dora/tests/src/test/java/alluxio/client/cli/fsadmin/command/DoctorCommandIntegrationTest.java b/dora/tests/src/test/java/alluxio/client/cli/fsadmin/command/DoctorCommandIntegrationTest.java index 5042e2b2ef7d..77ca12b9f417 100644 --- a/dora/tests/src/test/java/alluxio/client/cli/fsadmin/command/DoctorCommandIntegrationTest.java +++ b/dora/tests/src/test/java/alluxio/client/cli/fsadmin/command/DoctorCommandIntegrationTest.java @@ -15,11 +15,13 @@ import alluxio.client.cli.fsadmin.AbstractFsAdminShellTest; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Test; /** * Tests for doctor command. */ +@Ignore public final class DoctorCommandIntegrationTest extends AbstractFsAdminShellTest { @Test public void masterNotRunning() throws Exception { diff --git a/dora/tests/src/test/java/alluxio/client/cli/fsadmin/command/QuorumCommandIntegrationTest.java b/dora/tests/src/test/java/alluxio/client/cli/fsadmin/command/QuorumCommandIntegrationTest.java index 191903638cf2..25568501d3e6 100644 --- a/dora/tests/src/test/java/alluxio/client/cli/fsadmin/command/QuorumCommandIntegrationTest.java +++ b/dora/tests/src/test/java/alluxio/client/cli/fsadmin/command/QuorumCommandIntegrationTest.java @@ -36,6 +36,7 @@ import org.junit.After; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -48,6 +49,7 @@ /** * Integration tests for the embedded journal. */ +@Ignore public final class QuorumCommandIntegrationTest extends BaseIntegrationTest { @Rule public ConfigurationRule mConf = new ConfigurationRule( diff --git a/dora/tests/src/test/java/alluxio/server/configuration/ConfigCheckerIntegrationTest.java b/dora/tests/src/test/java/alluxio/server/configuration/ConfigCheckerIntegrationTest.java index 6f17d02e4b6e..99fd04917e54 100644 --- a/dora/tests/src/test/java/alluxio/server/configuration/ConfigCheckerIntegrationTest.java +++ b/dora/tests/src/test/java/alluxio/server/configuration/ConfigCheckerIntegrationTest.java @@ -102,6 +102,7 @@ public void multiMastersEmbeddedHA() throws Exception { } @Test + @Ignore public void multiWorkers() throws Exception { PropertyKey key = PropertyKey.WORKER_FREE_SPACE_TIMEOUT; Map> workerProperties diff --git a/dora/tests/src/test/java/alluxio/server/worker/WorkerMetadataSyncIntegrationTest.java b/dora/tests/src/test/java/alluxio/server/worker/WorkerMetadataSyncIntegrationTest.java index fb2687a59c21..e80d9d9f220c 100644 --- a/dora/tests/src/test/java/alluxio/server/worker/WorkerMetadataSyncIntegrationTest.java +++ b/dora/tests/src/test/java/alluxio/server/worker/WorkerMetadataSyncIntegrationTest.java @@ -22,6 +22,7 @@ import alluxio.testutils.LocalAlluxioClusterResource; import alluxio.util.WaitForOptions; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; @@ -44,7 +45,9 @@ public class WorkerMetadataSyncIntegrationTest { .build(), Configuration.modifiableGlobal()); + /* Not applied as registration is not going thru master any more */ @Test + @Ignore public void reRegisterWorker() throws Exception { mLocalAlluxioClusterResource.start(); @@ -61,7 +64,9 @@ public void reRegisterWorker() throws Exception { () -> master.getWorkerCount() == 1, WaitForOptions.defaults().setTimeoutMs(2000)); } + /* Not applied as registration is not going thru master any more */ @Test + @Ignore public void acquireLeaseNoStreaming() throws Exception { // test that registration works when lease is enabled and streaming is disabled mConfigurationRule.set(PropertyKey.WORKER_REGISTER_LEASE_ENABLED, true); diff --git a/pom.xml b/pom.xml index 3877bdd136ac..e1d38506162d 100644 --- a/pom.xml +++ b/pom.xml @@ -387,6 +387,11 @@ libcephfs ${libcephfs.version} + + io.etcd + jetcd-core + 0.7.5 + io.grpc grpc-api From facdff55caaa3f1b7e955d7a544188a689b083eb Mon Sep 17 00:00:00 2001 From: Huang Hua Date: Wed, 2 Aug 2023 17:53:14 +0800 Subject: [PATCH 12/12] Process the mount point from user input ### What changes are proposed in this pull request? e.g. removing trailing "/" at the end or in the middle. checking existance of parent dir and mount point ### Why are the changes needed? If the mount point is not processed, it may contains excessive "/" at the end or in the middle. But the 'mount' command will output mount point without those excessive "/". So, the script cannot determine if the mount is succeeded or not. ### Does this PR introduce any user facing changes? N/A pr-link: Alluxio/alluxio#17857 change-id: cid-cab07bfd4ac91f52248c9e51b58bf7cf1fa17733 --- dora/integration/fuse/bin/alluxio-fuse | 38 ++++++++++++++++++++++++-- 1 file changed, 36 insertions(+), 2 deletions(-) diff --git a/dora/integration/fuse/bin/alluxio-fuse b/dora/integration/fuse/bin/alluxio-fuse index c0b8c247d8f0..54ec59677ff4 100755 --- a/dora/integration/fuse/bin/alluxio-fuse +++ b/dora/integration/fuse/bin/alluxio-fuse @@ -127,6 +127,38 @@ print_mount_status() { fi } +####################################### +# Get Normalized Mount Point +# e.g. remove trailing "/" from the end or remove excessive "/" in the middle. +# and check existance of parent dir and mount point. +# +# The following inputs should translate to "/home/work/alluxio_fuse" or report error. +# /home/work/alluxio_fuse/// +# /home//work/alluxio_fuse/// +# /home/work///alluxio_fuse/// +# ////home/work/alluxio_fuse/// +# ///////home/work/alluxio_fuse/// +# /home/////////work/alluxio_fuse/// +# //hoooome/work////alluxio_fuse (parent dir does not exist) +# //home/work////alluxio_fuse_not_exist/ (mount point does not exist) +####################################### +get_mount_point() { + local MP_NAME=$(basename $1) + local DIR_NAME=$(dirname $1) + local OLDPWD="$(pwd)" + cd "$DIR_NAME" || { + err "Dir '$DIR_NAME' does not exist" + exit 1 + } + DIR_NAME="$(pwd)" + [ -d "$DIR_NAME/$MP_NAME" ] || { + err "Mount point '$DIR_NAME/$MP_NAME' does not exist" + exit 1 + } + cd "$OLDPWD" + export FUSE_MOUNT_POINT="$DIR_NAME/$MP_NAME" +} + ####################################### # Launch the Alluxio FUSE process. # Globals: @@ -145,7 +177,8 @@ launch_fuse_process() { return 1 fi declare -r ufs_address="$1" - declare -r mount_point="$2" + get_mount_point "$2" + declare -r mount_point=$FUSE_MOUNT_POINT shift 2 local mount_options="" @@ -277,7 +310,8 @@ unmount_command() { echo -e "${USAGE}" return 1 fi - declare -r mount_point="$1" + get_mount_point "$1" + declare -r mount_point=$FUSE_MOUNT_POINT shift local force_kill='false'