Move druid-api into the druid repo.

This is from druid-api-0.3.17, as of commit 51884f1d05d5512cacaf62cedfbb28c6ab2535cf
in the druid-api repo.
This commit is contained in:
Gian Merlino 2016-03-24 10:53:31 -07:00
parent bfc0ae7d2a
commit 7e7a886f65
124 changed files with 8254 additions and 9 deletions

103
api/pom.xml Normal file
View File

@ -0,0 +1,103 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ Licensed to Metamarkets Group Inc. (Metamarkets) under one
~ or more contributor license agreements. See the NOTICE file
~ distributed with this work for additional information
~ regarding copyright ownership. Metamarkets 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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>druid-api</artifactId>
<name>druid-api</name>
<description>Druid Extensions API</description>
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.9.1-SNAPSHOT</version>
</parent>
<dependencies>
<dependency>
<groupId>com.metamx</groupId>
<artifactId>java-util</artifactId>
</dependency>
<dependency>
<groupId>com.google.inject</groupId>
<artifactId>guice</artifactId>
</dependency>
<dependency>
<groupId>com.google.inject.extensions</groupId>
<artifactId>guice-multibindings</artifactId>
</dependency>
<dependency>
<groupId>io.airlift</groupId>
<artifactId>airline</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-smile</artifactId>
</dependency>
<dependency>
<groupId>org.hibernate</groupId>
<artifactId>hibernate-validator</artifactId>
</dependency>
<dependency>
<groupId>javax.validation</groupId>
<artifactId>validation-api</artifactId>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
<!-- Tests -->
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-simple</artifactId>
<scope>test</scope>
<optional>true</optional>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-release-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,29 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.cli;
import io.airlift.airline.Cli;
/**
*/
public interface CliCommandCreator
{
public void addCommands(Cli.CliBuilder builder);
}

View File

@ -0,0 +1,27 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.cli;
/**
*/
public interface CliRunnable extends Runnable
{
}

View File

@ -0,0 +1,32 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.ParseSpec;
import java.nio.ByteBuffer;
public interface ByteBufferInputRowParser extends InputRowParser<ByteBuffer>
{
@Override
public ByteBufferInputRowParser withParseSpec(ParseSpec parseSpec);
}

View File

@ -0,0 +1,32 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
/**
* Committer includes a Runnable and a Jackson-serialized metadata object containing the offset
*/
public interface Committer extends Runnable
{
/**
* @return A json serialized represenation of commit metadata,
* which needs to be serialized and deserialized by Jackson.
* Commit metadata can be a complex type, but we recommend keeping it to List/Map/"Primitive JSON" types
* */
public Object getMetadata();
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import java.io.Closeable;
/**
* This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this
* abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement
* one of these and register it with the Main.
*
* This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends
* Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this
* gets passed around as an Iterator.
* <p>
* The implementation of this interface only needs to be minimally thread-safe. The three methods ##hasMore(),
* ##nextRow() and ##commit() are all called from the same thread. ##commit(), however, returns a callback
* which will be called on another thread, so the operations inside of that callback must be thread-safe.
* </p>
*/
public interface Firehose extends Closeable
{
/**
* Returns whether there are more rows to process. This is used to indicate that another item is immediately
* available via ##nextRow(). Thus, if the stream is still available but there are no new messages on it, this call
* should block until a new message is available.
*
* If something happens such that the stream is no longer available, this should return false.
*
* @return true if and when there is another row available, false if the stream has dried up
*/
public boolean hasMore();
/**
* The next row available. Should only be called if hasMore returns true.
*
* @return The next row
*/
public InputRow nextRow() ;
/**
* Returns a runnable that will "commit" everything read up to the point at which commit() is called. This is
* often equivalent to everything that has been read since the last commit() call (or instantiation of the object),
* but doesn't necessarily have to be.
*
* This method is called when the main processing loop starts to persist its current batch of things to process.
* The returned runnable will be run when the current batch has been successfully persisted, there is usually
* some time lag between when this method is called and when the runnable is run. The Runnable is also run on
* a separate thread so its operation should be thread-safe.
*
* The Runnable is essentially just a lambda/closure that is run() after data supplied by this instance has
* been committed on the writer side of this interface protocol.
* <p>
* A simple implementation of this interface might do nothing when run() is called
* (in which case the same do-nothing instance can be returned every time), or
* a more complex implementation might clean up temporary resources that are no longer needed
* because of InputRows delivered by prior calls to ##nextRow().
* </p>
*/
public Runnable commit();
}

View File

@ -0,0 +1,41 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.common.parsers.ParseException;
import io.druid.data.input.impl.InputRowParser;
import java.io.IOException;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
public interface FirehoseFactory<T extends InputRowParser>
{
/**
* Initialization method that connects up the fire hose. If this method returns successfully it should be safe to
* call hasMore() on the returned Firehose (which might subsequently block).
* <p/>
* If this method returns null, then any attempt to call hasMore(), nextRow(), commit() and close() on the return
* value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on
* invalid configuration is preferred over returning null.
*/
public Firehose connect(T parser) throws IOException, ParseException;
}

View File

@ -0,0 +1,44 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.common.parsers.ParseException;
import io.druid.data.input.impl.InputRowParser;
import java.io.IOException;
/**
* Initialization method that connects up the FirehoseV2. If this method returns successfully it should be safe to
* call start() on the returned FirehoseV2 (which might subsequently block).
*
* In contrast to V1 version, FirehoseFactoryV2 is able to pass an additional json-serialized object to FirehoseV2,
* which contains last commit metadata
*
* <p/>
* If this method returns null, then any attempt to call start(), advance(), currRow(), makeCommitter() and close() on the return
* value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on
* invalid configuration is preferred over returning null.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
public interface FirehoseFactoryV2<T extends InputRowParser>
{
public FirehoseV2 connect(T parser, Object lastCommit) throws IOException, ParseException;
}

View File

@ -0,0 +1,88 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import java.io.Closeable;
/**
* This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this
* abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement
* one of these and register it with the Main.
*
* In contrast to Firehose v1 version, FirehoseV2 will always operate in a "peek, then advance" manner.
* And the intended usage patttern is
* 1. Call start()
* 2. Read currRow()
* 3. Call advance()
* 4. If index should be committed: commit()
* 5. GOTO 2
*
* Note that commit() is being called *after* advance.
*
* This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends
* Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this
* gets passed around as an Iterator.
* <p>
* The implementation of this interface only needs to be minimally thread-safe. The methods ##start(), ##advance(),
* ##currRow() and ##makeCommitter() are all called from the same thread. ##makeCommitter(), however, returns a callback
* which will be called on another thread, so the operations inside of that callback must be thread-safe.
* </p>
*/
public interface FirehoseV2 extends Closeable
{
/**
* For initial start
* */
void start() throws Exception;
/**
* Advance the firehose to the next offset. Implementations of this interface should make sure that
* if advance() is called and throws out an exception, the next call to currRow() should return a
* null value.
*
* @return true if and when there is another row available, false if the stream has dried up
*/
public boolean advance();
/**
* @return The current row
*/
public InputRow currRow() ;
/**
* Returns a Committer that will "commit" everything read up to the point at which makeCommitter() is called.
*
* This method is called when the main processing loop starts to persist its current batch of things to process.
* The returned committer will be run when the current batch has been successfully persisted
* and the metadata the committer carries can also be persisted along with segment data. There is usually
* some time lag between when this method is called and when the runnable is run. The Runnable is also run on
* a separate thread so its operation should be thread-safe.
*
* Note that "correct" usage of this interface will always call advance() before commit() if the current row
* is considered in the commit.
*
* The Runnable is essentially just a lambda/closure that is run() after data supplied by this instance has
* been committed on the writer side of this interface protocol.
* <p>
* A simple implementation of this interface might do nothing when run() is called,
* and save proper commit information in metadata
* </p>
*/
public Committer makeCommitter();
}

View File

@ -0,0 +1,40 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import java.util.List;
/**
* An InputRow is the interface definition of an event being input into the data ingestion layer.
*
* An InputRow is a Row with a self-describing list of the dimensions available. This list is used to
* implement "schema-less" data ingestion that allows the system to add new dimensions as they appear.
*
*/
public interface
InputRow extends Row
{
/**
* Returns the dimensions that exist in this row.
*
* @return the dimensions that exist in this row.
*/
public List<String> getDimensions();
}

View File

@ -0,0 +1,68 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import org.joda.time.DateTime;
import java.util.List;
import java.util.Map;
/**
*/
public class MapBasedInputRow extends MapBasedRow implements InputRow
{
private final List<String> dimensions;
public MapBasedInputRow(
long timestamp,
List<String> dimensions,
Map<String, Object> event
)
{
super(timestamp, event);
this.dimensions = dimensions;
}
public MapBasedInputRow(
DateTime timestamp,
List<String> dimensions,
Map<String, Object> event
)
{
super(timestamp, event);
this.dimensions = dimensions;
}
@Override
public List<String> getDimensions()
{
return dimensions;
}
@Override
public String toString()
{
return "MapBasedInputRow{" +
"timestamp=" + new DateTime(getTimestampFromEpoch()) +
", event=" + getEvent() +
", dimensions=" + dimensions +
'}';
}
}

View File

@ -0,0 +1,205 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
import com.metamx.common.IAE;
import com.metamx.common.logger.Logger;
import com.metamx.common.parsers.ParseException;
import org.joda.time.DateTime;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
/**
*/
public class MapBasedRow implements Row
{
private static final Logger log = new Logger(MapBasedRow.class);
private static final Function<Object, String> TO_STRING_INCLUDING_NULL = new Function<Object, String>() {
@Override
public String apply(final Object o)
{
return String.valueOf(o);
}
};
private final DateTime timestamp;
private final Map<String, Object> event;
private static final Pattern LONG_PAT = Pattern.compile("[-|+]?\\d+");
@JsonCreator
public MapBasedRow(
@JsonProperty("timestamp") DateTime timestamp,
@JsonProperty("event") Map<String, Object> event
)
{
this.timestamp = timestamp;
this.event = event;
}
public MapBasedRow(
long timestamp,
Map<String, Object> event
)
{
this(new DateTime(timestamp), event);
}
@Override
public long getTimestampFromEpoch()
{
return timestamp.getMillis();
}
@JsonProperty
public DateTime getTimestamp()
{
return timestamp;
}
@JsonProperty
public Map<String, Object> getEvent()
{
return event;
}
@Override
public List<String> getDimension(String dimension)
{
final Object dimValue = event.get(dimension);
if (dimValue == null) {
return Collections.emptyList();
} else if (dimValue instanceof List) {
// guava's toString function fails on null objects, so please do not use it
return Lists.transform(
(List) dimValue,
TO_STRING_INCLUDING_NULL);
} else {
return Collections.singletonList(String.valueOf(dimValue));
}
}
@Override
public Object getRaw(String dimension)
{
return event.get(dimension);
}
@Override
public float getFloatMetric(String metric)
{
Object metricValue = event.get(metric);
if (metricValue == null) {
return 0.0f;
}
if (metricValue instanceof Number) {
return ((Number) metricValue).floatValue();
} else if (metricValue instanceof String) {
try {
return Float.valueOf(((String) metricValue).replace(",", ""));
}
catch (Exception e) {
throw new ParseException(e, "Unable to parse metrics[%s], value[%s]", metric, metricValue);
}
} else {
throw new ParseException("Unknown type[%s]", metricValue.getClass());
}
}
@Override
public long getLongMetric(String metric)
{
Object metricValue = event.get(metric);
if (metricValue == null) {
return 0L;
}
if (metricValue instanceof Number) {
return ((Number) metricValue).longValue();
} else if (metricValue instanceof String) {
try {
String s = ((String) metricValue).replace(",", "");
return LONG_PAT.matcher(s).matches() ? Long.valueOf(s) : Double.valueOf(s).longValue();
}
catch (Exception e) {
throw new ParseException(e, "Unable to parse metrics[%s], value[%s]", metric, metricValue);
}
} else {
throw new ParseException("Unknown type[%s]", metricValue.getClass());
}
}
@Override
public String toString()
{
return "MapBasedRow{" +
"timestamp=" + timestamp +
", event=" + event +
'}';
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
MapBasedRow that = (MapBasedRow) o;
if (!event.equals(that.event)) {
return false;
}
if (!timestamp.equals(that.timestamp)) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = timestamp.hashCode();
result = 31 * result + event.hashCode();
return result;
}
@Override
public int compareTo(Row o)
{
return timestamp.compareTo(o.getTimestamp());
}
}

View File

@ -0,0 +1,94 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.common.parsers.ParseException;
import org.joda.time.DateTime;
import java.util.List;
/**
* A Row of data. This can be used for both input and output into various parts of the system. It assumes
* that the user already knows the schema of the row and can query for the parts that they care about.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "version", defaultImpl = MapBasedRow.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "v1", value = MapBasedRow.class)
})
public interface Row extends Comparable<Row>
{
/**
* Returns the timestamp from the epoch in milliseconds. If the event happened _right now_, this would return the
* same thing as System.currentTimeMillis();
*
* @return the timestamp from the epoch in milliseconds.
*/
public long getTimestampFromEpoch();
/**
* Returns the timestamp from the epoch as an org.joda.time.DateTime. If the event happened _right now_, this would return the
* same thing as new DateTime();
*
* @return the timestamp from the epoch as an org.joda.time.DateTime object.
*/
public DateTime getTimestamp();
/**
* Returns the list of dimension values for the given column name.
* <p/>
*
* @param dimension the column name of the dimension requested
*
* @return the list of values for the provided column name
*/
public List<String> getDimension(String dimension);
/**
* Returns the raw dimension value for the given column name. This is different from #getDimension which
* all values to strings before returning them.
*
* @param dimension the column name of the dimension requested
*
* @return the value of the provided column name
*/
public Object getRaw(String dimension);
/**
* Returns the float value of the given metric column.
* <p/>
*
* @param metric the column name of the metric requested
*
* @return the float value for the provided column name.
*/
public float getFloatMetric(String metric);
/**
* Returns the long value of the given metric column.
* <p/>
*
* @param metric the column name of the metric requested
*
* @return the long value for the provided column name.
*/
public long getLongMetric(String metric);
}

View File

@ -0,0 +1,71 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSortedSet;
import com.google.common.collect.Maps;
import com.metamx.common.ISE;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
/**
*/
public class Rows
{
public static InputRow toCaseInsensitiveInputRow(final Row row, final List<String> dimensions)
{
if (row instanceof MapBasedRow) {
MapBasedRow mapBasedRow = (MapBasedRow) row;
TreeMap<String, Object> caseInsensitiveMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
caseInsensitiveMap.putAll(mapBasedRow.getEvent());
return new MapBasedInputRow(
mapBasedRow.getTimestamp(),
dimensions,
caseInsensitiveMap
);
}
throw new ISE("Can only convert MapBasedRow objects because we are ghetto like that.");
}
/**
* @param timeStamp rollup up timestamp to be used to create group key
* @param inputRow input row
* @return groupKey for the given input row
*/
public static List<Object> toGroupKey(long timeStamp, InputRow inputRow)
{
final Map<String, Set<String>> dims = Maps.newTreeMap();
for (final String dim : inputRow.getDimensions()) {
final Set<String> dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim));
if (dimValues.size() > 0) {
dims.put(dim, dimValues);
}
}
return ImmutableList.of(
timeStamp,
dims
);
}
}

View File

@ -0,0 +1,100 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.metamx.common.parsers.CSVParser;
import com.metamx.common.parsers.ParseException;
import com.metamx.common.parsers.Parser;
import java.util.List;
/**
*/
public class CSVParseSpec extends ParseSpec
{
private final String listDelimiter;
private final List<String> columns;
@JsonCreator
public CSVParseSpec(
@JsonProperty("timestampSpec") TimestampSpec timestampSpec,
@JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
@JsonProperty("listDelimiter") String listDelimiter,
@JsonProperty("columns") List<String> columns
)
{
super(timestampSpec, dimensionsSpec);
this.listDelimiter = listDelimiter;
Preconditions.checkNotNull(columns, "columns");
for (String column : columns) {
Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column);
}
this.columns = columns;
verify(dimensionsSpec.getDimensionNames());
}
@JsonProperty
public String getListDelimiter()
{
return listDelimiter;
}
@JsonProperty("columns")
public List<String> getColumns()
{
return columns;
}
@Override
public void verify(List<String> usedCols)
{
for (String columnName : usedCols) {
Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName);
}
}
@Override
public Parser<String, Object> makeParser()
{
return new CSVParser(Optional.fromNullable(listDelimiter), columns);
}
@Override
public ParseSpec withTimestampSpec(TimestampSpec spec)
{
return new CSVParseSpec(spec, getDimensionsSpec(), listDelimiter, columns);
}
@Override
public ParseSpec withDimensionsSpec(DimensionsSpec spec)
{
return new CSVParseSpec(getTimestampSpec(), spec, listDelimiter, columns);
}
public ParseSpec withColumns(List<String> cols)
{
return new CSVParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, cols);
}
}

