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.hdfs.client; 019 020import java.io.IOException; 021import java.util.EnumSet; 022 023import org.apache.hadoop.classification.InterfaceAudience; 024import org.apache.hadoop.classification.InterfaceStability; 025import org.apache.hadoop.fs.FSDataOutputStream; 026import org.apache.hadoop.fs.FileSystem; 027import org.apache.hadoop.hdfs.DFSOutputStream; 028 029/** 030 * The Hdfs implementation of {@link FSDataOutputStream}. 031 */ 032@InterfaceAudience.Public 033@InterfaceStability.Evolving 034public class HdfsDataOutputStream extends FSDataOutputStream { 035 public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats, 036 long startPosition) throws IOException { 037 super(out, stats, startPosition); 038 } 039 040 public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats 041 ) throws IOException { 042 this(out, stats, 0L); 043 } 044 045 /** 046 * Get the actual number of replicas of the current block. 047 * 048 * This can be different from the designated replication factor of the file 049 * because the namenode does not maintain replication for the blocks which are 050 * currently being written to. Depending on the configuration, the client may 051 * continue to write to a block even if a few datanodes in the write pipeline 052 * have failed, or the client may add a new datanodes once a datanode has 053 * failed. 054 * 055 * @return the number of valid replicas of the current block 056 */ 057 public synchronized int getCurrentBlockReplication() throws IOException { 058 return ((DFSOutputStream)getWrappedStream()).getCurrentBlockReplication(); 059 } 060 061 /** 062 * Sync buffered data to DataNodes (flush to disk devices). 063 * 064 * @param syncFlags 065 * Indicate the detailed semantic and actions of the hsync. 066 * @throws IOException 067 * @see FSDataOutputStream#hsync() 068 */ 069 public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException { 070 ((DFSOutputStream) getWrappedStream()).hsync(syncFlags); 071 } 072 073 public static enum SyncFlag { 074 /** 075 * When doing sync to DataNodes, also update the metadata (block 076 * length) in the NameNode 077 */ 078 UPDATE_LENGTH; 079 } 080}