Merge pull request #594 from metamx/hdfs-data-segment-killer

hdfs data segment killer
This commit is contained in:
fjy 2014-06-10 12:24:34 -06:00
commit 12d1d32acf
2 changed files with 82 additions and 0 deletions

View File

@ -0,0 +1,81 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.storage.hdfs;
import com.google.inject.Inject;
import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
public class HdfsDataSegmentKiller implements DataSegmentKiller
{
private final Configuration config;
@Inject
public HdfsDataSegmentKiller(final Configuration config)
{
this.config = config;
}
@Override
public void kill(DataSegment segment) throws SegmentLoadingException
{
final Path path = getPath(segment);
final FileSystem fs = checkPathAndGetFilesystem(path);
try {
if (path.getName().endsWith(".zip")) {
// delete the parent directory containing the zip file and the descriptor
fs.delete(path.getParent(), true);
} else {
throw new SegmentLoadingException("Unknown file type[%s]", path);
}
}
catch (IOException e) {
throw new SegmentLoadingException(e, "Unable to kill segment");
}
}
private Path getPath(DataSegment segment)
{
return new Path(String.valueOf(segment.getLoadSpec().get("path")));
}
private FileSystem checkPathAndGetFilesystem(Path path) throws SegmentLoadingException
{
FileSystem fs;
try {
fs = path.getFileSystem(config);
if (!fs.exists(path)) {
throw new SegmentLoadingException("Path[%s] doesn't exist.", path);
}
return fs;
}
catch (IOException e) {
throw new SegmentLoadingException(e, "Problems interacting with filesystem[%s].", path);
}
}
}

View File

@ -55,6 +55,7 @@ public class HdfsStorageDruidModule implements DruidModule
{ {
Binders.dataSegmentPullerBinder(binder).addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); Binders.dataSegmentPullerBinder(binder).addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class);
Binders.dataSegmentPusherBinder(binder).addBinding("hdfs").to(HdfsDataSegmentPusher.class).in(LazySingleton.class); Binders.dataSegmentPusherBinder(binder).addBinding("hdfs").to(HdfsDataSegmentPusher.class).in(LazySingleton.class);
Binders.dataSegmentKillerBinder(binder).addBinding("hdfs").to(HdfsDataSegmentKiller.class).in(LazySingleton.class);
final Configuration conf = new Configuration(); final Configuration conf = new Configuration();
if (props != null) { if (props != null) {