View File

@ -0,0 +1,124 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.metamx.common.parsers.DelimitedParser;
import com.metamx.common.parsers.ParseException;
import com.metamx.common.parsers.Parser;
import java.util.List;
/**
*/
public class DelimitedParseSpec extends ParseSpec
{
private final String delimiter;
private final String listDelimiter;
private final List<String> columns;
@JsonCreator
public DelimitedParseSpec(
@JsonProperty("timestampSpec") TimestampSpec timestampSpec,
@JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
@JsonProperty("delimiter") String delimiter,
@JsonProperty("listDelimiter") String listDelimiter,
@JsonProperty("columns") List<String> columns
)
{
super(timestampSpec, dimensionsSpec);
this.delimiter = delimiter;
this.listDelimiter = listDelimiter;
Preconditions.checkNotNull(columns, "columns");
this.columns = columns;
for (String column : this.columns) {
Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column);
}
verify(dimensionsSpec.getDimensionNames());
}
@JsonProperty("delimiter")
public String getDelimiter()
{
return delimiter;
}
@JsonProperty("listDelimiter")
public String getListDelimiter()
{
return listDelimiter;
}
@JsonProperty("columns")
public List<String> getColumns()
{
return columns;
}
@Override
public void verify(List<String> usedCols)
{
for (String columnName : usedCols) {
Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName);
}
}
@Override
public Parser<String, Object> makeParser()
{
Parser<String, Object> retVal = new DelimitedParser(
Optional.fromNullable(delimiter),
Optional.fromNullable(listDelimiter)
);
retVal.setFieldNames(columns);
return retVal;
}
@Override
public ParseSpec withTimestampSpec(TimestampSpec spec)
{
return new DelimitedParseSpec(spec, getDimensionsSpec(), delimiter, listDelimiter, columns);
}
@Override
public ParseSpec withDimensionsSpec(DimensionsSpec spec)
{
return new DelimitedParseSpec(getTimestampSpec(), spec, delimiter, listDelimiter, columns);
}
public ParseSpec withDelimiter(String delim)
{
return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delim, listDelimiter, columns);
}
public ParseSpec withListDelimiter(String delim)
{
return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delimiter, delim, columns);
}
public ParseSpec withColumns(List<String> cols)
{
return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delimiter, listDelimiter, cols);
}
}

View File

@ -0,0 +1,114 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.annotation.JsonValue;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.base.Preconditions;
import java.util.List;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringDimensionSchema.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = DimensionSchema.STRING_TYPE_NAME, value = StringDimensionSchema.class),
@JsonSubTypes.Type(name = DimensionSchema.LONG_TYPE_NAME, value = LongDimensionSchema.class),
@JsonSubTypes.Type(name = DimensionSchema.FLOAT_TYPE_NAME, value = FloatDimensionSchema.class),
@JsonSubTypes.Type(name = DimensionSchema.SPATIAL_TYPE_NAME, value = NewSpatialDimensionSchema.class),
})
public abstract class DimensionSchema
{
public static final String STRING_TYPE_NAME = "string";
public static final String LONG_TYPE_NAME = "long";
public static final String FLOAT_TYPE_NAME = "float";
public static final String SPATIAL_TYPE_NAME = "spatial";
// main druid and druid-api should really use the same ValueType enum.
// merge them when druid-api is merged back into the main repo
public enum ValueType
{
FLOAT,
LONG,
STRING,
COMPLEX;
@JsonValue
@Override
public String toString()
{
return this.name().toUpperCase();
}
@JsonCreator
public static ValueType fromString(String name)
{
return valueOf(name.toUpperCase());
}
}
private final String name;
protected DimensionSchema(String name)
{
this.name = Preconditions.checkNotNull(name, "Dimension name cannot be null.");
}
@JsonProperty
public String getName()
{
return name;
};
@JsonIgnore
public abstract String getTypeName();
@JsonIgnore
public abstract ValueType getValueType();
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DimensionSchema that = (DimensionSchema) o;
return name.equals(that.name);
}
@Override
public int hashCode()
{
return name.hashCode();
}
}

View File

@ -0,0 +1,234 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.metamx.common.parsers.ParserUtils;
import javax.annotation.Nullable;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
public class DimensionsSpec
{
private final List<DimensionSchema> dimensions;
private final Set<String> dimensionExclusions;
private final Map<String, DimensionSchema> dimensionSchemaMap;
public static List<DimensionSchema> getDefaultSchemas(List<String> dimNames)
{
return Lists.transform(
dimNames,
new Function<String, DimensionSchema>()
{
@Override
public DimensionSchema apply(String input)
{
return new StringDimensionSchema(input);
}
}
);
}
public static DimensionSchema convertSpatialSchema(SpatialDimensionSchema spatialSchema)
{
return new NewSpatialDimensionSchema(spatialSchema.getDimName(), spatialSchema.getDims());
}
@JsonCreator
public DimensionsSpec(
@JsonProperty("dimensions") List<DimensionSchema> dimensions,
@JsonProperty("dimensionExclusions") List<String> dimensionExclusions,
@Deprecated @JsonProperty("spatialDimensions") List<SpatialDimensionSchema> spatialDimensions
)
{
this.dimensions = dimensions == null
? Lists.<DimensionSchema>newArrayList()
: Lists.newArrayList(dimensions);
this.dimensionExclusions = (dimensionExclusions == null)
? Sets.<String>newHashSet()
: Sets.newHashSet(dimensionExclusions);
List<SpatialDimensionSchema> spatialDims = (spatialDimensions == null)
? Lists.<SpatialDimensionSchema>newArrayList()
: spatialDimensions;
verify(spatialDims);
// Map for easy dimension name-based schema lookup
this.dimensionSchemaMap = new HashMap<>();
for (DimensionSchema schema : this.dimensions) {
dimensionSchemaMap.put(schema.getName(), schema);
}
for(SpatialDimensionSchema spatialSchema : spatialDims) {
DimensionSchema newSchema = DimensionsSpec.convertSpatialSchema(spatialSchema);
this.dimensions.add(newSchema);
dimensionSchemaMap.put(newSchema.getName(), newSchema);
}
}
@JsonProperty
public List<DimensionSchema> getDimensions()
{
return dimensions;
}
@JsonProperty
public Set<String> getDimensionExclusions()
{
return dimensionExclusions;
}
@Deprecated @JsonIgnore
public List<SpatialDimensionSchema> getSpatialDimensions()
{
Iterable<NewSpatialDimensionSchema> filteredList = Iterables.filter(
dimensions, NewSpatialDimensionSchema.class
);
Iterable<SpatialDimensionSchema> transformedList = Iterables.transform(
filteredList,
new Function<NewSpatialDimensionSchema, SpatialDimensionSchema>()
{
@Nullable
@Override
public SpatialDimensionSchema apply(NewSpatialDimensionSchema input)
{
return new SpatialDimensionSchema(input.getName(), input.getDims());
}
}
);
return Lists.newArrayList(transformedList);
}
@JsonIgnore
public List<String> getDimensionNames()
{
return Lists.transform(
dimensions,
new Function<DimensionSchema, String>()
{
@Override
public String apply(DimensionSchema input)
{
return input.getName();
}
}
);
}
public DimensionSchema getSchema(String dimension)
{
return dimensionSchemaMap.get(dimension);
}
public boolean hasCustomDimensions()
{
return !(dimensions == null || dimensions.isEmpty());
}
public DimensionsSpec withDimensions(List<DimensionSchema> dims)
{
return new DimensionsSpec(dims, ImmutableList.copyOf(dimensionExclusions), null);
}
public DimensionsSpec withDimensionExclusions(Set<String> dimExs)
{
return new DimensionsSpec(
dimensions,
ImmutableList.copyOf(Sets.union(dimensionExclusions, dimExs)),
null
);
}
@Deprecated
public DimensionsSpec withSpatialDimensions(List<SpatialDimensionSchema> spatials)
{
return new DimensionsSpec(dimensions, ImmutableList.copyOf(dimensionExclusions), spatials);
}
private void verify(List<SpatialDimensionSchema> spatialDimensions)
{
List<String> dimNames = getDimensionNames();
Preconditions.checkArgument(
Sets.intersection(this.dimensionExclusions, Sets.newHashSet(dimNames)).isEmpty(),
"dimensions and dimensions exclusions cannot overlap"
);
ParserUtils.validateFields(dimNames);
ParserUtils.validateFields(dimensionExclusions);
List<String> spatialDimNames = Lists.transform(
spatialDimensions,
new Function<SpatialDimensionSchema, String>()
{
@Override
public String apply(SpatialDimensionSchema input)
{
return input.getDimName();
}
}
);
// Don't allow duplicates between main list and deprecated spatial list
ParserUtils.validateFields(Iterables.concat(dimNames, spatialDimNames));
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DimensionsSpec that = (DimensionsSpec) o;
if (!dimensions.equals(that.dimensions)) {
return false;
}
return dimensionExclusions.equals(that.dimensionExclusions);
}
@Override
public int hashCode()
{
int result = dimensions.hashCode();
result = 31 * result + dimensionExclusions.hashCode();
return result;
}
}

View File

@ -0,0 +1,92 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.google.common.base.Throwables;
import io.druid.data.input.Firehose;
import io.druid.data.input.InputRow;
import io.druid.utils.Runnables;
import org.apache.commons.io.LineIterator;
import java.io.IOException;
import java.util.Iterator;
/**
*/
public class FileIteratingFirehose implements Firehose
{
private final Iterator<LineIterator> lineIterators;
private final StringInputRowParser parser;
private LineIterator lineIterator = null;
public FileIteratingFirehose(
Iterator<LineIterator> lineIterators,
StringInputRowParser parser
)
{
this.lineIterators = lineIterators;
this.parser = parser;
}
@Override
public boolean hasMore()
{
while ((lineIterator == null || !lineIterator.hasNext()) && lineIterators.hasNext()) {
lineIterator = lineIterators.next();
}
return lineIterator != null && lineIterator.hasNext();
}
@Override
public InputRow nextRow()
{
try {
if (lineIterator == null || !lineIterator.hasNext()) {
// Close old streams, maybe.
if (lineIterator != null) {
lineIterator.close();
}
lineIterator = lineIterators.next();
}
return parser.parse(lineIterator.next());
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override
public Runnable commit()
{
return Runnables.getNoopRunnable();
}
@Override
public void close() throws IOException
{
if (lineIterator != null) {
lineIterator.close();
}
}
}

View File

@ -0,0 +1,48 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
public class FloatDimensionSchema extends DimensionSchema
{
@JsonCreator
public FloatDimensionSchema(
@JsonProperty("name") String name
)
{
super(name);
}
@Override
public String getTypeName()
{
return DimensionSchema.FLOAT_TYPE_NAME;
}
@Override
@JsonIgnore
public ValueType getValueType()
{
return ValueType.FLOAT;
}
}

View File

@ -0,0 +1,38 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.common.parsers.ParseException;
import io.druid.data.input.InputRow;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "string", value = StringInputRowParser.class),
@JsonSubTypes.Type(name = "map", value = MapInputRowParser.class),
@JsonSubTypes.Type(name = "noop", value = NoopInputRowParser.class)
})
public interface InputRowParser<T>
{
public InputRow parse(T input) ;
public ParseSpec getParseSpec();
public InputRowParser withParseSpec(ParseSpec parseSpec) ;
}

View File

@ -0,0 +1,68 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.common.parsers.JSONToLowerParser;
import com.metamx.common.parsers.Parser;
import java.util.List;
/**
* This class is only here for backwards compatibility
*/
@Deprecated
public class JSONLowercaseParseSpec extends ParseSpec
{
private final ObjectMapper objectMapper;
@JsonCreator
public JSONLowercaseParseSpec(
@JsonProperty("timestampSpec") TimestampSpec timestampSpec,
@JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec
)
{
super(timestampSpec, dimensionsSpec);
this.objectMapper = new ObjectMapper();
}
@Override
public void verify(List<String> usedCols)
{
}
@Override
public Parser<String, Object> makeParser()
{
return new JSONToLowerParser(objectMapper, null, null);
}
@Override
public ParseSpec withTimestampSpec(TimestampSpec spec)
{
return new JSONLowercaseParseSpec(spec, getDimensionsSpec());
}
@Override
public ParseSpec withDimensionsSpec(DimensionsSpec spec)
{
return new JSONLowercaseParseSpec(getTimestampSpec(), spec);
}
}

View File

@ -0,0 +1,128 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.JsonParser.Feature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.common.parsers.JSONPathParser;
import com.metamx.common.parsers.Parser;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
*/
public class JSONParseSpec extends ParseSpec
{
private final ObjectMapper objectMapper;
private final JSONPathSpec flattenSpec;
private final Map<String, Boolean> featureSpec;
@JsonCreator
public JSONParseSpec(
@JsonProperty("timestampSpec") TimestampSpec timestampSpec,
@JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
@JsonProperty("flattenSpec") JSONPathSpec flattenSpec,
@JsonProperty("featureSpec") Map<String, Boolean> featureSpec
)
{
super(timestampSpec, dimensionsSpec);
this.objectMapper = new ObjectMapper();
this.flattenSpec = flattenSpec != null ? flattenSpec : new JSONPathSpec(true, null);
this.featureSpec = (featureSpec == null) ? new HashMap<String, Boolean>() : featureSpec;
for (Map.Entry<String, Boolean> entry : this.featureSpec.entrySet()) {
Feature feature = Feature.valueOf(entry.getKey());
objectMapper.configure(feature, entry.getValue());
}
}
@Deprecated
public JSONParseSpec(TimestampSpec ts, DimensionsSpec dims)
{
this(ts, dims, null, null);
}
@Override
public void verify(List<String> usedCols)
{
}
@Override
public Parser<String, Object> makeParser()
{
return new JSONPathParser(
convertFieldSpecs(flattenSpec.getFields()),
flattenSpec.isUseFieldDiscovery(),
objectMapper
);
}
@Override
public ParseSpec withTimestampSpec(TimestampSpec spec)
{
return new JSONParseSpec(spec, getDimensionsSpec(), getFlattenSpec(), getFeatureSpec());
}
@Override
public ParseSpec withDimensionsSpec(DimensionsSpec spec)
{
return new JSONParseSpec(getTimestampSpec(), spec, getFlattenSpec(), getFeatureSpec());
}
@JsonProperty
public JSONPathSpec getFlattenSpec()
{
return flattenSpec;
}
@JsonProperty
public Map<String, Boolean> getFeatureSpec()
{
return featureSpec;
}
private List<JSONPathParser.FieldSpec> convertFieldSpecs(List<JSONPathFieldSpec> druidFieldSpecs)
{
List<JSONPathParser.FieldSpec> newSpecs = new ArrayList<>();
for (JSONPathFieldSpec druidSpec : druidFieldSpecs) {
JSONPathParser.FieldType type;
switch (druidSpec.getType()) {
case ROOT:
type = JSONPathParser.FieldType.ROOT;
break;
case PATH:
type = JSONPathParser.FieldType.PATH;
break;
default:
throw new IllegalArgumentException("Invalid type for field " + druidSpec.getName());
}
JSONPathParser.FieldSpec newSpec = new JSONPathParser.FieldSpec(
type,
druidSpec.getName(),
druidSpec.getExpr()
);
newSpecs.add(newSpec);
}
return newSpecs;
}
}

View File

