Skip to content

Commit

Permalink
blob implementation for HDFS
Browse files Browse the repository at this point in the history
relates to #72
  • Loading branch information
costin committed Jan 6, 2014
1 parent c3cf3cb commit 165a251
Show file tree
Hide file tree
Showing 4 changed files with 323 additions and 0 deletions.
@@ -0,0 +1,121 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch 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.elasticsearch.hadoop.hdfs.blobstore;

import java.io.IOException;

import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.blobstore.BlobMetaData;
import org.elasticsearch.common.blobstore.BlobPath;
import org.elasticsearch.common.blobstore.support.AbstractBlobContainer;
import org.elasticsearch.common.blobstore.support.PlainBlobMetaData;
import org.elasticsearch.common.collect.ImmutableMap;

public class AbstractHdfsBlobContainer extends AbstractBlobContainer {

protected final HdfsBlobStore blobStore;
protected final Path path;

public AbstractHdfsBlobContainer(BlobPath blobPath, HdfsBlobStore blobStore, Path path) {
super(blobPath);
this.blobStore = blobStore;
this.path = path;
}

@Override
public boolean blobExists(String blobName) {
try {
return blobStore.fileSystem().exists(new Path(path, blobName));
} catch (Exception e) {
return false;
}
}

@Override
public boolean deleteBlob(String blobName) throws IOException {
return blobStore.fileSystem().delete(new Path(path, blobName), true);
}

@Override
public void readBlob(final String blobName, final ReadBlobListener listener) {
blobStore.executor().execute(new Runnable() {
@Override
public void run() {
byte[] buffer = new byte[blobStore.bufferSizeInBytes()];

FSDataInputStream fileStream;
try {
fileStream = blobStore.fileSystem().open(new Path(path, blobName));
} catch (IOException e) {
listener.onFailure(e);
return;
}
try {
int bytesRead;
while ((bytesRead = fileStream.read(buffer)) != -1) {
listener.onPartial(buffer, 0, bytesRead);
}
listener.onCompleted();
} catch (Exception e) {
try {
fileStream.close();
} catch (IOException e1) {
// ignore
}
listener.onFailure(e);
}
}
});
}

@Override
public ImmutableMap<String, BlobMetaData> listBlobsByPrefix(final @Nullable String blobNamePrefix) throws IOException {
FileStatus[] files = blobStore.fileSystem().listStatus(path, new PathFilter() {
@Override
public boolean accept(Path path) {
return path.getName().startsWith(blobNamePrefix);
}
});
if (files == null || files.length == 0) {
return ImmutableMap.of();
}
ImmutableMap.Builder<String, BlobMetaData> builder = ImmutableMap.builder();
for (FileStatus file : files) {
builder.put(file.getPath().getName(), new PlainBlobMetaData(file.getPath().getName(), file.getLen()));
}
return builder.build();
}

public ImmutableMap<String, BlobMetaData> listBlobs() throws IOException {
FileStatus[] files = blobStore.fileSystem().listStatus(path);
if (files == null || files.length == 0) {
return ImmutableMap.of();
}
ImmutableMap.Builder<String, BlobMetaData> builder = ImmutableMap.builder();
for (FileStatus file : files) {
builder.put(file.getPath().getName(), new PlainBlobMetaData(file.getPath().getName(), file.getLen()));
}
return builder.build();
}
}
@@ -0,0 +1,111 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch 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.elasticsearch.hadoop.hdfs.blobstore;

import java.io.IOException;
import java.util.concurrent.Executor;

import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.elasticsearch.common.blobstore.BlobPath;
import org.elasticsearch.common.blobstore.BlobStore;
import org.elasticsearch.common.blobstore.ImmutableBlobContainer;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;

