Github user kunal642 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2406#discussion_r200553686
  
    --- Diff: 
core/src/main/java/org/apache/carbondata/core/datastore/filesystem/S3CarbonFile.java
 ---
    @@ -0,0 +1,136 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.carbondata.core.datastore.filesystem;
    +
    +import java.io.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +import org.apache.carbondata.core.constants.CarbonCommonConstants;
    +import org.apache.carbondata.core.datastore.impl.FileFactory;
    +import org.apache.carbondata.core.util.CarbonUtil;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataOutputStream;
    +import org.apache.hadoop.fs.FileStatus;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +public class S3CarbonFile extends HDFSCarbonFile {
    +
    +  private static final LogService LOGGER =
    +      LogServiceFactory.getLogService(HDFSCarbonFile.class.getName());
    +
    +  public S3CarbonFile(String filePath) {
    +    super(filePath);
    +  }
    +
    +  public S3CarbonFile(String filePath, Configuration hadoopConf) {
    +    super(filePath, hadoopConf);
    +  }
    +
    +  public S3CarbonFile(Path path) {
    +    super(path);
    +  }
    +
    +  public S3CarbonFile(Path path, Configuration hadoopConf) {
    +    super(path, hadoopConf);
    +  }
    +
    +  public S3CarbonFile(FileStatus fileStatus) {
    +    super(fileStatus);
    +  }
    +
    +  @Override
    +  public boolean renameForce(String changetoName) {
    +    FileSystem fs;
    +    try {
    +      fs = fileStatus.getPath().getFileSystem(hadoopConf);
    +      return fs.rename(fileStatus.getPath(), new Path(changetoName));
    +    } catch (IOException e) {
    +      LOGGER.error("Exception occured: " + e.getMessage());
    +      return false;
    +    }
    +  }
    +
    +  @Override
    +  public DataOutputStream getDataOutputStreamUsingAppend(String path, 
FileFactory.FileType fileType)
    +      throws IOException {
    +    return getDataOutputStream(path, fileType, 
CarbonCommonConstants.BYTEBUFFER_SIZE, true);
    +  }
    +
    +  @Override public DataOutputStream getDataOutputStream(String path, 
FileFactory.FileType fileType,
    +      int bufferSize, boolean append) throws IOException {
    +    Path pt = new Path(path);
    +    FileSystem fileSystem = 
pt.getFileSystem(FileFactory.getConfiguration());
    +    FSDataOutputStream stream;
    +    if (append) {
    --- End diff --
    
    HDFS client only takes care of append mode in case of DistributedFileSystem 
for other it will create with overwrite. Therefore we need to read the file and 
keep the contents in memory for overwriting the file.


---

Reply via email to