@ -0,0 +1,76 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
public class JSONPathFieldSpec
{
private final JSONPathFieldType type;
private final String name;
private final String expr;
@JsonCreator
public JSONPathFieldSpec(
@JsonProperty("type") JSONPathFieldType type,
@JsonProperty("name") String name,
@JsonProperty("expr") String expr
)
{
this.type = type;
this.name = name;
this.expr = expr;
}
@JsonProperty
public JSONPathFieldType getType()
{
return type;
}
@JsonProperty
public String getName()
{
return name;
}
@JsonProperty
public String getExpr()
{
return expr;
}
@JsonCreator
public static JSONPathFieldSpec fromString(String name)
{
return JSONPathFieldSpec.createRootField(name);
}
public static JSONPathFieldSpec createNestedField(String name, String expr)
{
return new JSONPathFieldSpec(JSONPathFieldType.PATH, name, expr);
}
public static JSONPathFieldSpec createRootField(String name)
{
return new JSONPathFieldSpec(JSONPathFieldType.ROOT, name, name);
}
}

View File

@ -0,0 +1,42 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonValue;
public enum JSONPathFieldType
{
ROOT,
PATH;
@JsonValue
@Override
public String toString()
{
return this.name().toLowerCase();
}
@JsonCreator
public static JSONPathFieldType fromString(String name)
{
return valueOf(name.toUpperCase());
}
}

View File

@ -0,0 +1,54 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import java.util.List;
public class JSONPathSpec
{
private final boolean useFieldDiscovery;
private final List<JSONPathFieldSpec> fields;
@JsonCreator
public JSONPathSpec(
@JsonProperty("useFieldDiscovery") Boolean useFieldDiscovery,
@JsonProperty("fields") List<JSONPathFieldSpec> fields
)
{
this.useFieldDiscovery = useFieldDiscovery == null ? true : useFieldDiscovery;
this.fields = fields == null ? ImmutableList.<JSONPathFieldSpec>of() : fields;
}
@JsonProperty
public boolean isUseFieldDiscovery()
{
return useFieldDiscovery;
}
@JsonProperty
public List<JSONPathFieldSpec> getFields()
{
return fields;
}
}

View File

@ -0,0 +1,80 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.common.parsers.JavaScriptParser;
import com.metamx.common.parsers.Parser;
import java.util.List;
/**
*/
public class JavaScriptParseSpec extends ParseSpec
{
private final String function;
@JsonCreator
public JavaScriptParseSpec(
@JsonProperty("timestampSpec") TimestampSpec timestampSpec,
@JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
@JsonProperty("function") String function
)
{
super(timestampSpec, dimensionsSpec);
this.function = function;
}
@JsonProperty("function")
public String getFunction()
{
return function;
}
@Override
public void verify(List<String> usedCols)
{
}
@Override
public Parser<String, Object> makeParser()
{
return new JavaScriptParser(function);
}
@Override
public ParseSpec withTimestampSpec(TimestampSpec spec)
{
return new JavaScriptParseSpec(spec, getDimensionsSpec(), function);
}
@Override
public ParseSpec withDimensionsSpec(DimensionsSpec spec)
{
return new JavaScriptParseSpec(getTimestampSpec(), spec, function);
}
public ParseSpec withFunction(String fn)
{
return new JavaScriptParseSpec(getTimestampSpec(), getDimensionsSpec(), fn);
}
}

View File

@ -0,0 +1,48 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
public class LongDimensionSchema extends DimensionSchema
{
@JsonCreator
public LongDimensionSchema(
@JsonProperty("name") String name
)
{
super(name);
}
@Override
public String getTypeName()
{
return DimensionSchema.LONG_TYPE_NAME;
}
@Override
@JsonIgnore
public ValueType getValueType()
{
return ValueType.LONG;
}
}

View File

@ -0,0 +1,90 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.metamx.common.logger.Logger;
import com.metamx.common.parsers.ParseException;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import org.joda.time.DateTime;
import java.util.List;
import java.util.Map;
public class MapInputRowParser implements InputRowParser<Map<String, Object>>
{
private final ParseSpec parseSpec;
@JsonCreator
public MapInputRowParser(
@JsonProperty("parseSpec") ParseSpec parseSpec
)
{
this.parseSpec = parseSpec;
}
@Override
public InputRow parse(Map<String, Object> theMap)
{
final List<String> dimensions = parseSpec.getDimensionsSpec().hasCustomDimensions()
? parseSpec.getDimensionsSpec().getDimensionNames()
: Lists.newArrayList(
Sets.difference(
theMap.keySet(),
parseSpec.getDimensionsSpec()
.getDimensionExclusions()
)
);
final DateTime timestamp;
try {
timestamp = parseSpec.getTimestampSpec().extractTimestamp(theMap);
if (timestamp == null) {
final String input = theMap.toString();
throw new NullPointerException(
String.format(
"Null timestamp in input: %s",
input.length() < 100 ? input : input.substring(0, 100) + "..."
)
);
}
}
catch (Exception e) {
throw new ParseException(e, "Unparseable timestamp found!");
}
return new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap);
}
@JsonProperty
@Override
public ParseSpec getParseSpec()
{
return parseSpec;
}
@Override
public InputRowParser withParseSpec(ParseSpec parseSpec)
{
return new MapInputRowParser(parseSpec);
}
}

View File

@ -0,0 +1,88 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
/**
* NOTE:
* This class should be deprecated after Druid supports configurable index types on dimensions.
* When that exists, this should be the implementation: https://github.com/druid-io/druid/issues/2622
*
* This is a stop-gap solution to consolidate the dimension specs and remove the separate spatial
* section in DimensionsSpec.
*/
public class NewSpatialDimensionSchema extends DimensionSchema
{
private final List<String> dims;
@JsonCreator
public NewSpatialDimensionSchema(
@JsonProperty("name") String name,
@JsonProperty("dims") List<String> dims
)
{
super(name);
this.dims = dims;
}
@JsonProperty
public List<String> getDims()
{
return dims;
}
@Override
public String getTypeName()
{
return DimensionSchema.SPATIAL_TYPE_NAME;
}
@Override
@JsonIgnore
public ValueType getValueType()
{
return ValueType.STRING;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
NewSpatialDimensionSchema that = (NewSpatialDimensionSchema) o;
return dims != null ? dims.equals(that.dims) : that.dims == null;
}
@Override
public int hashCode()
{
return dims != null ? dims.hashCode() : 0;
}
}

View File

@ -0,0 +1,79 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.data.input.InputRow;
/**
*/
public class NoopInputRowParser implements InputRowParser<InputRow>
{
private final ParseSpec parseSpec;
@JsonCreator
public NoopInputRowParser(
@JsonProperty("parseSpec") ParseSpec parseSpec
)
{
this.parseSpec = parseSpec != null ? parseSpec : new TimeAndDimsParseSpec(null, null);
}
@Override
public InputRow parse(InputRow input)
{
return input;
}
@Override
public ParseSpec getParseSpec()
{
return parseSpec;
}
@Override
public InputRowParser withParseSpec(ParseSpec parseSpec)
{
return new NoopInputRowParser(parseSpec);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
NoopInputRowParser that = (NoopInputRowParser) o;
return parseSpec.equals(that.parseSpec);
}
@Override
public int hashCode()
{
return parseSpec.hashCode();
}
}

View File

@ -0,0 +1,111 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.common.parsers.Parser;
import java.util.List;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format", defaultImpl = DelimitedParseSpec.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "json", value = JSONParseSpec.class),
@JsonSubTypes.Type(name = "csv", value = CSVParseSpec.class),
@JsonSubTypes.Type(name = "tsv", value = DelimitedParseSpec.class),
@JsonSubTypes.Type(name = "jsonLowercase", value = JSONLowercaseParseSpec.class),
@JsonSubTypes.Type(name = "timeAndDims", value = TimeAndDimsParseSpec.class),
@JsonSubTypes.Type(name = "regex", value = RegexParseSpec.class),
@JsonSubTypes.Type(name = "javascript", value = JavaScriptParseSpec.class)
})
public abstract class ParseSpec
{
private final TimestampSpec timestampSpec;
private final DimensionsSpec dimensionsSpec;
protected ParseSpec(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec)
{
this.timestampSpec = timestampSpec;
this.dimensionsSpec = dimensionsSpec;
}
@JsonProperty
public TimestampSpec getTimestampSpec()
{
return timestampSpec;
}
@JsonProperty
public DimensionsSpec getDimensionsSpec()
{
return dimensionsSpec;
}
public void verify(List<String> usedCols)
{
// do nothing
}
public Parser<String, Object> makeParser()
{
return null;
}
public ParseSpec withTimestampSpec(TimestampSpec spec)
{
throw new UnsupportedOperationException();
}
public ParseSpec withDimensionsSpec(DimensionsSpec spec)
{
throw new UnsupportedOperationException();
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
ParseSpec parseSpec = (ParseSpec) o;
if (timestampSpec != null ? !timestampSpec.equals(parseSpec.timestampSpec) : parseSpec.timestampSpec != null) {
return false;
}
return !(dimensionsSpec != null
? !dimensionsSpec.equals(parseSpec.dimensionsSpec)
: parseSpec.dimensionsSpec != null);
}
@Override
public int hashCode()
{
int result = timestampSpec != null ? timestampSpec.hashCode() : 0;
result = 31 * result + (dimensionsSpec != null ? dimensionsSpec.hashCode() : 0);
return result;
}
}

View File

@ -0,0 +1,115 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.metamx.common.parsers.Parser;
import com.metamx.common.parsers.RegexParser;
import java.util.List;
/**
*/
public class RegexParseSpec extends ParseSpec
{
private final String listDelimiter;
private final List<String> columns;
private final String pattern;
@JsonCreator
public RegexParseSpec(
@JsonProperty("timestampSpec") TimestampSpec timestampSpec,
@JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
@JsonProperty("listDelimiter") String listDelimiter,
@JsonProperty("columns") List<String> columns,
@JsonProperty("pattern") String pattern
)
{
super(timestampSpec, dimensionsSpec);
this.listDelimiter = listDelimiter;
this.columns = columns;
this.pattern = pattern;
verify(dimensionsSpec.getDimensionNames());
}
@JsonProperty
public String getListDelimiter()
{
return listDelimiter;
}
@JsonProperty("pattern")
public String getPattern()
{
return pattern;
}
@JsonProperty
public List<String> getColumns()
{
return columns;
}
@Override
public void verify(List<String> usedCols)
{
if (columns != null) {
for (String columnName : usedCols) {
Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName);
}
}
}
@Override
public Parser<String, Object> makeParser()
{
if (columns == null) {
return new RegexParser(pattern, Optional.fromNullable(listDelimiter));
}
return new RegexParser(pattern, Optional.fromNullable(listDelimiter), columns);
}
@Override
public ParseSpec withTimestampSpec(TimestampSpec spec)
{
return new RegexParseSpec(spec, getDimensionsSpec(), listDelimiter, columns, pattern);
}
@Override
public ParseSpec withDimensionsSpec(DimensionsSpec spec)
{
return new RegexParseSpec(getTimestampSpec(), spec, listDelimiter, columns, pattern);
}
public ParseSpec withColumns(List<String> cols)
{
return new RegexParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, cols, pattern);
}
public ParseSpec withPattern(String pat)
{
return new RegexParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, columns, pat);
}
}

View File

@ -0,0 +1,82 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
/**
*/
@Deprecated
public class SpatialDimensionSchema
{
private final String dimName;
private final List<String> dims;
@JsonCreator
public SpatialDimensionSchema(
@JsonProperty("dimName") String dimName,
@JsonProperty("dims") List<String> dims
)
{
this.dimName = dimName;
this.dims = dims;
}
@JsonProperty
public String getDimName()
{
return dimName;
}
@JsonProperty
public List<String> getDims()
{
return dims;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SpatialDimensionSchema that = (SpatialDimensionSchema) o;
if (dimName != null ? !dimName.equals(that.dimName) : that.dimName != null) {
return false;
}
return dims != null ? dims.equals(that.dims) : that.dims == null;
}
@Override
public int hashCode()
{
int result = dimName != null ? dimName.hashCode() : 0;
result = 31 * result + (dims != null ? dims.hashCode() : 0);
return result;
}
}

View File

@ -0,0 +1,53 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
public class StringDimensionSchema extends DimensionSchema
{
@JsonCreator
public static StringDimensionSchema create(String name) {
return new StringDimensionSchema(name);
}
@JsonCreator
public StringDimensionSchema(
@JsonProperty("name") String name
)
{
super(name);
}
@Override
public String getTypeName()
{
return DimensionSchema.STRING_TYPE_NAME;
}
@Override
@JsonIgnore
public ValueType getValueType()
{
return ValueType.STRING;
}
}

View File