public class HdfsBlobStore extends AbstractComponent implements BlobStore {

private final FileSystem fs;
private final Path rootHdfsPath;
private final Executor executor;
private final int bufferSizeInBytes;

public HdfsBlobStore(Settings settings, FileSystem fs, Path path, Executor executor) throws IOException {
super(settings);
this.fs = fs;
this.rootHdfsPath = path;
this.executor = executor;

this.bufferSizeInBytes = (int) settings.getAsBytesSize("buffer_size", new ByteSizeValue(100, ByteSizeUnit.KB)).bytes();

if (!fs.exists(path)) {
fs.mkdirs(path);
}
}

@Override
public String toString() {
return rootHdfsPath.toUri().toString();
}

public FileSystem fileSystem() {
return fs;
}

public Path path() {
return rootHdfsPath;
}

public Executor executor() {
return executor;
}

public int bufferSizeInBytes() {
return bufferSizeInBytes;
}

@Override
public ImmutableBlobContainer immutableBlobContainer(BlobPath path) {
return new HdfsImmutableBlobContainer(path, this, buildHdfsPath(path));
}

@Override
public void delete(BlobPath path) {
try {
fs.delete(translateToHdfsPath(path), true);
} catch (IOException ex) {
}
}

private Path buildHdfsPath(BlobPath blobPath) {
Path path = translateToHdfsPath(blobPath);
try {
fs.mkdirs(path);
} catch (IOException e) {
// ignore
}
return path;
}

private Path translateToHdfsPath(BlobPath blobPath) {
Path path = path();
for (String p : blobPath) {
path = new Path(path, p);
}
return path;
}

@Override
public void close() {
//
}
}
@@ -0,0 +1,83 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch 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.elasticsearch.hadoop.hdfs.blobstore;

import java.io.IOException;
import java.io.InputStream;

import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
import org.elasticsearch.common.blobstore.BlobPath;
import org.elasticsearch.common.blobstore.ImmutableBlobContainer;
import org.elasticsearch.common.blobstore.support.BlobStores;

public class HdfsImmutableBlobContainer extends AbstractHdfsBlobContainer implements ImmutableBlobContainer {

public HdfsImmutableBlobContainer(BlobPath blobPath, HdfsBlobStore blobStore, Path path) {
super(blobPath, blobStore, path);
}

@Override
public void writeBlob(final String blobName, final InputStream is, final long sizeInBytes, final WriterListener listener) {
blobStore.executor().execute(new Runnable() {
@Override
public void run() {
Path file = new Path(path, blobName);

FSDataOutputStream fileStream;
try {
fileStream = blobStore.fileSystem().create(file, true);
} catch (IOException e) {
listener.onFailure(e);
return;
}
try {
try {
byte[] buffer = new byte[blobStore.bufferSizeInBytes()];
int bytesRead;
while ((bytesRead = is.read(buffer)) != -1) {
fileStream.write(buffer, 0, bytesRead);
}
} finally {
IOUtils.closeStream(is);
IOUtils.closeStream(fileStream);
}
listener.onCompleted();
} catch (Exception e) {
// just on the safe size, try and delete it on failure
try {
if (blobStore.fileSystem().exists(file)) {
blobStore.fileSystem().delete(file, true);
}
} catch (Exception e1) {
// ignore
}
listener.onFailure(e);
}
}
});
}

@Override
public void writeBlob(String blobName, InputStream is, long sizeInBytes) throws IOException {
BlobStores.syncWriteBlob(this, blobName, is, sizeInBytes);
}
}
8 changes: 8 additions & 0 deletions src/main/java/org/elasticsearch/hadoop/mr/HadoopCfgUtils.java
Expand Up @@ -27,6 +27,14 @@ public static boolean isLocal(Configuration cfg) {
return "local".equals(getJobTracker(cfg));
}

public static String getFileSystem(Configuration cfg) {
return get(cfg, "fs.defaultFS", "fs.default.name");
}

public static void setFileSystem(Configuration cfg, String value) {
set(cfg, value, "fs.defaultFS", "fs.default.name");
}

public static String getJobTracker(Configuration cfg) {
return get(cfg, "mapreduce.framework.name", "mapred.job.tracker");
}
Expand Down

0 comments on commit 165a251

Please sign in to comment.