001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io.asyncfs;
019
020import java.io.IOException;
021
022import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
023import org.apache.hadoop.fs.FSDataOutputStream;
024import org.apache.hadoop.fs.FileSystem;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.util.CommonFSUtils;
027import org.apache.hadoop.hdfs.DistributedFileSystem;
028import org.apache.yetus.audience.InterfaceAudience;
029
030import org.apache.hbase.thirdparty.io.netty.channel.Channel;
031import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
032
033/**
034 * Helper class for creating AsyncFSOutput.
035 */
036@InterfaceAudience.Private
037public final class AsyncFSOutputHelper {
038
039  private AsyncFSOutputHelper() {
040  }
041
042  /**
043   * Create {@link FanOutOneBlockAsyncDFSOutput} for {@link DistributedFileSystem}, and a simple
044   * implementation for other {@link FileSystem} which wraps around a {@link FSDataOutputStream}.
045   */
046  public static AsyncFSOutput createOutput(FileSystem fs, Path f, boolean overwrite,
047      boolean createParent, short replication, long blockSize, EventLoopGroup eventLoopGroup,
048      Class<? extends Channel> channelClass)
049      throws IOException, CommonFSUtils.StreamLacksCapabilityException {
050    if (fs instanceof DistributedFileSystem) {
051      return FanOutOneBlockAsyncDFSOutputHelper.createOutput((DistributedFileSystem) fs, f,
052        overwrite, createParent, replication, blockSize, eventLoopGroup, channelClass);
053    }
054    final FSDataOutputStream out;
055    int bufferSize = fs.getConf().getInt(CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
056      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
057    // This is not a Distributed File System, so it won't be erasure coded; no builder API needed
058    if (createParent) {
059      out = fs.create(f, overwrite, bufferSize, replication, blockSize, null);
060    } else {
061      out = fs.createNonRecursive(f, overwrite, bufferSize, replication, blockSize, null);
062    }
063    // After we create the stream but before we attempt to use it at all
064    // ensure that we can provide the level of data safety we're configured
065    // to provide.
066    if (fs.getConf().getBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, true) &&
067      !(CommonFSUtils.hasCapability(out, "hflush") &&
068        CommonFSUtils.hasCapability(out, "hsync"))) {
069      out.close();
070      throw new CommonFSUtils.StreamLacksCapabilityException("hflush and hsync");
071    }
072    return new WrapperAsyncFSOutput(f, out);
073  }
074}