@ -0,0 +1,138 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Charsets;
import com.metamx.common.parsers.ParseException;
import com.metamx.common.parsers.Parser;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.InputRow;
import java.nio.ByteBuffer;
import java.nio.CharBuffer;
import java.nio.charset.Charset;
import java.nio.charset.CoderResult;
import java.nio.charset.CodingErrorAction;
import java.util.Map;
/**
*/
public class StringInputRowParser implements ByteBufferInputRowParser
{
private static final Charset DEFAULT_CHARSET = Charsets.UTF_8;
private final ParseSpec parseSpec;
private final MapInputRowParser mapParser;
private final Parser<String, Object> parser;
private final Charset charset;
private CharBuffer chars = null;
@JsonCreator
public StringInputRowParser(
@JsonProperty("parseSpec") ParseSpec parseSpec,
@JsonProperty("encoding") String encoding
)
{
this.parseSpec = parseSpec;
this.mapParser = new MapInputRowParser(parseSpec);
this.parser = parseSpec.makeParser();
if (encoding != null) {
this.charset = Charset.forName(encoding);
} else {
this.charset = DEFAULT_CHARSET;
}
}
@Deprecated
public StringInputRowParser(ParseSpec parseSpec)
{
this(parseSpec, null);
}
@Override
public InputRow parse(ByteBuffer input)
{
return parseMap(buildStringKeyMap(input));
}
@JsonProperty
@Override
public ParseSpec getParseSpec()
{
return parseSpec;
}
@JsonProperty
public String getEncoding()
{
return charset.name();
}
@Override
public StringInputRowParser withParseSpec(ParseSpec parseSpec)
{
return new StringInputRowParser(parseSpec, getEncoding());
}
private Map<String, Object> buildStringKeyMap(ByteBuffer input)
{
int payloadSize = input.remaining();
if (chars == null || chars.remaining() < payloadSize) {
chars = CharBuffer.allocate(payloadSize);
}
final CoderResult coderResult = charset.newDecoder()
.onMalformedInput(CodingErrorAction.REPLACE)
.onUnmappableCharacter(CodingErrorAction.REPLACE)
.decode(input, chars, true);
Map<String, Object> theMap;
if (coderResult.isUnderflow()) {
chars.flip();
try {
theMap = parseString(chars.toString());
}
finally {
chars.clear();
}
} else {
throw new ParseException("Failed with CoderResult[%s]", coderResult);
}
return theMap;
}
private Map<String, Object> parseString(String inputString)
{
return parser.parse(inputString);
}
public InputRow parse(String input)
{
return parseMap(parseString(input));
}
private InputRow parseMap(Map<String, Object> theMap)
{
return mapParser.parse(theMap);
}
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.common.parsers.Parser;
import java.util.List;
import java.util.Map;
/**
*/
public class TimeAndDimsParseSpec extends ParseSpec
{
@JsonCreator
public TimeAndDimsParseSpec(
@JsonProperty("timestampSpec") TimestampSpec timestampSpec,
@JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec
)
{
super(
timestampSpec != null ? timestampSpec : new TimestampSpec(null, null, null),
dimensionsSpec != null ? dimensionsSpec : new DimensionsSpec(null, null, null)
);
}
public Parser<String, Object> makeParser()
{
return new Parser<String, Object>()
{
@Override
public Map<String, Object> parse(String input)
{
throw new UnsupportedOperationException("not supported");
}
@Override
public void setFieldNames(Iterable<String> fieldNames)
{
throw new UnsupportedOperationException("not supported");
}
@Override
public List<String> getFieldNames()
{
throw new UnsupportedOperationException("not supported");
}
};
}
public ParseSpec withTimestampSpec(TimestampSpec spec)
{
return new TimeAndDimsParseSpec(spec, getDimensionsSpec());
}
public ParseSpec withDimensionsSpec(DimensionsSpec spec)
{
return new TimeAndDimsParseSpec(getTimestampSpec(), spec);
}
}

View File

@ -0,0 +1,114 @@
/*
* Druid - a distributed column store.
* Copyright 2012 - 2015 Metamarkets Group Inc.
*
* Licensed 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 io.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.metamx.common.parsers.ParserUtils;
import com.metamx.common.parsers.TimestampParser;
import org.joda.time.DateTime;
import java.util.Map;
/**
*/
public class TimestampSpec
{
private static final String DEFAULT_COLUMN = "timestamp";
private static final String DEFAULT_FORMAT = "auto";
private static final DateTime DEFAULT_MISSING_VALUE = null;
private final String timestampColumn;
private final String timestampFormat;
private final Function<Object, DateTime> timestampConverter;
// this value should never be set for production data
private final DateTime missingValue;
@JsonCreator
public TimestampSpec(
@JsonProperty("column") String timestampColumn,
@JsonProperty("format") String format,
// this value should never be set for production data
@JsonProperty("missingValue") DateTime missingValue
)
{
this.timestampColumn = (timestampColumn == null) ? DEFAULT_COLUMN : timestampColumn;
this.timestampFormat = format == null ? DEFAULT_FORMAT : format;
this.timestampConverter = TimestampParser.createObjectTimestampParser(timestampFormat);
this.missingValue = missingValue == null
? DEFAULT_MISSING_VALUE
: missingValue;
}
@JsonProperty("column")
public String getTimestampColumn()
{
return timestampColumn;
}
@JsonProperty("format")
public String getTimestampFormat()
{
return timestampFormat;
}
@JsonProperty("missingValue")
public DateTime getMissingValue()
{
return missingValue;
}
public DateTime extractTimestamp(Map<String, Object> input)
{
final Object o = input.get(timestampColumn);
return o == null ? missingValue : timestampConverter.apply(o);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TimestampSpec that = (TimestampSpec) o;
if (!timestampColumn.equals(that.timestampColumn)) {
return false;
}
if (!timestampFormat.equals(that.timestampFormat)) {
return false;
}
return !(missingValue != null ? !missingValue.equals(that.missingValue) : that.missingValue != null);
}
@Override
public int hashCode()
{
int result = timestampColumn.hashCode();
result = 31 * result + timestampFormat.hashCode();
result = 31 * result + (missingValue != null ? missingValue.hashCode() : 0);
return result;
}
}

View File

@ -0,0 +1,71 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.Binder;
import com.google.inject.Key;
import com.google.inject.multibindings.MapBinder;
import io.druid.segment.loading.DataSegmentArchiver;
import io.druid.segment.loading.DataSegmentFinder;
import io.druid.segment.loading.DataSegmentMover;
import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.DataSegmentPuller;
import io.druid.segment.loading.DataSegmentPusher;
import io.druid.tasklogs.TaskLogs;
/**
*/
public class Binders
{
public static MapBinder<String, DataSegmentPuller> dataSegmentPullerBinder(Binder binder)
{
return MapBinder.newMapBinder(binder, String.class, DataSegmentPuller.class);
}
public static MapBinder<String, DataSegmentKiller> dataSegmentKillerBinder(Binder binder)
{
return MapBinder.newMapBinder(binder, String.class, DataSegmentKiller.class);
}
public static MapBinder<String, DataSegmentMover> dataSegmentMoverBinder(Binder binder)
{
return MapBinder.newMapBinder(binder, String.class, DataSegmentMover.class);
}
public static MapBinder<String, DataSegmentArchiver> dataSegmentArchiverBinder(Binder binder)
{
return MapBinder.newMapBinder(binder, String.class, DataSegmentArchiver.class);
}
public static MapBinder<String, DataSegmentPusher> dataSegmentPusherBinder(Binder binder)
{
return PolyBind.optionBinder(binder, Key.get(DataSegmentPusher.class));
}
public static MapBinder<String, DataSegmentFinder> dataSegmentFinderBinder(Binder binder)
{
return PolyBind.optionBinder(binder, Key.get(DataSegmentFinder.class));
}
public static MapBinder<String, TaskLogs> taskLogsBinder(Binder binder)
{
return PolyBind.optionBinder(binder, Key.get(TaskLogs.class));
}
}

View File

@ -0,0 +1,34 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.Binder;
import com.google.inject.Module;
/**
*/
public class DruidGuiceExtensions implements Module
{
@Override
public void configure(Binder binder)
{
binder.bindScope(LazySingleton.class, DruidScopes.SINGLETON);
}
}

View File

@ -0,0 +1,45 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.Key;
import com.google.inject.Provider;
import com.google.inject.Scope;
import com.google.inject.Scopes;
/**
*/
public class DruidScopes
{
public static final Scope SINGLETON = new Scope()
{
@Override
public <T> Provider<T> scope(Key<T> key, Provider<T> unscoped)
{
return Scopes.SINGLETON.scope(key, unscoped);
}
@Override
public String toString()
{
return "DruidScopes.SINGLETON";
}
};
}

View File

@ -0,0 +1,37 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.Binder;
import com.google.inject.TypeLiteral;
import com.google.inject.multibindings.Multibinder;
import io.druid.guice.annotations.JSR311Resource;
/**
*/
public class Jerseys
{
public static void addResource(Binder binder, Class<?> resourceClazz)
{
Multibinder.newSetBinder(binder, new TypeLiteral<Class<?>>(){}, JSR311Resource.class)
.addBinding()
.toInstance(resourceClazz);
}
}

View File

@ -0,0 +1,213 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.inject.Binder;
import com.google.inject.Inject;
import com.google.inject.Key;
import com.google.inject.Provider;
import com.google.inject.util.Types;
import java.lang.annotation.Annotation;
import java.lang.reflect.ParameterizedType;
import java.util.Properties;
/**
* Provides a singleton value of type {@code <T>} from {@code Properties} bound in guice.
* <br/>
* <h3>Usage</h3>
* To install this provider, bind it in your guice module, like below.
*
* <pre>
* JsonConfigProvider.bind(binder, "druid.server", DruidServerConfig.class);
* </pre>
* <br/>
* In the above case, {@code druid.server} should be a key found in the {@code Properties} bound elsewhere.
* The value of that key should directly relate to the fields in {@code DruidServerConfig.class}.
*
* <h3>Implementation</h3>
* <br/>
* The state of {@code <T>} is defined by the value of the property {@code propertyBase}.
* This value is a json structure, decoded via {@link JsonConfigurator#configurate(java.util.Properties, String, Class)}.
* <br/>
*
* An example might be if DruidServerConfig.class were
*
* <pre>
* public class DruidServerConfig
* {
* @JsonProperty @NotNull public String hostname = null;
* @JsonProperty @Min(1025) public int port = 8080;
* }
* </pre>
*
* And your Properties object had in it
*
* <pre>
* druid.server.hostname=0.0.0.0
* druid.server.port=3333
* </pre>
*
* Then this would bind a singleton instance of a DruidServerConfig object with hostname = "0.0.0.0" and port = 3333.
*
* If the port weren't set in the properties, then the default of 8080 would be taken. Essentially, it is the same as
* subtracting the "druid.server" prefix from the properties and building a Map which is then passed into
* ObjectMapper.convertValue()
*
* @param <T> type of config object to provide.
*/
public class JsonConfigProvider<T> implements Provider<Supplier<T>>
{
@SuppressWarnings("unchecked")
public static <T> void bind(Binder binder, String propertyBase, Class<T> classToProvide)
{
bind(
binder,
propertyBase,
classToProvide,
Key.get(classToProvide),
(Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide))
);
}
@SuppressWarnings("unchecked")
public static <T> void bind(Binder binder, String propertyBase, Class<T> classToProvide, Annotation annotation)
{
bind(
binder,
propertyBase,
classToProvide,
Key.get(classToProvide, annotation),
(Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide), annotation)
);
}
@SuppressWarnings("unchecked")
public static <T> void bind(
Binder binder,
String propertyBase,
Class<T> classToProvide,
Class<? extends Annotation> annotation
)
{
bind(
binder,
propertyBase,
classToProvide,
Key.get(classToProvide, annotation),
(Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide), annotation)
);
}
@SuppressWarnings("unchecked")
public static <T> void bind(
Binder binder,
String propertyBase,
Class<T> clazz,
Key<T> instanceKey,
Key<Supplier<T>> supplierKey
)
{
binder.bind(supplierKey).toProvider((Provider) of(propertyBase, clazz)).in(LazySingleton.class);
binder.bind(instanceKey).toProvider(new SupplierProvider<T>(supplierKey));
}
@SuppressWarnings("unchecked")
public static <T> void bindInstance(
Binder binder,
Key<T> bindKey,
T instance
)
{
binder.bind(bindKey).toInstance(instance);
final ParameterizedType supType = Types.newParameterizedType(Supplier.class, bindKey.getTypeLiteral().getType());
final Key supplierKey;
if (bindKey.getAnnotationType() != null) {
supplierKey = Key.get(supType, bindKey.getAnnotationType());
}
else if (bindKey.getAnnotation() != null) {
supplierKey = Key.get(supType, bindKey.getAnnotation());
}
else {
supplierKey = Key.get(supType);
}
binder.bind(supplierKey).toInstance(Suppliers.<T>ofInstance(instance));
}
public static <T> JsonConfigProvider<T> of(String propertyBase, Class<T> classToProvide)
{
return new JsonConfigProvider<T>(propertyBase, classToProvide);
}
private final String propertyBase;
private final Class<T> classToProvide;
private Properties props;
private JsonConfigurator configurator;
private Supplier<T> retVal = null;
public JsonConfigProvider(
String propertyBase,
Class<T> classToProvide
)
{
this.propertyBase = propertyBase;
this.classToProvide = classToProvide;
}
@Inject
public void inject(
Properties props,
JsonConfigurator configurator
)
{
this.props = props;
this.configurator = configurator;
}
@Override
public Supplier<T> get()
{
if (retVal != null) {
return retVal;
}
try {
final T config = configurator.configurate(props, propertyBase, classToProvide);
retVal = Suppliers.ofInstance(config);
}
catch (RuntimeException e) {
// When a runtime exception gets thrown out, this provider will get called again if the object is asked for again.
// This will have the same failed result, 'cause when it's called no parameters will have actually changed.
// Guice will then report the same error multiple times, which is pretty annoying. Cache a null supplier and
// return that instead. This is technically enforcing a singleton, but such is life.
retVal = Suppliers.ofInstance(null);
throw e;
}
return retVal;
}
}

View File

@ -0,0 +1,184 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.introspect.AnnotatedField;
import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition;
import com.google.common.base.Function;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
import com.google.inject.ProvisionException;
import com.google.inject.spi.Message;
import com.metamx.common.logger.Logger;
import javax.validation.ConstraintViolation;
import javax.validation.ElementKind;
import javax.validation.Path;
import javax.validation.Validator;
import java.io.IOException;
import java.lang.reflect.Field;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
/**
*/
public class JsonConfigurator
{
private static final Logger log = new Logger(JsonConfigurator.class);
private final ObjectMapper jsonMapper;
private final Validator validator;
@Inject
public JsonConfigurator(
ObjectMapper jsonMapper,
Validator validator
)
{
this.jsonMapper = jsonMapper;
this.validator = validator;
}
public <T> T configurate(Properties props, String propertyPrefix, Class<T> clazz) throws ProvisionException
{
verifyClazzIsConfigurable(clazz);
// Make it end with a period so we only include properties with sub-object thingies.
final String propertyBase = propertyPrefix.endsWith(".") ? propertyPrefix : propertyPrefix + ".";
Map<String, Object> jsonMap = Maps.newHashMap();
for (String prop : props.stringPropertyNames()) {
if (prop.startsWith(propertyBase)) {
final String propValue = props.getProperty(prop);
Object value;
try {
// If it's a String Jackson wants it to be quoted, so check if it's not an object or array and quote.
String modifiedPropValue = propValue;
if (! (modifiedPropValue.startsWith("[") || modifiedPropValue.startsWith("{"))) {
modifiedPropValue = jsonMapper.writeValueAsString(propValue);
}
value = jsonMapper.readValue(modifiedPropValue, Object.class);
}
catch (IOException e) {
log.info(e, "Unable to parse [%s]=[%s] as a json object, using as is.", prop, propValue);
value = propValue;
}
jsonMap.put(prop.substring(propertyBase.length()), value);
}
}
final T config;
try {
config = jsonMapper.convertValue(jsonMap, clazz);
}
catch (IllegalArgumentException e) {
throw new ProvisionException(
String.format("Problem parsing object at prefix[%s]: %s.", propertyPrefix, e.getMessage()), e
);
}
final Set<ConstraintViolation<T>> violations = validator.validate(config);
if (!violations.isEmpty()) {
List<String> messages = Lists.newArrayList();
for (ConstraintViolation<T> violation : violations) {
String path = "";
try {
Class<?> beanClazz = violation.getRootBeanClass();
final Iterator<Path.Node> iter = violation.getPropertyPath().iterator();
while (iter.hasNext()) {
Path.Node next = iter.next();
if (next.getKind() == ElementKind.PROPERTY) {
final String fieldName = next.getName();
final Field theField = beanClazz.getDeclaredField(fieldName);
if (theField.getAnnotation(JacksonInject.class) != null) {
path = String.format(" -- Injected field[%s] not bound!?", fieldName);
break;
}
JsonProperty annotation = theField.getAnnotation(JsonProperty.class);
final boolean noAnnotationValue = annotation == null || Strings.isNullOrEmpty(annotation.value());
final String pathPart = noAnnotationValue ? fieldName : annotation.value();
if (path.isEmpty()) {
path += pathPart;
}
else {
path += "." + pathPart;
}
}
}
}
catch (NoSuchFieldException e) {
throw Throwables.propagate(e);
}
messages.add(String.format("%s - %s", path, violation.getMessage()));
}
throw new ProvisionException(
Iterables.transform(
messages,
new Function<String, Message>()
{
@Override
public Message apply(String input)
{
return new Message(String.format("%s%s", propertyBase, input));
}
}
)
);
}
log.info("Loaded class[%s] from props[%s] as [%s]", clazz, propertyBase, config);
return config;
}
private <T> void verifyClazzIsConfigurable(Class<T> clazz)
{
final List<BeanPropertyDefinition> beanDefs = jsonMapper.getSerializationConfig()
.introspect(jsonMapper.constructType(clazz))
.findProperties();
for (BeanPropertyDefinition beanDef : beanDefs) {
final AnnotatedField field = beanDef.getField();
if (field == null || !field.hasAnnotation(JsonProperty.class)) {
throw new ProvisionException(
String.format(
"JsonConfigurator requires Jackson-annotated Config objects to have field annotations. %s doesn't",
clazz
)
);
}
}
}
}

View File

@ -0,0 +1,41 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.Key;
/**
*/
public class KeyHolder<T>
{
private final Key<? extends T> key;
public KeyHolder(
Key<? extends T> key
)
{
this.key = key;
}
public Key<? extends T> getKey()
{
return key;
}
}

View File

@ -0,0 +1,37 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.ScopeAnnotation;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.Target;
import static java.lang.annotation.RetentionPolicy.RUNTIME;
/**
*/
@Target({ ElementType.TYPE, ElementType.METHOD })
@Retention(RUNTIME)
@ScopeAnnotation
public @interface LazySingleton
{
}

View File

