mirror of https://github.com/apache/druid.git
Merge pull request #1395 from druid-io/new-logs
Additional schema validations and better logs for common extensions
This commit is contained in:
commit
fe286d28d1
|
@ -57,6 +57,8 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
|
|||
this.config = config;
|
||||
this.hadoopConfig = hadoopConfig;
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
||||
log.info("Configured HDFS as deep storage");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -69,6 +71,14 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher
|
|||
public DataSegment push(File inDir, DataSegment segment) throws IOException
|
||||
{
|
||||
final String storageDir = DataSegmentPusherUtil.getHdfsStorageDir(segment);
|
||||
|
||||
log.info(
|
||||
"Copying segment[%s] to HDFS at location[%s/%s]",
|
||||
segment.getIdentifier(),
|
||||
config.getStorageDirectory(),
|
||||
storageDir
|
||||
);
|
||||
|
||||
Path outFile = new Path(String.format("%s/%s/index.zip", config.getStorageDirectory(), storageDir));
|
||||
FileSystem fs = outFile.getFileSystem(hadoopConfig);
|
||||
|
||||
|
|
|
@ -56,6 +56,8 @@ public class MySQLConnector extends SQLMetadataConnector
|
|||
datasource.setConnectionInitSqls(ImmutableList.of("SET sql_mode='ANSI_QUOTES'"));
|
||||
|
||||
this.dbi = new DBI(datasource);
|
||||
|
||||
log.info("Configured MySQL as metadata storage");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.util.List;
|
|||
|
||||
public class MySQLMetadataStorageModule extends SQLMetadataStorageDruidModule implements DruidModule
|
||||
{
|
||||
|
||||
public static final String TYPE = "mysql";
|
||||
|
||||
public MySQLMetadataStorageModule()
|
||||
|
|
|
@ -51,6 +51,8 @@ public class PostgreSQLConnector extends SQLMetadataConnector
|
|||
datasource.setDriverClassName("org.postgresql.Driver");
|
||||
|
||||
this.dbi = new DBI(datasource);
|
||||
|
||||
log.info("Configured PostgreSQL as metadata storage");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -55,6 +55,8 @@ public class S3DataSegmentPusher implements DataSegmentPusher
|
|||
this.s3Client = s3Client;
|
||||
this.config = config;
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
||||
log.info("Configured S3 as deep storage");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -66,8 +68,10 @@ public class S3DataSegmentPusher implements DataSegmentPusher
|
|||
@Override
|
||||
public DataSegment push(final File indexFilesDir, final DataSegment inSegment) throws IOException
|
||||
{
|
||||
log.info("Uploading [%s] to S3", indexFilesDir);
|
||||
final String s3Path = S3Utils.constructSegmentPath(config.getBaseKey(), inSegment);
|
||||
|
||||
log.info("Copying segment[%s] to S3 at location[%s]", inSegment.getIdentifier(), s3Path);
|
||||
|
||||
final File zipOutFile = File.createTempFile("druid", "index.zip");
|
||||
final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile);
|
||||
|
||||
|
|
|
@ -44,6 +44,7 @@ public class HadoopIngestionSpecTest
|
|||
schema = jsonReadWriteRead(
|
||||
"{\n"
|
||||
+ " \"dataSchema\": {\n"
|
||||
+ " \"dataSource\": \"foo\",\n"
|
||||
+ " \"metricsSpec\": [],\n"
|
||||
+ " \"granularitySpec\": {\n"
|
||||
+ " \"type\": \"uniform\",\n"
|
||||
|
|
|
@ -19,6 +19,7 @@ package io.druid.metadata.storage.derby;
|
|||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
import io.druid.metadata.SQLMetadataConnector;
|
||||
|
@ -28,6 +29,7 @@ import org.skife.jdbi.v2.Handle;
|
|||
|
||||
public class DerbyConnector extends SQLMetadataConnector
|
||||
{
|
||||
private static final Logger log = new Logger(DerbyConnector.class);
|
||||
private static final String SERIAL_TYPE = "BIGINT GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1)";
|
||||
private final DBI dbi;
|
||||
|
||||
|
@ -41,6 +43,8 @@ public class DerbyConnector extends SQLMetadataConnector
|
|||
datasource.setDriverClassName("org.apache.derby.jdbc.ClientDriver");
|
||||
|
||||
this.dbi = new DBI(datasource);
|
||||
|
||||
log.info("Configured Derby as metadata storage");
|
||||
}
|
||||
|
||||
public DerbyConnector(
|
||||
|
|
|
@ -19,7 +19,9 @@ package io.druid.segment.indexing;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.api.client.repackaged.com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
|
@ -33,6 +35,8 @@ import java.util.Set;
|
|||
*/
|
||||
public class DataSchema
|
||||
{
|
||||
private static final Logger log = new Logger(DataSchema.class);
|
||||
|
||||
private final String dataSource;
|
||||
private final InputRowParser parser;
|
||||
private final AggregatorFactory[] aggregators;
|
||||
|
@ -46,6 +50,8 @@ public class DataSchema
|
|||
@JsonProperty("granularitySpec") GranularitySpec granularitySpec
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(dataSource, "dataSource cannot be null. Please provide a dataSource.");
|
||||
|
||||
this.dataSource = dataSource;
|
||||
|
||||
final Set<String> dimensionExclusions = Sets.newHashSet();
|
||||
|
@ -68,23 +74,34 @@ public class DataSchema
|
|||
parser.getParseSpec()
|
||||
.withDimensionsSpec(
|
||||
dimensionsSpec
|
||||
.withDimensionExclusions(
|
||||
Sets.difference(dimensionExclusions,
|
||||
Sets.newHashSet(dimensionsSpec.getDimensions()))
|
||||
)
|
||||
.withDimensionExclusions(
|
||||
Sets.difference(
|
||||
dimensionExclusions,
|
||||
Sets.newHashSet(dimensionsSpec.getDimensions())
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
} else {
|
||||
this.parser = parser;
|
||||
}
|
||||
} else {
|
||||
log.warn("No parser or parseSpec has been specified");
|
||||
|
||||
this.parser = parser;
|
||||
}
|
||||
|
||||
if (aggregators.length == 0) {
|
||||
log.warn("No metricsSpec has been specified. Are you sure this is what you want?");
|
||||
}
|
||||
this.aggregators = aggregators;
|
||||
this.granularitySpec = granularitySpec == null
|
||||
? new UniformGranularitySpec(null, null, null)
|
||||
: granularitySpec;
|
||||
|
||||
if (granularitySpec == null) {
|
||||
log.warn("No granularitySpec has been specified. Using UniformGranularitySpec as default.");
|
||||
this.granularitySpec = new UniformGranularitySpec(null, null, null);
|
||||
} else {
|
||||
this.granularitySpec = granularitySpec;
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
|
|
@ -47,6 +47,8 @@ public class LocalDataSegmentPusher implements DataSegmentPusher
|
|||
{
|
||||
this.config = config;
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
||||
log.info("Configured local filesystem as deep storage");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -60,6 +62,8 @@ public class LocalDataSegmentPusher implements DataSegmentPusher
|
|||
{
|
||||
File outDir = new File(config.getStorageDirectory(), DataSegmentPusherUtil.getStorageDir(segment));
|
||||
|
||||
log.info("Copying segment[%s] to local filesystem at location[%s]", segment.getIdentifier(), outDir.toString());
|
||||
|
||||
if (dataSegmentFile.equals(outDir)) {
|
||||
long size = 0;
|
||||
for (File file : dataSegmentFile.listFiles()) {
|
||||
|
|
Loading…
Reference in New Issue