Addresed Gian's code review

This commit is contained in:
Deep Ganguli 2013-01-24 17:54:06 -08:00
parent 017d4779d6
commit cb845e6f09
3 changed files with 12 additions and 11 deletions

View File

@ -45,7 +45,6 @@ import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.index.v1.IndexMerger;
import com.metamx.druid.indexer.rollup.DataRollupSpec;
import com.metamx.druid.input.MapBasedInputRow;
import com.metamx.druid.jackson.DefaultObjectMapper;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -70,8 +69,6 @@ import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
import org.codehaus.jackson.map.ObjectMapper;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.tweak.HandleCallback;
import javax.annotation.Nullable;
import java.io.BufferedOutputStream;
@ -185,7 +182,7 @@ public class IndexGeneratorJob implements Jobby
for (FileStatus status : fs.listStatus(descriptorInfoDir)) {
final DataSegment segment = jsonMapper.readValue(fs.open(status.getPath()), DataSegment.class);
publishedSegmentsBuilder.add(segment);
log.info("Published %s", segment.getIdentifier());
log.info("Adding segment %s to the list of published segments", segment.getIdentifier());
}
}
catch (IOException e) {

View File

@ -26,9 +26,10 @@ public class HadoopIndexTask extends AbstractTask
/**
* @param config is used by the HadoopDruidIndexerJob to set up the appropriate parameters
* for creating Druid index segments. It may be modified. Here, we will ensure that the
* UpDaterJobSpec field of the config is set to null, such that the job does not push a
* list of published segments the database. Instead, we will use the method
* for creating Druid index segments. It may be modified.
* <p/>
* Here, we will ensure that the UpDaterJobSpec field of the config is set to null, such that the
* job does not push a list of published segments the database. Instead, we will use the method
* IndexGeneratorJob.getPublishedSegments() to simply return a list of the published
* segments, and let the indexing service report these segments to the database.
*/
@ -37,13 +38,15 @@ public class HadoopIndexTask extends AbstractTask
public HadoopIndexTask(@JsonProperty("config") HadoopDruidIndexerConfig config)
{
super(
String.format("index_hadoop_%s_interval_%s", config.getDataSource(), new DateTime().now()),
String.format("index_hadoop_%s_interval_%s", config.getDataSource(), new DateTime()),
config.getDataSource(),
JodaUtils.umbrellaInterval(config.getIntervals())
);
if (config.isUpdaterJobSpecSet()) {
throw new IllegalArgumentException("UpDaterJobSpec is defined");
throw new IllegalArgumentException(
"The UpDaterJobSpec field of the Hadoop Druid indexer config must be set to null "
);
}
this.config = config;
}
@ -51,7 +54,7 @@ public class HadoopIndexTask extends AbstractTask
@Override
public Type getType()
{
return Type.INDEX;
return Type.HADOOPINDEX;
}
@Override

View File

@ -47,6 +47,7 @@ public interface Task
enum Type
{
INDEX,
HADOOPINDEX,
MERGE,
APPEND,
DELETE,