@ -0,0 +1,163 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Module;
import com.google.inject.Provides;
import com.google.inject.TypeLiteral;
import com.google.inject.multibindings.Multibinder;
import com.google.inject.name.Names;
import com.metamx.common.lifecycle.Lifecycle;
import java.lang.annotation.Annotation;
import java.util.Set;
/**
* A Module to add lifecycle management to the injector. {@link DruidGuiceExtensions} must also be included.
*/
public class LifecycleModule implements Module
{
private final LifecycleScope scope = new LifecycleScope(Lifecycle.Stage.NORMAL);
private final LifecycleScope lastScope = new LifecycleScope(Lifecycle.Stage.LAST);
/**
* Registers a class to instantiate eagerly. Classes mentioned here will be pulled out of
* the injector with an injector.getInstance() call when the lifecycle is created.
*
* Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper
* scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial
* side-effect even if nothing actually directly depends on the instance.
*
* This mechanism exists to allow the {@link com.metamx.common.lifecycle.Lifecycle} to be the primary entry point from the injector, not to
* auto-register things with the {@link com.metamx.common.lifecycle.Lifecycle}. It is also possible to just bind things eagerly with Guice,
* it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules
* is less error-prone.
*
* @param clazz, the class to instantiate
* @return this, for chaining.
*/
public static void register(Binder binder, Class<?> clazz)
{
registerKey(binder, Key.get(clazz));
}
/**
* Registers a class/annotation combination to instantiate eagerly. Classes mentioned here will be pulled out of
* the injector with an injector.getInstance() call when the lifecycle is created.
*
* Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper
* scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial
* side-effect even if nothing actually directly depends on the instance.
*
* This mechanism exists to allow the {@link com.metamx.common.lifecycle.Lifecycle} to be the primary entry point from the injector, not to
* auto-register things with the {@link com.metamx.common.lifecycle.Lifecycle}. It is also possible to just bind things eagerly with Guice,
* it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules
* is less error-prone.
*
* @param clazz, the class to instantiate
* @param annotation The annotation instance to register with Guice, usually a Named annotation
* @return this, for chaining.
*/
public static void register(Binder binder, Class<?> clazz, Annotation annotation)
{
registerKey(binder, Key.get(clazz, annotation));
}
/**
* Registers a class/annotation combination to instantiate eagerly. Classes mentioned here will be pulled out of
* the injector with an injector.getInstance() call when the lifecycle is created.
*
* Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper
* scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial
* side-effect even if nothing actually directly depends on the instance.
*
* This mechanism exists to allow the {@link com.metamx.common.lifecycle.Lifecycle} to be the primary entry point from the injector, not to
* auto-register things with the {@link com.metamx.common.lifecycle.Lifecycle}. It is also possible to just bind things eagerly with Guice,
* it is not clear which is actually the best approach. This is more explicit, but eager bindings inside of modules
* is less error-prone.
*
* @param clazz, the class to instantiate
* @param annotation The annotation class to register with Guice
* @return this, for chaining
*/
public static void register(Binder binder, Class<?> clazz, Class<? extends Annotation> annotation)
{
registerKey(binder, Key.get(clazz, annotation));
}
/**
* Registers a key to instantiate eagerly. {@link com.google.inject.Key}s mentioned here will be pulled out of
* the injector with an injector.getInstance() call when the lifecycle is created.
*
* Eagerly loaded classes will *not* be automatically added to the Lifecycle unless they are bound to the proper
* scope. That is, they are generally eagerly loaded because the loading operation will produce some beneficial
* side-effect even if nothing actually directly depends on the instance.
*
* This mechanism exists to allow the {@link com.metamx.common.lifecycle.Lifecycle} to be the primary entry point
* from the injector, not to auto-register things with the {@link com.metamx.common.lifecycle.Lifecycle}. It is
* also possible to just bind things eagerly with Guice, it is not clear which is actually the best approach.
* This is more explicit, but eager bindings inside of modules is less error-prone.
*
* @param key The key to use in finding the DruidNode instance
*/
public static void registerKey(Binder binder, Key<?> key)
{
getEagerBinder(binder).addBinding().toInstance(new KeyHolder<Object>(key));
}
private static Multibinder<KeyHolder> getEagerBinder(Binder binder)
{
return Multibinder.newSetBinder(binder, KeyHolder.class, Names.named("lifecycle"));
}
@Override
public void configure(Binder binder)
{
getEagerBinder(binder); // Load up the eager binder so that it will inject the empty set at a minimum.
binder.bindScope(ManageLifecycle.class, scope);
binder.bindScope(ManageLifecycleLast.class, lastScope);
}
@Provides @LazySingleton
public Lifecycle getLifecycle(final Injector injector)
{
final Key<Set<KeyHolder>> keyHolderKey = Key.get(new TypeLiteral<Set<KeyHolder>>(){}, Names.named("lifecycle"));
final Set<KeyHolder> eagerClasses = injector.getInstance(keyHolderKey);
Lifecycle lifecycle = new Lifecycle(){
@Override
public void start() throws Exception
{
for (KeyHolder<?> holder : eagerClasses) {
injector.getInstance(holder.getKey()); // Pull the key so as to "eagerly" load up the class.
}
super.start();
}
};
scope.setLifecycle(lifecycle);
lastScope.setLifecycle(lifecycle);
return lifecycle;
}
}

View File

@ -0,0 +1,92 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.common.collect.Lists;
import com.google.inject.Key;
import com.google.inject.Provider;
import com.google.inject.Scope;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import java.util.List;
/**
* A scope that adds objects to the Lifecycle. This is by definition also a lazy singleton scope.
*/
public class LifecycleScope implements Scope
{
private static final Logger log = new Logger(LifecycleScope.class);
private final Lifecycle.Stage stage;
private Lifecycle lifecycle;
private List<Object> instances = Lists.newLinkedList();
public LifecycleScope(Lifecycle.Stage stage)
{
this.stage = stage;
}
public void setLifecycle(Lifecycle lifecycle)
{
this.lifecycle = lifecycle;
synchronized (instances) {
for (Object instance : instances) {
lifecycle.addManagedInstance(instance);
}
}
}
@Override
public <T> Provider<T> scope(final Key<T> key, final Provider<T> unscoped)
{
return new Provider<T>()
{
private volatile T value = null;
@Override
public synchronized T get()
{
if (value == null) {
final T retVal = unscoped.get();
synchronized (instances) {
if (lifecycle == null) {
instances.add(retVal);
}
else {
try {
lifecycle.addMaybeStartManagedInstance(retVal, stage);
}
catch (Exception e) {
log.warn(e, "Caught exception when trying to create a[%s]", key);
return null;
}
}
}
value = retVal;
}
return value;
}
};
}
}

View File

@ -0,0 +1,40 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.ScopeAnnotation;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.Target;
import static java.lang.annotation.RetentionPolicy.RUNTIME;
/**
* Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle}
*
* This Scope gets defined by {@link io.druid.guice.LifecycleModule}
*/
@Target({ ElementType.TYPE, ElementType.METHOD })
@Retention(RUNTIME)
@ScopeAnnotation
public @interface ManageLifecycle
{
}

View File

@ -0,0 +1,40 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.ScopeAnnotation;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.Target;
import static java.lang.annotation.RetentionPolicy.RUNTIME;
/**
* Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} and set to be on Stage.LAST
*
* This Scope gets defined by {@link io.druid.guice.LifecycleModule}
*/
@Target({ ElementType.TYPE, ElementType.METHOD })
@Retention(RUNTIME)
@ScopeAnnotation
public @interface ManageLifecycleLast
{
}

View File

@ -0,0 +1,184 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.inject.Binder;
import com.google.inject.Inject;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Provider;
import com.google.inject.ProvisionException;
import com.google.inject.TypeLiteral;
import com.google.inject.binder.ScopedBindingBuilder;
import com.google.inject.multibindings.MapBinder;
import com.google.inject.util.Types;
import java.lang.reflect.ParameterizedType;
import java.util.Map;
import java.util.Properties;
/**
* Provides the ability to create "polymorphic" bindings. Where the polymorphism is actually just making a decision
* based on a value in a Properties.
*
* The workflow is that you first create a choice by calling createChoice(). Then you create options using the binder
* returned by the optionBinder() method. Multiple different modules can call optionBinder and all options will be
* reflected at injection time as long as equivalent interface Key objects are passed into the various methods.
*/
public class PolyBind
{
/**
* Sets up a "choice" for the injector to resolve at injection time.
*
* @param binder the binder for the injector that is being configured
* @param property the property that will be checked to determine the implementation choice
* @param interfaceKey the interface that will be injected using this choice
* @param defaultKey the default instance to be injected if the property doesn't match a choice. Can be null
* @param <T> interface type
* @return A ScopedBindingBuilder so that scopes can be added to the binding, if required.
*/
public static <T> ScopedBindingBuilder createChoice(
Binder binder,
String property,
Key<T> interfaceKey,
Key<? extends T> defaultKey
)
{
return createChoiceWithDefault(binder, property, interfaceKey, defaultKey, null);
}
/**
* Sets up a "choice" for the injector to resolve at injection time.
*
* @param binder the binder for the injector that is being configured
* @param property the property that will be checked to determine the implementation choice
* @param interfaceKey the interface that will be injected using this choice
* @param defaultKey the default instance to be injected if the property doesn't match a choice. Can be null
* @param defaultPropertyValue the default property value to use if the property is not set.
* @param <T> interface type
* @return A ScopedBindingBuilder so that scopes can be added to the binding, if required.
*/
public static <T> ScopedBindingBuilder createChoiceWithDefault(
Binder binder,
String property,
Key<T> interfaceKey,
Key<? extends T> defaultKey,
String defaultPropertyValue
)
{
return binder.bind(interfaceKey).toProvider(new ConfiggedProvider<T>(interfaceKey, property, defaultKey, defaultPropertyValue));
}
/**
* Binds an option for a specific choice. The choice must already be registered on the injector for this to work.
*
* @param binder the binder for the injector that is being configured
* @param interfaceKey the interface that will have an option added to it. This must equal the
* Key provided to createChoice
* @param <T> interface type
* @return A MapBinder that can be used to create the actual option bindings.
*/
public static <T> MapBinder<String, T> optionBinder(Binder binder, Key<T> interfaceKey)
{
final TypeLiteral<T> interfaceType = interfaceKey.getTypeLiteral();
if (interfaceKey.getAnnotation() != null) {
return MapBinder.newMapBinder(
binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotation()
);
}
else if (interfaceKey.getAnnotationType() != null) {
return MapBinder.newMapBinder(
binder, TypeLiteral.get(String.class), interfaceType, interfaceKey.getAnnotationType()
);
}
else {
return MapBinder.newMapBinder(binder, TypeLiteral.get(String.class), interfaceType);
}
}
static class ConfiggedProvider<T> implements Provider<T>
{
private final Key<T> key;
private final String property;
private final Key<? extends T> defaultKey;
private final String defaultPropertyValue;
private Injector injector;
private Properties props;
ConfiggedProvider(
Key<T> key,
String property,
Key<? extends T> defaultKey,
String defaultPropertyValue
)
{
this.key = key;
this.property = property;
this.defaultKey = defaultKey;
this.defaultPropertyValue = defaultPropertyValue;
}
@Inject
void configure(Injector injector, Properties props)
{
this.injector = injector;
this.props = props;
}
@Override
@SuppressWarnings("unchecked")
public T get()
{
final ParameterizedType mapType = Types.mapOf(
String.class, Types.newParameterizedType(Provider.class, key.getTypeLiteral().getType())
);
final Map<String, Provider<T>> implsMap;
if (key.getAnnotation() != null) {
implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType, key.getAnnotation()));
}
else if (key.getAnnotationType() != null) {
implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType, key.getAnnotation()));
}
else {
implsMap = (Map<String, Provider<T>>) injector.getInstance(Key.get(mapType));
}
String implName = props.getProperty(property);
if (implName == null) {
implName = defaultPropertyValue;
}
final Provider<T> provider = implsMap.get(implName);
if (provider == null) {
if (defaultKey == null) {
throw new ProvisionException(
String.format("Unknown provider[%s] of %s, known options[%s]", implName, key, implsMap.keySet())
);
}
return injector.getInstance(defaultKey);
}
return provider.get();
}
}
}

View File

@ -0,0 +1,54 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice;
import com.google.common.base.Supplier;
import com.google.inject.Inject;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Provider;
/**
*/
public class SupplierProvider<T> implements Provider<T>
{
private final Key<Supplier<T>> supplierKey;
private Provider<Supplier<T>> supplierProvider;
public SupplierProvider(
Key<Supplier<T>> supplierKey
)
{
this.supplierKey = supplierKey;
}
@Inject
public void configure(Injector injector)
{
this.supplierProvider = injector.getProvider(supplierKey);
}
@Override
public T get()
{
return supplierProvider.get().get();
}
}

View File

@ -0,0 +1,36 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice.annotations;
import com.google.inject.BindingAnnotation;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
*/
@BindingAnnotation
@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
@Retention(RetentionPolicy.RUNTIME)
public @interface Global
{
}

View File

@ -0,0 +1,36 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice.annotations;
import com.google.inject.BindingAnnotation;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
*/
@BindingAnnotation
@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
@Retention(RetentionPolicy.RUNTIME)
public @interface JSR311Resource
{
}

View File

@ -0,0 +1,36 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice.annotations;
import com.google.inject.BindingAnnotation;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
*/
@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
@Retention(RetentionPolicy.RUNTIME)
@BindingAnnotation
public @interface Json
{
}

View File

@ -0,0 +1,36 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice.annotations;
import com.google.inject.BindingAnnotation;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
*/
@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
@Retention(RetentionPolicy.RUNTIME)
@BindingAnnotation
public @interface Self
{
}

View File

@ -0,0 +1,36 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.guice.annotations;
import com.google.inject.BindingAnnotation;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
*/
@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
@Retention(RetentionPolicy.RUNTIME)
@BindingAnnotation
public @interface Smile
{
}

View File

@ -0,0 +1,31 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.initialization;
import com.fasterxml.jackson.databind.Module;
import java.util.List;
/**
*/
public interface DruidModule extends com.google.inject.Module
{
public List<? extends Module> getJacksonModules();
}

View File

@ -0,0 +1,46 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.jackson;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
/**
*/
public class CommaListJoinDeserializer extends StdScalarDeserializer<List<String>>
{
protected CommaListJoinDeserializer()
{
super(List.class);
}
@Override
public List<String> deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
throws IOException, JsonProcessingException
{
return Arrays.asList(jsonParser.getText().split(","));
}
}

View File

@ -0,0 +1,48 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.jackson;
import com.fasterxml.jackson.core.JsonGenerationException;
import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer;
import com.google.common.base.Joiner;
import java.io.IOException;
import java.util.List;
/**
*/
public class CommaListJoinSerializer extends StdScalarSerializer<List<String>>
{
private static final Joiner joiner = Joiner.on(",");
protected CommaListJoinSerializer()
{
super(List.class, true);
}
@Override
public void serialize(List<String> value, JsonGenerator jgen, SerializerProvider provider)
throws IOException, JsonGenerationException
{
jgen.writeString(joiner.join(value));
}
}

View File

