Hadoop Plugin: Use HDFS as gateway storage, closes #189.
This commit is contained in:
parent
116cfce6f2
commit
28fa384b32
|
@ -38,6 +38,7 @@ import org.elasticsearch.util.xcontent.XContentType;
|
||||||
import org.elasticsearch.util.xcontent.builder.BinaryXContentBuilder;
|
import org.elasticsearch.util.xcontent.builder.BinaryXContentBuilder;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.net.URI;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -45,8 +46,12 @@ import java.util.Map;
|
||||||
*/
|
*/
|
||||||
public class HdfsGateway extends AbstractLifecycleComponent<Gateway> implements Gateway {
|
public class HdfsGateway extends AbstractLifecycleComponent<Gateway> implements Gateway {
|
||||||
|
|
||||||
|
private final boolean closeFileSystem;
|
||||||
|
|
||||||
private final FileSystem fileSystem;
|
private final FileSystem fileSystem;
|
||||||
|
|
||||||
|
private final String uri;
|
||||||
|
|
||||||
private final Path path;
|
private final Path path;
|
||||||
|
|
||||||
private final Path metaDataPath;
|
private final Path metaDataPath;
|
||||||
|
@ -56,12 +61,19 @@ public class HdfsGateway extends AbstractLifecycleComponent<Gateway> implements
|
||||||
@Inject public HdfsGateway(Settings settings, ClusterName clusterName) throws IOException {
|
@Inject public HdfsGateway(Settings settings, ClusterName clusterName) throws IOException {
|
||||||
super(settings);
|
super(settings);
|
||||||
|
|
||||||
|
this.closeFileSystem = componentSettings.getAsBoolean("close_fs", true);
|
||||||
|
this.uri = componentSettings.get("uri");
|
||||||
|
if (uri == null) {
|
||||||
|
throw new ElasticSearchIllegalArgumentException("hdfs gateway requires the 'uri' setting to be set");
|
||||||
|
}
|
||||||
String path = componentSettings.get("path");
|
String path = componentSettings.get("path");
|
||||||
if (path == null) {
|
if (path == null) {
|
||||||
throw new ElasticSearchIllegalArgumentException("hdfs gateway requires the 'path' path setting to be set");
|
throw new ElasticSearchIllegalArgumentException("hdfs gateway requires the 'path' path setting to be set");
|
||||||
}
|
}
|
||||||
this.path = new Path(new Path(path), clusterName.value());
|
this.path = new Path(new Path(path), clusterName.value());
|
||||||
|
|
||||||
|
logger.debug("Using uri [{}], path [{}]", this.uri, this.path);
|
||||||
|
|
||||||
this.metaDataPath = new Path(this.path, "metadata");
|
this.metaDataPath = new Path(this.path, "metadata");
|
||||||
|
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
|
@ -70,7 +82,7 @@ public class HdfsGateway extends AbstractLifecycleComponent<Gateway> implements
|
||||||
conf.set(entry.getKey(), entry.getValue());
|
conf.set(entry.getKey(), entry.getValue());
|
||||||
}
|
}
|
||||||
|
|
||||||
fileSystem = FileSystem.get(conf);
|
fileSystem = FileSystem.get(URI.create(uri), conf);
|
||||||
|
|
||||||
fileSystem.mkdirs(metaDataPath);
|
fileSystem.mkdirs(metaDataPath);
|
||||||
|
|
||||||
|
@ -93,10 +105,12 @@ public class HdfsGateway extends AbstractLifecycleComponent<Gateway> implements
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override protected void doClose() throws ElasticSearchException {
|
@Override protected void doClose() throws ElasticSearchException {
|
||||||
try {
|
if (closeFileSystem) {
|
||||||
fileSystem.close();
|
try {
|
||||||
} catch (IOException e) {
|
fileSystem.close();
|
||||||
logger.warn("Failed to close file system {}", fileSystem);
|
} catch (IOException e) {
|
||||||
|
logger.warn("Failed to close file system {}", fileSystem);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -124,8 +138,10 @@ public class HdfsGateway extends AbstractLifecycleComponent<Gateway> implements
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
for (FileStatus oldFile : oldFiles) {
|
if (oldFiles != null) {
|
||||||
fileSystem.delete(oldFile.getPath(), false);
|
for (FileStatus oldFile : oldFiles) {
|
||||||
|
fileSystem.delete(oldFile.getPath(), false);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
@ -161,6 +177,9 @@ public class HdfsGateway extends AbstractLifecycleComponent<Gateway> implements
|
||||||
return path.getName().startsWith("metadata-");
|
return path.getName().startsWith("metadata-");
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
if (files == null || files.length == 0) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
int index = -1;
|
int index = -1;
|
||||||
for (FileStatus file : files) {
|
for (FileStatus file : files) {
|
||||||
|
@ -174,7 +193,7 @@ public class HdfsGateway extends AbstractLifecycleComponent<Gateway> implements
|
||||||
try {
|
try {
|
||||||
readMetaData(file.getPath());
|
readMetaData(file.getPath());
|
||||||
index = fileIndex;
|
index = fileIndex;
|
||||||
} catch (IOException e) {
|
} catch (Exception e) {
|
||||||
logger.warn("[findLatestMetadata]: Failed to read metadata from [" + file + "], ignoring...", e);
|
logger.warn("[findLatestMetadata]: Failed to read metadata from [" + file + "], ignoring...", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -290,16 +290,18 @@ public class HdfsIndexShardGateway extends AbstractIndexShardComponent implement
|
||||||
if (indexDirty) {
|
if (indexDirty) {
|
||||||
try {
|
try {
|
||||||
FileStatus[] existingFiles = fileSystem.listStatus(indexPath);
|
FileStatus[] existingFiles = fileSystem.listStatus(indexPath);
|
||||||
for (FileStatus existingFile : existingFiles) {
|
if (existingFiles != null) {
|
||||||
boolean found = false;
|
for (FileStatus existingFile : existingFiles) {
|
||||||
for (final String fileName : snapshotIndexCommit.getFiles()) {
|
boolean found = false;
|
||||||
if (existingFile.getPath().getName().equals(fileName)) {
|
for (final String fileName : snapshotIndexCommit.getFiles()) {
|
||||||
found = true;
|
if (existingFile.getPath().getName().equals(fileName)) {
|
||||||
break;
|
found = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (!found) {
|
||||||
|
fileSystem.delete(existingFile.getPath(), false);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
if (!found) {
|
|
||||||
fileSystem.delete(existingFile.getPath(), false);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
|
@ -320,6 +322,10 @@ public class HdfsIndexShardGateway extends AbstractIndexShardComponent implement
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new IndexShardGatewayRecoveryException(shardId(), "Failed to list files", e);
|
throw new IndexShardGatewayRecoveryException(shardId(), "Failed to list files", e);
|
||||||
}
|
}
|
||||||
|
if (files == null || files.length == 0) {
|
||||||
|
return new RecoveryStatus.Index(-1, 0, new SizeValue(0, SizeUnit.BYTES), TimeValue.timeValueMillis(0));
|
||||||
|
}
|
||||||
|
|
||||||
final CountDownLatch latch = new CountDownLatch(files.length);
|
final CountDownLatch latch = new CountDownLatch(files.length);
|
||||||
final AtomicReference<Exception> lastException = new AtomicReference<Exception>();
|
final AtomicReference<Exception> lastException = new AtomicReference<Exception>();
|
||||||
final AtomicLong throttlingWaitTime = new AtomicLong();
|
final AtomicLong throttlingWaitTime = new AtomicLong();
|
||||||
|
@ -334,7 +340,6 @@ public class HdfsIndexShardGateway extends AbstractIndexShardComponent implement
|
||||||
throttlingWaitTime.addAndGet(System.currentTimeMillis() - throttlingStartTime);
|
throttlingWaitTime.addAndGet(System.currentTimeMillis() - throttlingStartTime);
|
||||||
FSDataInputStream fileStream = fileSystem.open(file.getPath());
|
FSDataInputStream fileStream = fileSystem.open(file.getPath());
|
||||||
Directories.copyToDirectory(fileStream, store.directory(), file.getPath().getName());
|
Directories.copyToDirectory(fileStream, store.directory(), file.getPath().getName());
|
||||||
fileSystem.close();
|
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.debug("Failed to read [" + file + "] into [" + store + "]", e);
|
logger.debug("Failed to read [" + file + "] into [" + store + "]", e);
|
||||||
lastException.set(e);
|
lastException.set(e);
|
||||||
|
@ -397,7 +402,7 @@ public class HdfsIndexShardGateway extends AbstractIndexShardComponent implement
|
||||||
} finally {
|
} finally {
|
||||||
if (fileStream != null) {
|
if (fileStream != null) {
|
||||||
try {
|
try {
|
||||||
fileSystem.close();
|
fileStream.close();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
// ignore
|
// ignore
|
||||||
}
|
}
|
||||||
|
|
|
@ -52,15 +52,19 @@ public class HdfsGatewayTests {
|
||||||
private Node node;
|
private Node node;
|
||||||
|
|
||||||
@BeforeMethod void setUpNodes() throws Exception {
|
@BeforeMethod void setUpNodes() throws Exception {
|
||||||
|
// start the node and reset the gateway
|
||||||
node = buildNode();
|
node = buildNode();
|
||||||
((InternalNode) node).injector().getInstance(Gateway.class).reset();
|
((InternalNode) node).injector().getInstance(Gateway.class).reset();
|
||||||
node.start();
|
node.close();
|
||||||
|
// now start the node clean
|
||||||
|
node = buildNode().start();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Node buildNode() {
|
private Node buildNode() {
|
||||||
Settings settings = settingsBuilder()
|
Settings settings = settingsBuilder()
|
||||||
// .put("hdfs.conf.fs.default.name", "file://work")
|
|
||||||
.put("gateway.type", "hdfs")
|
.put("gateway.type", "hdfs")
|
||||||
|
// .put("gateway.hdfs.uri", "hdfs://training-vm.local:8022")
|
||||||
|
.put("gateway.hdfs.uri", "file:///")
|
||||||
.put("gateway.hdfs.path", "work/hdfs/gateway")
|
.put("gateway.hdfs.path", "work/hdfs/gateway")
|
||||||
.build();
|
.build();
|
||||||
return nodeBuilder().settings(settingsBuilder().put(settings).put("node.name", "node1")).build();
|
return nodeBuilder().settings(settingsBuilder().put(settings).put("node.name", "node1")).build();
|
||||||
|
|
Loading…
Reference in New Issue