@ -0,0 +1,107 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.query;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Interval;
/**
*/
public class SegmentDescriptor
{
private final Interval interval;
private final String version;
private final int partitionNumber;
@JsonCreator
public SegmentDescriptor(
@JsonProperty("itvl") Interval interval,
@JsonProperty("ver") String version,
@JsonProperty("part") int partitionNumber
)
{
this.interval = interval;
this.version = version;
this.partitionNumber = partitionNumber;
}
@JsonProperty("itvl")
public Interval getInterval()
{
return interval;
}
@JsonProperty("ver")
public String getVersion()
{
return version;
}
@JsonProperty("part")
public int getPartitionNumber()
{
return partitionNumber;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SegmentDescriptor that = (SegmentDescriptor) o;
if (partitionNumber != that.partitionNumber) {
return false;
}
if (interval != null ? !interval.equals(that.interval) : that.interval != null) {
return false;
}
if (version != null ? !version.equals(that.version) : that.version != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = interval != null ? interval.hashCode() : 0;
result = 31 * result + (version != null ? version.hashCode() : 0);
result = 31 * result + partitionNumber;
return result;
}
@Override
public String toString()
{
return "SegmentDescriptor{" +
"interval=" + interval +
", version='" + version + '\'' +
", partitionNumber=" + partitionNumber +
'}';
}
}

View File

@ -0,0 +1,48 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment;
import com.google.common.io.Files;
import com.google.common.primitives.Ints;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
/**
*/
public class SegmentUtils
{
public static int getVersionFromDir(File inDir) throws IOException
{
File versionFile = new File(inDir, "version.bin");
if (versionFile.exists()) {
return Ints.fromByteArray(Files.toByteArray(versionFile));
}
final File indexFile = new File(inDir, "index.drd");
int version;
try (InputStream in = new FileInputStream(indexFile)) {
version = in.read();
}
return version;
}
}

View File

@ -0,0 +1,28 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
import io.druid.timeline.DataSegment;
public interface DataSegmentArchiver
{
public DataSegment archive(DataSegment segment) throws SegmentLoadingException;
public DataSegment restore(DataSegment segment) throws SegmentLoadingException;
}

View File

@ -0,0 +1,46 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
import io.druid.timeline.DataSegment;
import java.util.Set;
/**
* A DataSegmentFinder is responsible for finding Druid segments underneath a specified directory and optionally updates
* all descriptor.json files on deep storage with correct loadSpec.
*/
public interface DataSegmentFinder
{
/**
* This method should first recursively look for descriptor.json underneath workingDirPath and then verify that
* index.zip exists in the same folder. If not, it should throw SegmentLoadingException to let the caller know that
* descriptor.json exists while index.zip doesn't. If a segment is found and updateDescriptor is set, then this method
* should update the loadSpec in descriptor.json to reflect the location from where it was found. After the search,
* this method should return the set of segments that were found.
*
* @param workingDirPath the String representation of the working directory path
* @param updateDescriptor if true, update loadSpec in descriptor.json if loadSpec's location is different from where
* desciptor.json was found
*
* @return a set of segments that were found underneath workingDirPath
*/
Set<DataSegment> findSegments(String workingDirPath, boolean updateDescriptor) throws SegmentLoadingException;
}

View File

@ -0,0 +1,29 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
import io.druid.timeline.DataSegment;
/**
*/
public interface DataSegmentKiller
{
public void kill(DataSegment segments) throws SegmentLoadingException;
}

View File

@ -0,0 +1,29 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
import io.druid.timeline.DataSegment;
import java.util.Map;
public interface DataSegmentMover
{
public DataSegment move(DataSegment segment, Map<String, Object> targetLoadSpec) throws SegmentLoadingException;
}

View File

@ -0,0 +1,40 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
import io.druid.timeline.DataSegment;
import java.io.File;
/**
* A DataSegmentPuller is responsible for pulling data for a particular segment into a particular directory
*/
public interface DataSegmentPuller
{
/**
* Pull down segment files for the given DataSegment and put them in the given directory.
*
* @param segment The segment to pull down files for
* @param dir The directory to store the files in
*
* @throws SegmentLoadingException if there are any errors
*/
public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException;
}

View File

@ -0,0 +1,31 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
import io.druid.timeline.DataSegment;
import java.io.File;
import java.io.IOException;
public interface DataSegmentPusher
{
public String getPathForHadoop(String dataSource);
public DataSegment push(File file, DataSegment segment) throws IOException;
}

View File

@ -0,0 +1,67 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
import com.google.common.base.Joiner;
import io.druid.timeline.DataSegment;
import org.joda.time.format.ISODateTimeFormat;
/**
*/
public class DataSegmentPusherUtil
{
private static final Joiner JOINER = Joiner.on("/").skipNulls();
// Note: storage directory structure format = .../dataSource/interval/version/partitionNumber/
// If above format is ever changed, make sure to change it appropriately in other places
// e.g. HDFSDataSegmentKiller uses this information to clean the version, interval and dataSource directories
// on segment deletion if segment being deleted was the only segment
public static String getStorageDir(DataSegment segment)
{
return JOINER.join(
segment.getDataSource(),
String.format(
"%s_%s",
segment.getInterval().getStart(),
segment.getInterval().getEnd()
),
segment.getVersion(),
segment.getShardSpec().getPartitionNum()
);
}
/**
* Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in
* path names. So we format paths differently for HDFS.
*/
public static String getHdfsStorageDir(DataSegment segment)
{
return JOINER.join(
segment.getDataSource(),
String.format(
"%s_%s",
segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()),
segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime())
),
segment.getVersion().replaceAll(":", "_"),
segment.getShardSpec().getPartitionNum()
);
}
}

View File

@ -0,0 +1,49 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import java.io.File;
/**
* A means of pulling segment files into a destination directory
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
public interface LoadSpec
{
/**
* Method should put the segment files in the directory passed
* @param destDir The destination directory
* @return The byte count of data put in the destination directory
*/
public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException;
// Hold interesting data about the results of the segment load
public static class LoadSpecResult{
private final long size;
public LoadSpecResult(long size){
this.size = size;
}
public long getSize(){
return this.size;
}
}
}

View File

@ -0,0 +1,42 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
/**
*/
public class SegmentLoadingException extends Exception
{
public SegmentLoadingException(
String formatString,
Object... objs
)
{
super(String.format(formatString, objs));
}
public SegmentLoadingException(
Throwable cause,
String formatString,
Object... objs
)
{
super(String.format(formatString, objs), cause);
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.segment.loading;
import com.google.common.base.Predicate;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
/**
* A URIDataPuller has handlings for URI based data
*/
public interface URIDataPuller
{
/**
* Create a new InputStream based on the URI
*
* @param uri The URI to open an Input Stream to
*
* @return A new InputStream which streams the URI in question
*
* @throws IOException
*/
public InputStream getInputStream(URI uri) throws IOException;
/**
* Returns an abstract "version" for the URI. The exact meaning of the version is left up to the implementation.
*
* @param uri The URI to check
*
* @return A "version" as interpreted by the URIDataPuller implementation
*
* @throws IOException on error
*/
public String getVersion(URI uri) throws IOException;
/**
* Evaluates a Throwable to see if it is recoverable. This is expected to be used in conjunction with the other methods
* to determine if anything thrown from the method should be retried.
*
* @return Predicate function indicating if the Throwable is recoverable
*/
public Predicate<Throwable> shouldRetryPredicate();
}

View File

@ -0,0 +1,46 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.tasklogs;
import com.google.common.base.Optional;
import com.google.common.io.ByteSource;
import com.google.common.io.InputSupplier;
import com.metamx.common.logger.Logger;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
public class NoopTaskLogs implements TaskLogs
{
private final Logger log = new Logger(TaskLogs.class);
@Override
public Optional<ByteSource> streamTaskLog(String taskid, long offset) throws IOException
{
return Optional.absent();
}
@Override
public void pushTaskLog(String taskid, File logFile) throws IOException
{
log.info("Not pushing logs for task: %s", taskid);
}
}

View File

@ -0,0 +1,31 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.tasklogs;
import java.io.File;
import java.io.IOException;
/**
* Something that knows how to persist local task logs to some form of long-term storage.
*/
public interface TaskLogPusher
{
public void pushTaskLog(String taskid, File logFile) throws IOException;
}

View File

@ -0,0 +1,43 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.tasklogs;
import com.google.common.base.Optional;
import com.google.common.io.ByteSource;
import com.google.common.io.InputSupplier;
import java.io.IOException;
import java.io.InputStream;
/**
* Something that knows how to stream logs for tasks.
*/
public interface TaskLogStreamer
{
/**
* Stream log for a task.
*
* @param offset If zero, stream the entire log. If positive, attempt to read from this position onwards. If
* negative, attempt to read this many bytes from the end of the file (like <tt>tail -n</tt>).
*
* @return input supplier for this log, if available from this provider
*/
public Optional<ByteSource> streamTaskLog(String taskid, long offset) throws IOException;
}

View File

@ -0,0 +1,24 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.tasklogs;
public interface TaskLogs extends TaskLogStreamer, TaskLogPusher
{
}

View File

@ -0,0 +1,418 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.timeline;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Interner;
import com.google.common.collect.Interners;
import com.google.common.collect.Iterables;
import com.metamx.common.Granularity;
import io.druid.jackson.CommaListJoinDeserializer;
import io.druid.jackson.CommaListJoinSerializer;
import io.druid.query.SegmentDescriptor;
import io.druid.timeline.partition.NoneShardSpec;
import io.druid.timeline.partition.ShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
/**
*/
public class DataSegment implements Comparable<DataSegment>
{
public static String delimiter = "_";
private final Integer binaryVersion;
private static final Interner<String> interner = Interners.newWeakInterner();
private static final Function<String, String> internFun = new Function<String, String>()
{
@Override
public String apply(String input)
{
return interner.intern(input);
}
};
public static String makeDataSegmentIdentifier(
String dataSource,
DateTime start,
DateTime end,
String version,
ShardSpec shardSpec
)
{
StringBuilder sb = new StringBuilder();
sb.append(dataSource).append(delimiter)
.append(start).append(delimiter)
.append(end).append(delimiter)
.append(version);
if (shardSpec.getPartitionNum() != 0) {
sb.append(delimiter).append(shardSpec.getPartitionNum());
}
return sb.toString();
}
private final String dataSource;
private final Interval interval;
private final String version;
private final Map<String, Object> loadSpec;
private final List<String> dimensions;
private final List<String> metrics;
private final ShardSpec shardSpec;
private final long size;
private final String identifier;
@JsonCreator
public DataSegment(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval,
@JsonProperty("version") String version,
// use `Map` *NOT* `LoadSpec` because we want to do lazy materialization to prevent dependency pollution
@JsonProperty("loadSpec") Map<String, Object> loadSpec,
@JsonProperty("dimensions") @JsonDeserialize(using = CommaListJoinDeserializer.class) List<String> dimensions,
@JsonProperty("metrics") @JsonDeserialize(using = CommaListJoinDeserializer.class) List<String> metrics,
@JsonProperty("shardSpec") ShardSpec shardSpec,
@JsonProperty("binaryVersion") Integer binaryVersion,
@JsonProperty("size") long size
)
{
final Predicate<String> nonEmpty = new Predicate<String>()
{
@Override
public boolean apply(String input)
{
return input != null && !input.isEmpty();
}
};
// dataSource, dimensions & metrics are stored as canonical string values to decrease memory required for storing large numbers of segments.
this.dataSource = interner.intern(dataSource);
this.interval = interval;
this.loadSpec = loadSpec;
this.version = version;
this.dimensions = dimensions == null
? ImmutableList.<String>of()
: ImmutableList.copyOf(Iterables.transform(Iterables.filter(dimensions, nonEmpty), internFun));
this.metrics = metrics == null
? ImmutableList.<String>of()
: ImmutableList.copyOf(Iterables.transform(Iterables.filter(metrics, nonEmpty), internFun));
this.shardSpec = (shardSpec == null) ? new NoneShardSpec() : shardSpec;
this.binaryVersion = binaryVersion;
this.size = size;
this.identifier = makeDataSegmentIdentifier(
this.dataSource,
this.interval.getStart(),
this.interval.getEnd(),
this.version,
this.shardSpec
);
}
/**
* Get dataSource
*
* @return the dataSource
*/
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
@JsonProperty
public Map<String, Object> getLoadSpec()
{
return loadSpec;
}
@JsonProperty
public String getVersion()
{
return version;
}
@JsonProperty
@JsonSerialize(using = CommaListJoinSerializer.class)
public List<String> getDimensions()
{
return dimensions;
}
@JsonProperty
@JsonSerialize(using = CommaListJoinSerializer.class)
public List<String> getMetrics()
{
return metrics;
}
@JsonProperty
public ShardSpec getShardSpec()
{
return shardSpec;
}
@JsonProperty
public Integer getBinaryVersion()
{
return binaryVersion;
}
@JsonProperty
public long getSize()
{
return size;
}
@JsonProperty
public String getIdentifier()
{
return identifier;
}
public SegmentDescriptor toDescriptor()
{
return new SegmentDescriptor(interval, version, shardSpec.getPartitionNum());
}
public DataSegment withLoadSpec(Map<String, Object> loadSpec)
{
return builder(this).loadSpec(loadSpec).build();
}
public DataSegment withDimensions(List<String> dimensions)
{
return builder(this).dimensions(dimensions).build();
}
public DataSegment withSize(long size)
{
return builder(this).size(size).build();
}
public DataSegment withVersion(String version)
{
return builder(this).version(version).build();
}
public DataSegment withBinaryVersion(int binaryVersion)
{
return builder(this).binaryVersion(binaryVersion).build();
}
@Override
public int compareTo(DataSegment dataSegment)
{
return getIdentifier().compareTo(dataSegment.getIdentifier());
}
@Override
public boolean equals(Object o)
{
if (o instanceof DataSegment) {
return getIdentifier().equals(((DataSegment) o).getIdentifier());
}
return false;
}
@Override
public int hashCode()
{
return getIdentifier().hashCode();
}
@Override
public String toString()
{
return "DataSegment{" +
"size=" + size +
", shardSpec=" + shardSpec +
", metrics=" + metrics +
", dimensions=" + dimensions +
", version='" + version + '\'' +
", loadSpec=" + loadSpec +
", interval=" + interval +
", dataSource='" + dataSource + '\'' +
", binaryVersion='" + binaryVersion + '\'' +
'}';
}
public static Comparator<DataSegment> bucketMonthComparator()
{
return new Comparator<DataSegment>()
{
@Override
public int compare(DataSegment lhs, DataSegment rhs)
{
int retVal;
DateTime lhsMonth = Granularity.MONTH.truncate(lhs.getInterval().getStart());
DateTime rhsMonth = Granularity.MONTH.truncate(rhs.getInterval().getStart());
retVal = lhsMonth.compareTo(rhsMonth);
if (retVal != 0) {
return retVal;
}
return lhs.compareTo(rhs);
}
};
}
public static Builder builder()
{
return new Builder();
}
public static Builder builder(DataSegment segment)
{
return new Builder(segment);
}
public static class Builder
{
private String dataSource;
private Interval interval;
private String version;
private Map<String, Object> loadSpec;
private List<String> dimensions;
private List<String> metrics;
private ShardSpec shardSpec;
private Integer binaryVersion;
private long size;
public Builder()
{
this.loadSpec = ImmutableMap.of();
this.dimensions = ImmutableList.of();
this.metrics = ImmutableList.of();
this.shardSpec = new NoneShardSpec();
this.size = -1;
}
public Builder(DataSegment segment)
{
this.dataSource = segment.getDataSource();
this.interval = segment.getInterval();
this.version = segment.getVersion();
this.loadSpec = segment.getLoadSpec();
this.dimensions = segment.getDimensions();
this.metrics = segment.getMetrics();
this.shardSpec = segment.getShardSpec();
this.binaryVersion = segment.getBinaryVersion();
this.size = segment.getSize();
}
public Builder dataSource(String dataSource)
{
this.dataSource = dataSource;
return this;
}
public Builder interval(Interval interval)
{
this.interval = interval;
return this;
}
public Builder version(String version)
{
this.version = version;
return this;
}
public Builder loadSpec(Map<String, Object> loadSpec)
{
this.loadSpec = loadSpec;
return this;
}
public Builder dimensions(List<String> dimensions)
{
this.dimensions = dimensions;
return this;
}
public Builder metrics(List<String> metrics)
{
this.metrics = metrics;
return this;
}
public Builder shardSpec(ShardSpec shardSpec)
{
this.shardSpec = shardSpec;
return this;
}
public Builder binaryVersion(Integer binaryVersion)
{
this.binaryVersion = binaryVersion;
return this;
}
public Builder size(long size)
{
this.size = size;
return this;
}
public DataSegment build()
{
// Check stuff that goes into the identifier, at least.
Preconditions.checkNotNull(dataSource, "dataSource");
Preconditions.checkNotNull(interval, "interval");
Preconditions.checkNotNull(version, "version");
Preconditions.checkNotNull(shardSpec, "shardSpec");
return new DataSegment(
dataSource,
interval,
version,
loadSpec,
dimensions,
metrics,
shardSpec,
binaryVersion,
size
);
}
}
}

View File

@ -0,0 +1,84 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.timeline.partition;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.metamx.common.ISE;
import io.druid.data.input.InputRow;
import java.util.Collections;
import java.util.List;
import java.util.Set;
/**
*/
public class NoneShardSpec implements ShardSpec
{
@Override
public <T> PartitionChunk<T> createChunk(T obj)
{
return new SingleElementPartitionChunk<T>(obj);
}
@Override
public boolean isInChunk(long timestamp, InputRow inputRow)
{
return true;
}
@Override
public int getPartitionNum()
{
return 0;
}
@Override
public ShardSpecLookup getLookup(final List<ShardSpec> shardSpecs)
{
return new ShardSpecLookup()
{
@Override
public ShardSpec getShardSpec(long timestamp, InputRow row)
{
return shardSpecs.get(0);
}
};
}
@Override
public boolean equals(Object obj)
{
return obj instanceof NoneShardSpec;
}
@Override
public int hashCode()
{
return 0;
}
@Override
public String toString()
{
return "NoneShardSpec";
}
}

View File

@ -0,0 +1,72 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.timeline.partition;
/**
* A PartitionChunk represents a chunk of a partitioned(sharded) space. It has knowledge of whether it is
* the start of the domain of partitions, the end of the domain, if it abuts another partition and where it stands
* inside of a sorted collection of partitions.
*
* The ordering of PartitionChunks is based entirely upon the partition boundaries defined inside the concrete
* PartitionChunk class. That is, the payload (the object returned by getObject()) should *not* be involved in
* comparisons between PartitionChunk objects.
*/
public interface PartitionChunk<T> extends Comparable<PartitionChunk<T>>
{
/**
* Returns the payload, generally an object that can be used to perform some action against the shard.
*
* @return the payload
*/
public T getObject();
/**
* Determines if this PartitionChunk abuts another PartitionChunk. A sequence of abutting PartitionChunks should
* start with an object where isStart() == true and eventually end with an object where isEnd() == true.
*
* @param chunk input chunk
* @return true if this chunk abuts the input chunk
*/
public boolean abuts(PartitionChunk<T> chunk);
/**
* Returns true if this chunk is the beginning of the partition. Most commonly, that means it represents the range
* [-infinity, X) for some concrete X.
*
* @return true if the chunk is the beginning of the partition
*/
public boolean isStart();
/**
* Returns true if this chunk is the end of the partition. Most commonly, that means it represents the range
* [X, infinity] for some concrete X.
*
* @return true if the chunk is the beginning of the partition
*/
public boolean isEnd();
/**
* Returns the partition chunk number of this PartitionChunk. I.e. if there are 4 partitions in total and this
* is the 3rd partition, it will return 2
*
* @return the sequential numerical id of this partition chunk
*/
public int getChunkNumber();
}

View File

@ -0,0 +1,44 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.timeline.partition;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import io.druid.data.input.InputRow;
import java.util.List;
/**
* A Marker interface that exists to combine ShardSpec objects together for Jackson
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes({
@JsonSubTypes.Type(name = "none", value = NoneShardSpec.class),
})
public interface ShardSpec
{
public <T> PartitionChunk<T> createChunk(T obj);
public boolean isInChunk(long timestamp, InputRow inputRow);
public int getPartitionNum();
public ShardSpecLookup getLookup(List<ShardSpec> shardSpecs);
}

View File

@ -0,0 +1,27 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.timeline.partition;
import io.druid.data.input.InputRow;
public interface ShardSpecLookup
{
ShardSpec getShardSpec(long timestamp, InputRow row);
}

View File

@ -0,0 +1,109 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.timeline.partition;
/**
*/
public class SingleElementPartitionChunk<T> implements PartitionChunk<T>
{
private final T element;
public SingleElementPartitionChunk
(
T element
)
{
this.element = element;
}
@Override
public T getObject()
{
return element;
}
@Override
public boolean abuts(PartitionChunk<T> tPartitionChunk)
{
return false;
}
@Override
public boolean isStart()
{
return true;
}
@Override
public boolean isEnd()
{
return true;
}
@Override
public int getChunkNumber()
{
return 0;
}
/**
* The ordering of PartitionChunks is determined entirely by the partition boundaries and has nothing to do
* with the object. Thus, if there are two SingleElementPartitionChunks, they are equal because they both
* represent the full partition space.
*
* SingleElementPartitionChunks are currently defined as less than every other type of PartitionChunk. There
* is no good reason for it, nor is there a bad reason, that's just the way it is. This is subject to change.
*
* @param chunk
* @return
*/
@Override
public int compareTo(PartitionChunk<T> chunk)
{
return chunk instanceof SingleElementPartitionChunk ? 0 : -1;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
return true;
}
@Override
public int hashCode()
{
return element != null ? element.hashCode() : 0;
}
@Override
public String toString()
{
return "SingleElementPartitionChunk{" +
"element=" + element +
'}';
}
}

View File

@ -0,0 +1,81 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.utils;
import com.metamx.common.logger.Logger;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
/**
*/
public class CompressionUtils
{
private static final Logger log = new Logger(CompressionUtils.class);
@Deprecated // Use com.metamx.common.CompressionUtils.zip
public static long zip(File directory, File outputZipFile) throws IOException
{
return com.metamx.common.CompressionUtils.zip(directory, outputZipFile);
}
@Deprecated // Use com.metamx.common.CompressionUtils.zip
public static long zip(File directory, OutputStream out) throws IOException
{
return com.metamx.common.CompressionUtils.zip(directory, out);
}
@Deprecated // Use com.metamx.common.CompressionUtils.unzip
public static void unzip(File pulledFile, File outDir) throws IOException
{
com.metamx.common.CompressionUtils.unzip(pulledFile, outDir);
}
@Deprecated // Use com.metamx.common.CompressionUtils.unzip
public static void unzip(InputStream in, File outDir) throws IOException
{
com.metamx.common.CompressionUtils.unzip(in, outDir);
}
/**
* Uncompress using a gzip uncompress algorithm from the `pulledFile` to the `outDir`.
* Unlike `com.metamx.common.CompressionUtils.gunzip`, this function takes an output *DIRECTORY* and tries to guess the file name.
* It is recommended that the caller use `com.metamx.common.CompressionUtils.gunzip` and specify the output file themselves to ensure names are as expected
*
* @param pulledFile The source file
* @param outDir The destination directory to put the resulting file
*
* @throws IOException on propogated IO exception, IAE if it cannot determine the proper new name for `pulledFile`
*/
@Deprecated // See description for alternative
public static void gunzip(File pulledFile, File outDir) throws IOException
{
final File outFile = new File(outDir, com.metamx.common.CompressionUtils.getGzBaseName(pulledFile.getName()));
com.metamx.common.CompressionUtils.gunzip(pulledFile, outFile);
if (!pulledFile.delete()) {
log.error("Could not delete tmpFile[%s].", pulledFile);
}
}
}

View File

@ -0,0 +1,31 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.utils;
/**
*/
public class Runnables
{
public static Runnable getNoopRunnable(){
return new Runnable(){
public void run(){}
};
}
}

View File

@ -0,0 +1,70 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.MapperFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
import org.joda.time.Interval;
import java.io.IOException;
/**
*/
public class TestObjectMapper extends ObjectMapper
{
public TestObjectMapper()
{
configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
configure(MapperFeature.AUTO_DETECT_GETTERS, false);
configure(MapperFeature.AUTO_DETECT_FIELDS, false);
configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false);
configure(MapperFeature.AUTO_DETECT_SETTERS, false);
configure(SerializationFeature.INDENT_OUTPUT, false);
registerModule(new TestModule());
}
public static class TestModule extends SimpleModule
{
TestModule()
{
addSerializer(Interval.class, ToStringSerializer.instance);
addDeserializer(
Interval.class, new StdDeserializer<Interval>(Interval.class)
{
@Override
public Interval deserialize(
JsonParser jsonParser, DeserializationContext deserializationContext
) throws IOException, JsonProcessingException
{
return new Interval(jsonParser.getText());
}
}
);
}
}
}

View File

@ -0,0 +1,54 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
import com.google.common.collect.ImmutableMap;
public class MapBasedRowTest
{
@Test
public void testGetLongMetricFromString()
{
MapBasedRow row = new MapBasedRow(
new DateTime(),
ImmutableMap.<String,Object>builder()
.put("k0", "-1.2")
.put("k1", "1.23")
.put("k2", "1.8")
.put("k3", "1e5")
.put("k4", "9223372036854775806")
.put("k5", "-9223372036854775807")
.put("k6", "+9223372036854775802")
.build()
);
Assert.assertEquals(-1, row.getLongMetric("k0"));
Assert.assertEquals(1, row.getLongMetric("k1"));
Assert.assertEquals(1, row.getLongMetric("k2"));
Assert.assertEquals(100000, row.getLongMetric("k3"));
Assert.assertEquals(9223372036854775806L, row.getLongMetric("k4"));
Assert.assertEquals(-9223372036854775807L, row.getLongMetric("k5"));
Assert.assertEquals(9223372036854775802L, row.getLongMetric("k6"));
}
}

View File

@ -0,0 +1,67 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.google.common.collect.Lists;
import com.metamx.common.parsers.ParseException;
import org.junit.Test;
import java.util.Arrays;
public class CSVParseSpecTest
{
@Test(expected = IllegalArgumentException.class)
public void testColumnMissing() throws Exception
{
final ParseSpec spec = new CSVParseSpec(
new TimestampSpec(
"timestamp",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b")),
Lists.<String>newArrayList(),
Lists.<SpatialDimensionSchema>newArrayList()
),
",",
Arrays.asList("a")
);
}
@Test(expected = IllegalArgumentException.class)
public void testComma() throws Exception
{
final ParseSpec spec = new CSVParseSpec(
new TimestampSpec(
"timestamp",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b")),
Lists.<String>newArrayList(),
Lists.<SpatialDimensionSchema>newArrayList()
),
",",
Arrays.asList("a")
);
}
}

View File

@ -0,0 +1,118 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import io.druid.TestObjectMapper;
import org.junit.Assert;
import org.junit.Test;
import javax.validation.constraints.Null;
import java.io.IOException;
import java.util.Arrays;
public class DelimitedParseSpecTest
{
private final ObjectMapper jsonMapper = new TestObjectMapper();
@Test
public void testSerde() throws IOException
{
DelimitedParseSpec spec = new DelimitedParseSpec(
new TimestampSpec("abc", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null),
"\u0001",
"\u0002",
Arrays.asList("abc")
);
final DelimitedParseSpec serde = jsonMapper.readValue(
jsonMapper.writeValueAsString(spec),
DelimitedParseSpec.class
);
Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn());
Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
Assert.assertEquals(Arrays.asList("abc"), serde.getColumns());
Assert.assertEquals("\u0001", serde.getDelimiter());
Assert.assertEquals("\u0002", serde.getListDelimiter());
Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames());
}
@Test(expected = IllegalArgumentException.class)
public void testColumnMissing() throws Exception
{
final ParseSpec spec = new DelimitedParseSpec(
new TimestampSpec(
"timestamp",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b")),
Lists.<String>newArrayList(),
Lists.<SpatialDimensionSchema>newArrayList()
),
",",
" ",
Arrays.asList("a")
);
}
@Test(expected = IllegalArgumentException.class)
public void testComma() throws Exception
{
final ParseSpec spec = new DelimitedParseSpec(
new TimestampSpec(
"timestamp",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b")),
Lists.<String>newArrayList(),
Lists.<SpatialDimensionSchema>newArrayList()
),
",",
null,
Arrays.asList("a")
);
}
@Test(expected = NullPointerException.class)
public void testDefaultColumnList(){
final DelimitedParseSpec spec = new DelimitedParseSpec(
new TimestampSpec(
"timestamp",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b")),
Lists.<String>newArrayList(),
Lists.<SpatialDimensionSchema>newArrayList()
),
",",
null,
// pass null columns not allowed
null
);
}
}

View File

@ -0,0 +1,78 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.databind.ObjectMapper;
import junit.framework.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.List;
/**
*/
public class DimensionsSpecSerdeTest
{
private final ObjectMapper mapper = new ObjectMapper();
@Test
public void testDimensionsSpecSerde() throws Exception
{
DimensionsSpec expected = new DimensionsSpec(
Arrays.asList(
new StringDimensionSchema("AAA"),
new StringDimensionSchema("BBB"),
new FloatDimensionSchema("C++"),
new NewSpatialDimensionSchema("DDT", null),
new LongDimensionSchema("EEE"),
new NewSpatialDimensionSchema("DDT2", Arrays.asList("A", "B")),
new NewSpatialDimensionSchema("IMPR", Arrays.asList("S", "P", "Q", "R"))
),
Arrays.asList("FOO", "HAR"),
null
);
String jsonStr = "{\"dimensions\":"
+ "[\"AAA\", \"BBB\","
+ "{\"name\":\"C++\", \"type\":\"float\"},"
+ "{\"name\":\"DDT\", \"type\":\"spatial\"},"
+ "{\"name\":\"EEE\", \"type\":\"long\"},"
+ "{\"name\":\"DDT2\", \"type\": \"spatial\", \"dims\":[\"A\", \"B\"]}],"
+ "\"dimensionExclusions\": [\"FOO\", \"HAR\"],"
+ "\"spatialDimensions\": [{\"dimName\":\"IMPR\", \"dims\":[\"S\",\"P\",\"Q\",\"R\"]}]"
+ "}";
DimensionsSpec actual = mapper.readValue(
mapper.writeValueAsString(
mapper.readValue(jsonStr, DimensionsSpec.class)
),
DimensionsSpec.class
);
List<SpatialDimensionSchema> expectedSpatials = Arrays.asList(
new SpatialDimensionSchema("DDT", null),
new SpatialDimensionSchema("DDT2", Arrays.asList("A","B")),
new SpatialDimensionSchema("IMPR", Arrays.asList("S","P","Q","R"))
);
Assert.assertEquals(expected, actual);
Assert.assertEquals(expectedSpatials, actual.getSpatialDimensions());
}
}

View File

@ -0,0 +1,82 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.metamx.common.Pair;
import junit.framework.Assert;
import org.apache.commons.io.LineIterator;
import org.junit.Test;
import java.io.StringReader;
import java.util.Arrays;
import java.util.List;
public class FileIteratingFirehoseTest
{
private static final List<Pair<String[], ImmutableList<String>>> fixtures = ImmutableList.of(
Pair.of(new String[]{"2000,foo"}, ImmutableList.of("foo")),
Pair.of(new String[]{"2000,foo\n2000,bar\n"}, ImmutableList.of("foo", "bar")),
Pair.of(new String[]{"2000,foo\n2000,bar\n", "2000,baz"}, ImmutableList.of("foo", "bar", "baz")),
Pair.of(new String[]{"2000,foo\n2000,bar\n", "", "2000,baz"}, ImmutableList.of("foo", "bar", "baz")),
Pair.of(new String[]{"2000,foo\n2000,bar\n", "", "2000,baz", ""}, ImmutableList.of("foo", "bar", "baz")),
Pair.of(new String[]{""}, ImmutableList.<String>of()),
Pair.of(new String[]{}, ImmutableList.<String>of())
);
@Test
public void testFirehose() throws Exception
{
for (Pair<String[], ImmutableList<String>> fixture : fixtures) {
final List<LineIterator> lineIterators = Lists.transform(
Arrays.asList(fixture.lhs),
new Function<String, LineIterator>()
{
@Override
public LineIterator apply(String s)
{
return new LineIterator(new StringReader(s));
}
}
);
final StringInputRowParser parser = new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("ts", "auto", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null),
",",
ImmutableList.of("ts", "x")
)
);
final FileIteratingFirehose firehose = new FileIteratingFirehose(lineIterators.iterator(), parser);
final List<String> results = Lists.newArrayList();
while (firehose.hasMore()) {
results.add(Joiner.on("|").join(firehose.nextRow().getDimension("x")));
}
Assert.assertEquals(fixture.rhs, results);
}
}
}

View File

@ -0,0 +1,231 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import io.druid.TestObjectMapper;
import io.druid.data.input.ByteBufferInputRowParser;
import io.druid.data.input.InputRow;
import junit.framework.Assert;
import org.joda.time.DateTime;
import org.junit.Test;
import java.nio.ByteBuffer;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.List;
public class InputRowParserSerdeTest
{
private final ObjectMapper jsonMapper = new TestObjectMapper();
@Test
public void testStringInputRowParserSerde() throws Exception
{
final StringInputRowParser parser = new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar")), null, null),
null,
null
)
);
final ByteBufferInputRowParser parser2 = jsonMapper.readValue(
jsonMapper.writeValueAsBytes(parser),
ByteBufferInputRowParser.class
);
final InputRow parsed = parser2.parse(
ByteBuffer.wrap(
"{\"foo\":\"x\",\"bar\":\"y\",\"qux\":\"z\",\"timestamp\":\"2000\"}".getBytes(Charsets.UTF_8)
)
);
Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions());
Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo"));
Assert.assertEquals(ImmutableList.of("y"), parsed.getDimension("bar"));
Assert.assertEquals(new DateTime("2000").getMillis(), parsed.getTimestampFromEpoch());
}
@Test
public void testStringInputRowParserSerdeMultiCharset() throws Exception
{
Charset[] testCharsets = {
Charsets.US_ASCII, Charsets.ISO_8859_1, Charsets.UTF_8,
Charsets.UTF_16BE, Charsets.UTF_16LE, Charsets.UTF_16
};
for (Charset testCharset : testCharsets) {
InputRow parsed = testCharsetParseHelper(testCharset);
Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions());
Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo"));
Assert.assertEquals(ImmutableList.of("y"), parsed.getDimension("bar"));
Assert.assertEquals(new DateTime("3000").getMillis(), parsed.getTimestampFromEpoch());
}
}
@Test
public void testMapInputRowParserSerde() throws Exception
{
final MapInputRowParser parser = new MapInputRowParser(
new JSONParseSpec(
new TimestampSpec("timeposix", "posix", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar")), ImmutableList.of("baz"), null),
null,
null
)
);
final MapInputRowParser parser2 = jsonMapper.readValue(
jsonMapper.writeValueAsBytes(parser),
MapInputRowParser.class
);
final InputRow parsed = parser2.parse(
ImmutableMap.<String, Object>of(
"foo", "x",
"bar", "y",
"qux", "z",
"timeposix", "1"
)
);
Assert.assertEquals(ImmutableList.of("foo", "bar"), parsed.getDimensions());
Assert.assertEquals(ImmutableList.of("x"), parsed.getDimension("foo"));
Assert.assertEquals(ImmutableList.of("y"), parsed.getDimension("bar"));
Assert.assertEquals(1000, parsed.getTimestampFromEpoch());
}
@Test
public void testMapInputRowParserNumbersSerde() throws Exception
{
final MapInputRowParser parser = new MapInputRowParser(
new JSONParseSpec(
new TimestampSpec("timemillis", "millis", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "values")), ImmutableList.of("toobig", "value"), null),
null,
null
)
);
final MapInputRowParser parser2 = jsonMapper.readValue(
jsonMapper.writeValueAsBytes(parser),
MapInputRowParser.class
);
final InputRow parsed = parser2.parse(
ImmutableMap.<String, Object>of(
"timemillis", 1412705931123L,
"toobig", 123E64,
"value", 123.456,
"long", 123456789000L,
"values", Lists.newArrayList(1412705931123L, 123.456, 123E45, "hello")
)
);
Assert.assertEquals(ImmutableList.of("foo", "values"), parsed.getDimensions());
Assert.assertEquals(ImmutableList.of(), parsed.getDimension("foo"));
Assert.assertEquals(
ImmutableList.of("1412705931123", "123.456", "1.23E47", "hello"),
parsed.getDimension("values")
);
Assert.assertEquals(Float.POSITIVE_INFINITY, parsed.getFloatMetric("toobig"));
Assert.assertEquals(123E64, parsed.getRaw("toobig"));
Assert.assertEquals(123.456f, parsed.getFloatMetric("value"));
Assert.assertEquals(123456789000L, parsed.getRaw("long"));
Assert.assertEquals(1.23456791E11f, parsed.getFloatMetric("long"));
Assert.assertEquals(1412705931123L, parsed.getTimestampFromEpoch());
}
private InputRow testCharsetParseHelper(Charset charset) throws Exception
{
final StringInputRowParser parser = new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar")), null, null),
null,
null
),
charset.name()
);
final ByteBufferInputRowParser parser2 = jsonMapper.readValue(
jsonMapper.writeValueAsBytes(parser),
ByteBufferInputRowParser.class
);
final InputRow parsed = parser2.parse(
ByteBuffer.wrap(
"{\"foo\":\"x\",\"bar\":\"y\",\"qux\":\"z\",\"timestamp\":\"3000\"}".getBytes(charset)
)
);
return parsed;
}
@Test
public void testFlattenParse() throws Exception
{
List<JSONPathFieldSpec> fields = new ArrayList<>();
fields.add(JSONPathFieldSpec.createNestedField("foobar1", "$.foo.bar1"));
fields.add(JSONPathFieldSpec.createNestedField("foobar2", "$.foo.bar2"));
fields.add(JSONPathFieldSpec.createNestedField("baz0", "$.baz[0]"));
fields.add(JSONPathFieldSpec.createNestedField("baz1", "$.baz[1]"));
fields.add(JSONPathFieldSpec.createNestedField("baz2", "$.baz[2]"));
fields.add(JSONPathFieldSpec.createNestedField("hey0barx", "$.hey[0].barx"));
fields.add(JSONPathFieldSpec.createNestedField("metA", "$.met.a"));
fields.add(JSONPathFieldSpec.createRootField("timestamp"));
fields.add(JSONPathFieldSpec.createRootField("foo.bar1"));
JSONPathSpec flattenSpec = new JSONPathSpec(true, fields);
final StringInputRowParser parser = new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(null, null, null),
flattenSpec,
null
)
);
final StringInputRowParser parser2 = jsonMapper.readValue(
jsonMapper.writeValueAsBytes(parser),
StringInputRowParser.class
);
final InputRow parsed = parser2.parse(
"{\"blah\":[4,5,6], \"newmet\":5, \"foo\":{\"bar1\":\"aaa\", \"bar2\":\"bbb\"}, \"baz\":[1,2,3], \"timestamp\":\"2999\", \"foo.bar1\":\"Hello world!\", \"hey\":[{\"barx\":\"asdf\"}], \"met\":{\"a\":456}}"
);
Assert.assertEquals(ImmutableList.of("foobar1", "foobar2", "baz0", "baz1", "baz2", "hey0barx", "metA", "timestamp", "foo.bar1", "blah", "newmet", "baz"), parsed.getDimensions());
Assert.assertEquals(ImmutableList.of("aaa"), parsed.getDimension("foobar1"));
Assert.assertEquals(ImmutableList.of("bbb"), parsed.getDimension("foobar2"));
Assert.assertEquals(ImmutableList.of("1"), parsed.getDimension("baz0"));
Assert.assertEquals(ImmutableList.of("2"), parsed.getDimension("baz1"));
Assert.assertEquals(ImmutableList.of("3"), parsed.getDimension("baz2"));
Assert.assertEquals(ImmutableList.of("Hello world!"), parsed.getDimension("foo.bar1"));
Assert.assertEquals(ImmutableList.of("asdf"), parsed.getDimension("hey0barx"));
Assert.assertEquals(ImmutableList.of("456"), parsed.getDimension("metA"));
Assert.assertEquals(ImmutableList.of("5"), parsed.getDimension("newmet"));
Assert.assertEquals(new DateTime("2999").getMillis(), parsed.getTimestampFromEpoch());
String testSpec = "{\"enabled\": true,\"useFieldDiscovery\": true, \"fields\": [\"parseThisRootField\"]}";
final JSONPathSpec parsedSpec = jsonMapper.readValue(testSpec, JSONPathSpec.class);
List<JSONPathFieldSpec> fieldSpecs = parsedSpec.getFields();
Assert.assertEquals(JSONPathFieldType.ROOT, fieldSpecs.get(0).getType());
Assert.assertEquals("parseThisRootField", fieldSpecs.get(0).getName());
Assert.assertEquals("parseThisRootField", fieldSpecs.get(0).getExpr());
}
}

View File

@ -0,0 +1,52 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.google.common.collect.Lists;
import com.metamx.common.parsers.JSONToLowerParser;
import com.metamx.common.parsers.Parser;
import junit.framework.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.Map;
public class JSONLowercaseParseSpecTest
{
@Test
public void testLowercasing() throws Exception
{
JSONLowercaseParseSpec spec = new JSONLowercaseParseSpec(
new TimestampSpec(
"timestamp",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("A", "B")),
Lists.<String>newArrayList(),
Lists.<SpatialDimensionSchema>newArrayList()
)
);
Parser parser = spec.makeParser();
Map<String, Object> event = parser.parse("{\"timestamp\":\"2015-01-01\",\"A\":\"foo\"}");
Assert.assertEquals("foo", event.get("a"));
}
}

View File

@ -0,0 +1,59 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import io.druid.TestObjectMapper;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import org.junit.Assert;
import org.junit.Test;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
public class JSONParseSpecTest {
private final ObjectMapper jsonMapper = new TestObjectMapper();
@Test
public void testSerde() throws IOException
{
HashMap<String, Boolean> feature = new HashMap<String, Boolean>();
feature.put("ALLOW_UNQUOTED_CONTROL_CHARS", true);
JSONParseSpec spec = new JSONParseSpec(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")), null, null),
null,
feature
);
final JSONParseSpec serde = jsonMapper.readValue(
jsonMapper.writeValueAsString(spec),
JSONParseSpec.class
);
Assert.assertEquals("timestamp", serde.getTimestampSpec().getTimestampColumn());
Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
Assert.assertEquals(Arrays.asList("bar", "foo"), serde.getDimensionsSpec().getDimensionNames());
Assert.assertEquals(feature, serde.getFeatureSpec());
}
}

View File

@ -0,0 +1,81 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import io.druid.TestObjectMapper;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
public class JSONPathSpecTest
{
private final ObjectMapper jsonMapper = new TestObjectMapper();
@Test
public void testSerde() throws IOException
{
List<JSONPathFieldSpec> fields = new ArrayList<>();
fields.add(JSONPathFieldSpec.createNestedField("foobar1", "$.foo.bar1"));
fields.add(JSONPathFieldSpec.createNestedField("baz0", "$.baz[0]"));
fields.add(JSONPathFieldSpec.createNestedField("hey0barx", "$.hey[0].barx"));
fields.add(JSONPathFieldSpec.createRootField("timestamp"));
fields.add(JSONPathFieldSpec.createRootField("foo.bar1"));
JSONPathSpec flattenSpec = new JSONPathSpec(true, fields);
final JSONPathSpec serde = jsonMapper.readValue(
jsonMapper.writeValueAsString(flattenSpec),
JSONPathSpec.class
);
Assert.assertTrue(serde.isUseFieldDiscovery());
List<JSONPathFieldSpec> serdeFields = serde.getFields();
JSONPathFieldSpec foobar1 = serdeFields.get(0);
JSONPathFieldSpec baz0 = serdeFields.get(1);
JSONPathFieldSpec hey0barx = serdeFields.get(2);
JSONPathFieldSpec timestamp = serdeFields.get(3);
JSONPathFieldSpec foodotbar1 = serdeFields.get(4);
Assert.assertEquals(JSONPathFieldType.PATH, foobar1.getType());
Assert.assertEquals("foobar1", foobar1.getName());
Assert.assertEquals("$.foo.bar1", foobar1.getExpr());
Assert.assertEquals(JSONPathFieldType.PATH, baz0.getType());
Assert.assertEquals("baz0", baz0.getName());
Assert.assertEquals("$.baz[0]", baz0.getExpr());
Assert.assertEquals(JSONPathFieldType.PATH, hey0barx.getType());
Assert.assertEquals("hey0barx", hey0barx.getName());
Assert.assertEquals("$.hey[0].barx", hey0barx.getExpr());
Assert.assertEquals(JSONPathFieldType.ROOT, timestamp.getType());
Assert.assertEquals("timestamp", timestamp.getName());
Assert.assertEquals("timestamp", timestamp.getExpr());
Assert.assertEquals(JSONPathFieldType.ROOT, foodotbar1.getType());
Assert.assertEquals("foo.bar1", foodotbar1.getName());
Assert.assertEquals("foo.bar1", foodotbar1.getExpr());
}
}

View File

@ -0,0 +1,54 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.TestObjectMapper;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
/**
*/
public class JavaScriptParseSpecTest
{
private final ObjectMapper jsonMapper = new TestObjectMapper();
@Test
public void testSerde() throws IOException
{
JavaScriptParseSpec spec = new JavaScriptParseSpec(
new TimestampSpec("abc", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null),
"abc"
);
final JavaScriptParseSpec serde = jsonMapper.readValue(
jsonMapper.writeValueAsString(spec),
JavaScriptParseSpec.class
);
Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn());
Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
Assert.assertEquals("abc", serde.getFunction());
Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames());
}
}

View File

@ -0,0 +1,73 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import junit.framework.Assert;
import org.junit.Test;
/**
*/
public class NoopInputRowParserTest
{
private final ObjectMapper mapper = new ObjectMapper();
@Test
public void testSerdeWithNullParseSpec() throws Exception
{
String jsonStr = "{ \"type\":\"noop\" }";
InputRowParser actual = mapper.readValue(
mapper.writeValueAsString(
mapper.readValue(jsonStr, InputRowParser.class)
),
InputRowParser.class
);
Assert.assertEquals(new NoopInputRowParser(null), actual);
}
@Test
public void testSerdeWithNonNullParseSpec() throws Exception
{
String jsonStr = "{"
+ "\"type\":\"noop\","
+ "\"parseSpec\":{ \"format\":\"timeAndDims\", \"dimensionsSpec\": { \"dimensions\": [\"host\"] } }"
+ "}";
InputRowParser actual = mapper.readValue(
mapper.writeValueAsString(
mapper.readValue(jsonStr, InputRowParser.class)
),
InputRowParser.class
);
Assert.assertEquals(
new NoopInputRowParser(
new TimeAndDimsParseSpec(
null,
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null)
)
),
actual
);
}
}

View File

@ -0,0 +1,89 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.google.common.collect.Lists;
import com.metamx.common.parsers.ParseException;
import org.junit.Test;
import java.util.Arrays;
public class ParseSpecTest
{
@Test(expected = ParseException.class)
public void testDuplicateNames() throws Exception
{
final ParseSpec spec = new DelimitedParseSpec(
new TimestampSpec(
"timestamp",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b", "a")),
Lists.<String>newArrayList(),
Lists.<SpatialDimensionSchema>newArrayList()
),
",",
" ",
Arrays.asList("a", "b")
);
}
@Test(expected = IllegalArgumentException.class)
public void testDimAndDimExcluOverlap() throws Exception
{
final ParseSpec spec = new DelimitedParseSpec(
new TimestampSpec(
"timestamp",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "B")),
Lists.newArrayList("B"),
Lists.<SpatialDimensionSchema>newArrayList()
),
",",
null,
Arrays.asList("a", "B")
);
}
@Test
public void testDimExclusionDuplicate() throws Exception
{
final ParseSpec spec = new DelimitedParseSpec(
new TimestampSpec(
"timestamp",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("a")),
Lists.newArrayList("B", "B"),
Lists.<SpatialDimensionSchema>newArrayList()
),
",",
null,
Arrays.asList("a", "B")
);
}
}

View File

@ -0,0 +1,57 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.TestObjectMapper;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
/**
*/
public class RegexParseSpecTest
{
private final ObjectMapper jsonMapper = new TestObjectMapper();
@Test
public void testSerde() throws IOException
{
RegexParseSpec spec = new RegexParseSpec(
new TimestampSpec("abc", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null),
"\u0001",
Arrays.asList("abc"),
"abc"
);
final RegexParseSpec serde = jsonMapper.readValue(
jsonMapper.writeValueAsString(spec),
RegexParseSpec.class
);
Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn());
Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
Assert.assertEquals("abc", serde.getPattern());
Assert.assertEquals("\u0001", serde.getListDelimiter());
Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames());
}
}

View File

@ -0,0 +1,72 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import junit.framework.Assert;
import org.junit.Test;
/**
*/
public class TimeAndDimsParseSpecTest
{
private final ObjectMapper mapper = new ObjectMapper();
@Test
public void testSerdeWithNulls() throws Exception
{
String jsonStr = "{ \"format\":\"timeAndDims\" }";
ParseSpec actual = mapper.readValue(
mapper.writeValueAsString(
mapper.readValue(jsonStr, ParseSpec.class)
),
ParseSpec.class
);
Assert.assertEquals(new TimeAndDimsParseSpec(null, null), actual);
}
@Test
public void testSerdeWithNonNulls() throws Exception
{
String jsonStr = "{"
+ "\"format\":\"timeAndDims\","
+ "\"timestampSpec\": { \"column\": \"tcol\" },"
+ "\"dimensionsSpec\": { \"dimensions\": [\"host\"] }"
+ "}";
ParseSpec actual = mapper.readValue(
mapper.writeValueAsString(
mapper.readValue(jsonStr, ParseSpec.class)
),
ParseSpec.class
);
Assert.assertEquals(
new TimeAndDimsParseSpec(
new TimestampSpec("tcol", null, null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null)
),
actual
);
}
}

View File

@ -0,0 +1,48 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.data.input.impl;
import com.google.common.collect.ImmutableMap;
import junit.framework.Assert;
import org.joda.time.DateTime;
import org.junit.Test;
public class TimestampSpecTest
{
@Test
public void testExtractTimestamp() throws Exception
{
TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null);
Assert.assertEquals(
new DateTime("2014-03-01"),
spec.extractTimestamp(ImmutableMap.<String, Object>of("TIMEstamp", "2014-03-01"))
);
}
@Test
public void testExtractTimestampWithMissingTimestampColumn() throws Exception
{
TimestampSpec spec = new TimestampSpec(null, null, new DateTime(0));
Assert.assertEquals(
new DateTime("1970-01-01"),
spec.extractTimestamp(ImmutableMap.<String, Object>of("dim", "foo"))
);
}
}

Some files were not shown because too many files have changed in this diff Show More