29 | {
30 | @Override
31 | public ByteBufferInputRowParser withParseSpec(ParseSpec parseSpec);
32 | }
33 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/Committer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 | /**
22 | * Committer includes a Runnable and a Jackson-serialized metadata object containing the offset
23 | */
24 | public interface Committer extends Runnable
25 | {
26 | /**
27 | * @return A json serialized represenation of commit metadata,
28 | * which needs to be serialized and deserialized by Jackson.
29 | * Commit metadata can be a complex type, but we recommend keeping it to List/Map/"Primitive JSON" types
30 | * */
31 | public Object getMetadata();
32 | }
33 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/Firehose.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 |
22 | import java.io.Closeable;
23 |
24 | /**
25 | * This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this
26 | * abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement
27 | * one of these and register it with the Main.
28 | *
29 | * This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends
30 | * Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this
31 | * gets passed around as an Iterator.
32 | *
33 | * The implementation of this interface only needs to be minimally thread-safe. The three methods ##hasMore(),
34 | * ##nextRow() and ##commit() are all called from the same thread. ##commit(), however, returns a callback
35 | * which will be called on another thread, so the operations inside of that callback must be thread-safe.
36 | *
37 | */
38 | public interface Firehose extends Closeable
39 | {
40 | /**
41 | * Returns whether there are more rows to process. This is used to indicate that another item is immediately
42 | * available via ##nextRow(). Thus, if the stream is still available but there are no new messages on it, this call
43 | * should block until a new message is available.
44 | *
45 | * If something happens such that the stream is no longer available, this should return false.
46 | *
47 | * @return true if and when there is another row available, false if the stream has dried up
48 | */
49 | public boolean hasMore();
50 |
51 | /**
52 | * The next row available. Should only be called if hasMore returns true.
53 | *
54 | * @return The next row
55 | */
56 | public InputRow nextRow() ;
57 |
58 | /**
59 | * Returns a runnable that will "commit" everything read up to the point at which commit() is called. This is
60 | * often equivalent to everything that has been read since the last commit() call (or instantiation of the object),
61 | * but doesn't necessarily have to be.
62 | *
63 | * This method is called when the main processing loop starts to persist its current batch of things to process.
64 | * The returned runnable will be run when the current batch has been successfully persisted, there is usually
65 | * some time lag between when this method is called and when the runnable is run. The Runnable is also run on
66 | * a separate thread so its operation should be thread-safe.
67 | *
68 | * The Runnable is essentially just a lambda/closure that is run() after data supplied by this instance has
69 | * been committed on the writer side of this interface protocol.
70 | *
71 | * A simple implementation of this interface might do nothing when run() is called
72 | * (in which case the same do-nothing instance can be returned every time), or
73 | * a more complex implementation might clean up temporary resources that are no longer needed
74 | * because of InputRows delivered by prior calls to ##nextRow().
75 | *
76 | */
77 | public Runnable commit();
78 | }
79 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/FirehoseFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 |
22 | import com.fasterxml.jackson.annotation.JsonTypeInfo;
23 | import com.metamx.common.parsers.ParseException;
24 | import io.druid.data.input.impl.InputRowParser;
25 |
26 | import java.io.IOException;
27 |
28 | @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
29 | public interface FirehoseFactory
30 | {
31 | /**
32 | * Initialization method that connects up the fire hose. If this method returns successfully it should be safe to
33 | * call hasMore() on the returned Firehose (which might subsequently block).
34 | *
35 | * If this method returns null, then any attempt to call hasMore(), nextRow(), commit() and close() on the return
36 | * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on
37 | * invalid configuration is preferred over returning null.
38 | */
39 | public Firehose connect(T parser) throws IOException, ParseException;
40 |
41 | }
42 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/FirehoseFactoryV2.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 |
22 | import com.fasterxml.jackson.annotation.JsonTypeInfo;
23 | import com.metamx.common.parsers.ParseException;
24 | import io.druid.data.input.impl.InputRowParser;
25 |
26 | import java.io.IOException;
27 | /**
28 | * Initialization method that connects up the FirehoseV2. If this method returns successfully it should be safe to
29 | * call start() on the returned FirehoseV2 (which might subsequently block).
30 | *
31 | * In contrast to V1 version, FirehoseFactoryV2 is able to pass an additional json-serialized object to FirehoseV2,
32 | * which contains last commit metadata
33 | *
34 | *
35 | * If this method returns null, then any attempt to call start(), advance(), currRow(), makeCommitter() and close() on the return
36 | * value will throw a surprising NPE. Throwing IOException on connection failure or runtime exception on
37 | * invalid configuration is preferred over returning null.
38 | */
39 | @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
40 | public interface FirehoseFactoryV2
41 | {
42 | public FirehoseV2 connect(T parser, Object lastCommit) throws IOException, ParseException;
43 |
44 | }
45 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/FirehoseV2.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 |
22 | import java.io.Closeable;
23 | /**
24 | * This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this
25 | * abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement
26 | * one of these and register it with the Main.
27 | *
28 | * In contrast to Firehose v1 version, FirehoseV2 will always operate in a "peek, then advance" manner.
29 | * And the intended usage patttern is
30 | * 1. Call start()
31 | * 2. Read currRow()
32 | * 3. Call advance()
33 | * 4. If index should be committed: commit()
34 | * 5. GOTO 2
35 | *
36 | * Note that commit() is being called *after* advance.
37 | *
38 | * This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends
39 | * Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this
40 | * gets passed around as an Iterator.
41 | *
42 | * The implementation of this interface only needs to be minimally thread-safe. The methods ##start(), ##advance(),
43 | * ##currRow() and ##makeCommitter() are all called from the same thread. ##makeCommitter(), however, returns a callback
44 | * which will be called on another thread, so the operations inside of that callback must be thread-safe.
45 | *
46 | */
47 | public interface FirehoseV2 extends Closeable
48 | {
49 | /**
50 | * For initial start
51 | * */
52 | void start() throws Exception;
53 |
54 | /**
55 | * Advance the firehose to the next offset. Implementations of this interface should make sure that
56 | * if advance() is called and throws out an exception, the next call to currRow() should return a
57 | * null value.
58 | *
59 | * @return true if and when there is another row available, false if the stream has dried up
60 | */
61 | public boolean advance();
62 |
63 | /**
64 | * @return The current row
65 | */
66 | public InputRow currRow() ;
67 |
68 | /**
69 | * Returns a Committer that will "commit" everything read up to the point at which makeCommitter() is called.
70 | *
71 | * This method is called when the main processing loop starts to persist its current batch of things to process.
72 | * The returned committer will be run when the current batch has been successfully persisted
73 | * and the metadata the committer carries can also be persisted along with segment data. There is usually
74 | * some time lag between when this method is called and when the runnable is run. The Runnable is also run on
75 | * a separate thread so its operation should be thread-safe.
76 | *
77 | * Note that "correct" usage of this interface will always call advance() before commit() if the current row
78 | * is considered in the commit.
79 | *
80 | * The Runnable is essentially just a lambda/closure that is run() after data supplied by this instance has
81 | * been committed on the writer side of this interface protocol.
82 | *
83 | * A simple implementation of this interface might do nothing when run() is called,
84 | * and save proper commit information in metadata
85 | *
86 | */
87 | public Committer makeCommitter();
88 | }
89 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/InputRow.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 |
22 | import java.util.List;
23 |
24 | /**
25 | * An InputRow is the interface definition of an event being input into the data ingestion layer.
26 | *
27 | * An InputRow is a Row with a self-describing list of the dimensions available. This list is used to
28 | * implement "schema-less" data ingestion that allows the system to add new dimensions as they appear.
29 | *
30 | */
31 | public interface
32 | InputRow extends Row
33 | {
34 | /**
35 | * Returns the dimensions that exist in this row.
36 | *
37 | * @return the dimensions that exist in this row.
38 | */
39 | public List getDimensions();
40 | }
41 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/MapBasedInputRow.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 |
22 | import org.joda.time.DateTime;
23 |
24 | import java.util.List;
25 | import java.util.Map;
26 |
27 | /**
28 | */
29 | public class MapBasedInputRow extends MapBasedRow implements InputRow
30 | {
31 | private final List dimensions;
32 |
33 | public MapBasedInputRow(
34 | long timestamp,
35 | List dimensions,
36 | Map event
37 | )
38 | {
39 | super(timestamp, event);
40 | this.dimensions = dimensions;
41 | }
42 |
43 | public MapBasedInputRow(
44 | DateTime timestamp,
45 | List dimensions,
46 | Map event
47 | )
48 | {
49 | super(timestamp, event);
50 | this.dimensions = dimensions;
51 | }
52 |
53 | @Override
54 | public List getDimensions()
55 | {
56 | return dimensions;
57 | }
58 |
59 | @Override
60 | public String toString()
61 | {
62 | return "MapBasedInputRow{" +
63 | "timestamp=" + new DateTime(getTimestampFromEpoch()) +
64 | ", event=" + getEvent() +
65 | ", dimensions=" + dimensions +
66 | '}';
67 | }
68 | }
69 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/Row.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 |
22 | import com.fasterxml.jackson.annotation.JsonSubTypes;
23 | import com.fasterxml.jackson.annotation.JsonTypeInfo;
24 | import com.metamx.common.parsers.ParseException;
25 | import org.joda.time.DateTime;
26 |
27 | import java.util.List;
28 |
29 | /**
30 | * A Row of data. This can be used for both input and output into various parts of the system. It assumes
31 | * that the user already knows the schema of the row and can query for the parts that they care about.
32 | */
33 | @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "version", defaultImpl = MapBasedRow.class)
34 | @JsonSubTypes(value = {
35 | @JsonSubTypes.Type(name = "v1", value = MapBasedRow.class)
36 | })
37 | public interface Row extends Comparable
38 | {
39 | /**
40 | * Returns the timestamp from the epoch in milliseconds. If the event happened _right now_, this would return the
41 | * same thing as System.currentTimeMillis();
42 | *
43 | * @return the timestamp from the epoch in milliseconds.
44 | */
45 | public long getTimestampFromEpoch();
46 |
47 | /**
48 | * Returns the timestamp from the epoch as an org.joda.time.DateTime. If the event happened _right now_, this would return the
49 | * same thing as new DateTime();
50 | *
51 | * @return the timestamp from the epoch as an org.joda.time.DateTime object.
52 | */
53 | public DateTime getTimestamp();
54 |
55 | /**
56 | * Returns the list of dimension values for the given column name.
57 | *
58 | *
59 | * @param dimension the column name of the dimension requested
60 | *
61 | * @return the list of values for the provided column name
62 | */
63 | public List getDimension(String dimension);
64 |
65 | /**
66 | * Returns the raw dimension value for the given column name. This is different from #getDimension which
67 | * all values to strings before returning them.
68 | *
69 | * @param dimension the column name of the dimension requested
70 | *
71 | * @return the value of the provided column name
72 | */
73 | public Object getRaw(String dimension);
74 |
75 | /**
76 | * Returns the float value of the given metric column.
77 | *
78 | *
79 | * @param metric the column name of the metric requested
80 | *
81 | * @return the float value for the provided column name.
82 | */
83 | public float getFloatMetric(String metric);
84 |
85 | /**
86 | * Returns the long value of the given metric column.
87 | *
88 | *
89 | * @param metric the column name of the metric requested
90 | *
91 | * @return the long value for the provided column name.
92 | */
93 | public long getLongMetric(String metric);
94 | }
95 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/Rows.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 |
22 | import com.google.common.collect.ImmutableList;
23 | import com.google.common.collect.ImmutableSortedSet;
24 | import com.google.common.collect.Maps;
25 | import com.metamx.common.ISE;
26 |
27 | import java.util.List;
28 | import java.util.Map;
29 | import java.util.Set;
30 | import java.util.TreeMap;
31 |
32 | /**
33 | */
34 | public class Rows
35 | {
36 | public static InputRow toCaseInsensitiveInputRow(final Row row, final List dimensions)
37 | {
38 | if (row instanceof MapBasedRow) {
39 | MapBasedRow mapBasedRow = (MapBasedRow) row;
40 |
41 | TreeMap caseInsensitiveMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
42 | caseInsensitiveMap.putAll(mapBasedRow.getEvent());
43 | return new MapBasedInputRow(
44 | mapBasedRow.getTimestamp(),
45 | dimensions,
46 | caseInsensitiveMap
47 | );
48 | }
49 | throw new ISE("Can only convert MapBasedRow objects because we are ghetto like that.");
50 | }
51 |
52 | /**
53 | * @param timeStamp rollup up timestamp to be used to create group key
54 | * @param inputRow input row
55 | * @return groupKey for the given input row
56 | */
57 | public static List toGroupKey(long timeStamp, InputRow inputRow)
58 | {
59 | final Map> dims = Maps.newTreeMap();
60 | for (final String dim : inputRow.getDimensions()) {
61 | final Set dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim));
62 | if (dimValues.size() > 0) {
63 | dims.put(dim, dimValues);
64 | }
65 | }
66 | return ImmutableList.of(
67 | timeStamp,
68 | dims
69 | );
70 | }
71 | }
72 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/CSVParseSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonCreator;
21 | import com.fasterxml.jackson.annotation.JsonProperty;
22 | import com.google.common.base.Optional;
23 | import com.google.common.base.Preconditions;
24 | import com.metamx.common.parsers.CSVParser;
25 | import com.metamx.common.parsers.ParseException;
26 | import com.metamx.common.parsers.Parser;
27 |
28 | import java.util.List;
29 |
30 | /**
31 | */
32 | public class CSVParseSpec extends ParseSpec
33 | {
34 | private final String listDelimiter;
35 | private final List columns;
36 |
37 | @JsonCreator
38 | public CSVParseSpec(
39 | @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
40 | @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
41 | @JsonProperty("listDelimiter") String listDelimiter,
42 | @JsonProperty("columns") List columns
43 | )
44 | {
45 | super(timestampSpec, dimensionsSpec);
46 |
47 | this.listDelimiter = listDelimiter;
48 | Preconditions.checkNotNull(columns, "columns");
49 | for (String column : columns) {
50 | Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column);
51 | }
52 |
53 | this.columns = columns;
54 |
55 | verify(dimensionsSpec.getDimensionNames());
56 | }
57 |
58 | @JsonProperty
59 | public String getListDelimiter()
60 | {
61 | return listDelimiter;
62 | }
63 |
64 | @JsonProperty("columns")
65 | public List getColumns()
66 | {
67 | return columns;
68 | }
69 |
70 | @Override
71 | public void verify(List usedCols)
72 | {
73 | for (String columnName : usedCols) {
74 | Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName);
75 | }
76 | }
77 |
78 | @Override
79 | public Parser makeParser()
80 | {
81 | return new CSVParser(Optional.fromNullable(listDelimiter), columns);
82 | }
83 |
84 | @Override
85 | public ParseSpec withTimestampSpec(TimestampSpec spec)
86 | {
87 | return new CSVParseSpec(spec, getDimensionsSpec(), listDelimiter, columns);
88 | }
89 |
90 | @Override
91 | public ParseSpec withDimensionsSpec(DimensionsSpec spec)
92 | {
93 | return new CSVParseSpec(getTimestampSpec(), spec, listDelimiter, columns);
94 | }
95 |
96 | public ParseSpec withColumns(List cols)
97 | {
98 | return new CSVParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, cols);
99 | }
100 | }
101 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/DelimitedParseSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonCreator;
21 | import com.fasterxml.jackson.annotation.JsonProperty;
22 | import com.google.common.base.Optional;
23 | import com.google.common.base.Preconditions;
24 | import com.google.common.collect.Lists;
25 | import com.metamx.common.parsers.DelimitedParser;
26 | import com.metamx.common.parsers.ParseException;
27 | import com.metamx.common.parsers.Parser;
28 |
29 | import java.util.List;
30 |
31 | /**
32 | */
33 | public class DelimitedParseSpec extends ParseSpec
34 | {
35 | private final String delimiter;
36 | private final String listDelimiter;
37 | private final List columns;
38 |
39 | @JsonCreator
40 | public DelimitedParseSpec(
41 | @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
42 | @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
43 | @JsonProperty("delimiter") String delimiter,
44 | @JsonProperty("listDelimiter") String listDelimiter,
45 | @JsonProperty("columns") List columns
46 | )
47 | {
48 | super(timestampSpec, dimensionsSpec);
49 |
50 | this.delimiter = delimiter;
51 | this.listDelimiter = listDelimiter;
52 | Preconditions.checkNotNull(columns, "columns");
53 | this.columns = columns;
54 | for (String column : this.columns) {
55 | Preconditions.checkArgument(!column.contains(","), "Column[%s] has a comma, it cannot", column);
56 | }
57 |
58 | verify(dimensionsSpec.getDimensionNames());
59 | }
60 |
61 | @JsonProperty("delimiter")
62 | public String getDelimiter()
63 | {
64 | return delimiter;
65 | }
66 |
67 | @JsonProperty("listDelimiter")
68 | public String getListDelimiter()
69 | {
70 | return listDelimiter;
71 | }
72 |
73 | @JsonProperty("columns")
74 | public List getColumns()
75 | {
76 | return columns;
77 | }
78 |
79 | @Override
80 | public void verify(List usedCols)
81 | {
82 | for (String columnName : usedCols) {
83 | Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName);
84 | }
85 | }
86 |
87 | @Override
88 | public Parser makeParser()
89 | {
90 | Parser retVal = new DelimitedParser(
91 | Optional.fromNullable(delimiter),
92 | Optional.fromNullable(listDelimiter)
93 | );
94 | retVal.setFieldNames(columns);
95 | return retVal;
96 | }
97 |
98 | @Override
99 | public ParseSpec withTimestampSpec(TimestampSpec spec)
100 | {
101 | return new DelimitedParseSpec(spec, getDimensionsSpec(), delimiter, listDelimiter, columns);
102 | }
103 |
104 | @Override
105 | public ParseSpec withDimensionsSpec(DimensionsSpec spec)
106 | {
107 | return new DelimitedParseSpec(getTimestampSpec(), spec, delimiter, listDelimiter, columns);
108 | }
109 |
110 | public ParseSpec withDelimiter(String delim)
111 | {
112 | return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delim, listDelimiter, columns);
113 | }
114 |
115 | public ParseSpec withListDelimiter(String delim)
116 | {
117 | return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delimiter, delim, columns);
118 | }
119 |
120 | public ParseSpec withColumns(List cols)
121 | {
122 | return new DelimitedParseSpec(getTimestampSpec(), getDimensionsSpec(), delimiter, listDelimiter, cols);
123 | }
124 | }
125 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/DimensionSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonIgnore;
24 | import com.fasterxml.jackson.annotation.JsonProperty;
25 | import com.fasterxml.jackson.annotation.JsonSubTypes;
26 | import com.fasterxml.jackson.annotation.JsonTypeInfo;
27 | import com.fasterxml.jackson.annotation.JsonValue;
28 | import com.fasterxml.jackson.databind.jsontype.NamedType;
29 | import com.fasterxml.jackson.databind.module.SimpleModule;
30 | import com.google.common.base.Preconditions;
31 |
32 | import java.util.List;
33 |
34 | /**
35 | */
36 | @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringDimensionSchema.class)
37 | @JsonSubTypes(value = {
38 | @JsonSubTypes.Type(name = DimensionSchema.STRING_TYPE_NAME, value = StringDimensionSchema.class),
39 | @JsonSubTypes.Type(name = DimensionSchema.LONG_TYPE_NAME, value = LongDimensionSchema.class),
40 | @JsonSubTypes.Type(name = DimensionSchema.FLOAT_TYPE_NAME, value = FloatDimensionSchema.class),
41 | @JsonSubTypes.Type(name = DimensionSchema.SPATIAL_TYPE_NAME, value = NewSpatialDimensionSchema.class),
42 | })
43 | public abstract class DimensionSchema
44 | {
45 | public static final String STRING_TYPE_NAME = "string";
46 | public static final String LONG_TYPE_NAME = "long";
47 | public static final String FLOAT_TYPE_NAME = "float";
48 | public static final String SPATIAL_TYPE_NAME = "spatial";
49 |
50 |
51 | // main druid and druid-api should really use the same ValueType enum.
52 | // merge them when druid-api is merged back into the main repo
53 | public enum ValueType
54 | {
55 | FLOAT,
56 | LONG,
57 | STRING,
58 | COMPLEX;
59 |
60 | @JsonValue
61 | @Override
62 | public String toString()
63 | {
64 | return this.name().toUpperCase();
65 | }
66 |
67 | @JsonCreator
68 | public static ValueType fromString(String name)
69 | {
70 | return valueOf(name.toUpperCase());
71 | }
72 | }
73 |
74 | private final String name;
75 |
76 | protected DimensionSchema(String name)
77 | {
78 | this.name = Preconditions.checkNotNull(name, "Dimension name cannot be null.");
79 | }
80 |
81 | @JsonProperty
82 | public String getName()
83 | {
84 | return name;
85 | };
86 |
87 | @JsonIgnore
88 | public abstract String getTypeName();
89 |
90 | @JsonIgnore
91 | public abstract ValueType getValueType();
92 |
93 | @Override
94 | public boolean equals(Object o)
95 | {
96 | if (this == o) {
97 | return true;
98 | }
99 | if (o == null || getClass() != o.getClass()) {
100 | return false;
101 | }
102 |
103 | DimensionSchema that = (DimensionSchema) o;
104 |
105 | return name.equals(that.name);
106 |
107 | }
108 |
109 | @Override
110 | public int hashCode()
111 | {
112 | return name.hashCode();
113 | }
114 | }
115 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.google.common.base.Throwables;
23 | import io.druid.data.input.Firehose;
24 | import io.druid.data.input.InputRow;
25 | import io.druid.utils.Runnables;
26 | import org.apache.commons.io.LineIterator;
27 |
28 | import java.io.IOException;
29 | import java.util.Iterator;
30 |
31 | /**
32 | */
33 | public class FileIteratingFirehose implements Firehose
34 | {
35 | private final Iterator lineIterators;
36 | private final StringInputRowParser parser;
37 |
38 | private LineIterator lineIterator = null;
39 |
40 | public FileIteratingFirehose(
41 | Iterator lineIterators,
42 | StringInputRowParser parser
43 | )
44 | {
45 | this.lineIterators = lineIterators;
46 | this.parser = parser;
47 | }
48 |
49 | @Override
50 | public boolean hasMore()
51 | {
52 | while ((lineIterator == null || !lineIterator.hasNext()) && lineIterators.hasNext()) {
53 | lineIterator = lineIterators.next();
54 | }
55 |
56 | return lineIterator != null && lineIterator.hasNext();
57 | }
58 |
59 | @Override
60 | public InputRow nextRow()
61 | {
62 | try {
63 | if (lineIterator == null || !lineIterator.hasNext()) {
64 | // Close old streams, maybe.
65 | if (lineIterator != null) {
66 | lineIterator.close();
67 | }
68 |
69 | lineIterator = lineIterators.next();
70 | }
71 |
72 | return parser.parse(lineIterator.next());
73 | }
74 | catch (Exception e) {
75 | throw Throwables.propagate(e);
76 | }
77 | }
78 |
79 | @Override
80 | public Runnable commit()
81 | {
82 | return Runnables.getNoopRunnable();
83 | }
84 |
85 | @Override
86 | public void close() throws IOException
87 | {
88 | if (lineIterator != null) {
89 | lineIterator.close();
90 | }
91 | }
92 | }
93 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/FloatDimensionSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonIgnore;
24 | import com.fasterxml.jackson.annotation.JsonProperty;
25 |
26 | public class FloatDimensionSchema extends DimensionSchema
27 | {
28 | @JsonCreator
29 | public FloatDimensionSchema(
30 | @JsonProperty("name") String name
31 | )
32 | {
33 | super(name);
34 | }
35 |
36 | @Override
37 | public String getTypeName()
38 | {
39 | return DimensionSchema.FLOAT_TYPE_NAME;
40 | }
41 |
42 | @Override
43 | @JsonIgnore
44 | public ValueType getValueType()
45 | {
46 | return ValueType.FLOAT;
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/InputRowParser.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonSubTypes;
21 | import com.fasterxml.jackson.annotation.JsonTypeInfo;
22 | import com.metamx.common.parsers.ParseException;
23 | import io.druid.data.input.InputRow;
24 |
25 | @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class)
26 | @JsonSubTypes(value = {
27 | @JsonSubTypes.Type(name = "string", value = StringInputRowParser.class),
28 | @JsonSubTypes.Type(name = "map", value = MapInputRowParser.class),
29 | @JsonSubTypes.Type(name = "noop", value = NoopInputRowParser.class)
30 | })
31 | public interface InputRowParser
32 | {
33 | public InputRow parse(T input) ;
34 |
35 | public ParseSpec getParseSpec();
36 |
37 | public InputRowParser withParseSpec(ParseSpec parseSpec) ;
38 | }
39 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/JSONLowercaseParseSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonCreator;
21 | import com.fasterxml.jackson.annotation.JsonProperty;
22 | import com.fasterxml.jackson.databind.ObjectMapper;
23 | import com.metamx.common.parsers.JSONToLowerParser;
24 | import com.metamx.common.parsers.Parser;
25 |
26 | import java.util.List;
27 |
28 | /**
29 | * This class is only here for backwards compatibility
30 | */
31 | @Deprecated
32 | public class JSONLowercaseParseSpec extends ParseSpec
33 | {
34 | private final ObjectMapper objectMapper;
35 |
36 | @JsonCreator
37 | public JSONLowercaseParseSpec(
38 | @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
39 | @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec
40 | )
41 | {
42 | super(timestampSpec, dimensionsSpec);
43 | this.objectMapper = new ObjectMapper();
44 | }
45 |
46 | @Override
47 | public void verify(List usedCols)
48 | {
49 | }
50 |
51 | @Override
52 | public Parser makeParser()
53 | {
54 | return new JSONToLowerParser(objectMapper, null, null);
55 | }
56 |
57 | @Override
58 | public ParseSpec withTimestampSpec(TimestampSpec spec)
59 | {
60 | return new JSONLowercaseParseSpec(spec, getDimensionsSpec());
61 | }
62 |
63 | @Override
64 | public ParseSpec withDimensionsSpec(DimensionsSpec spec)
65 | {
66 | return new JSONLowercaseParseSpec(getTimestampSpec(), spec);
67 | }
68 | }
69 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/JSONParseSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonCreator;
21 | import com.fasterxml.jackson.annotation.JsonProperty;
22 | import com.fasterxml.jackson.core.JsonParser.Feature;
23 | import com.fasterxml.jackson.databind.ObjectMapper;
24 | import com.metamx.common.parsers.JSONPathParser;
25 | import com.metamx.common.parsers.Parser;
26 |
27 | import java.util.ArrayList;
28 | import java.util.HashMap;
29 | import java.util.List;
30 | import java.util.Map;
31 |
32 | /**
33 | */
34 | public class JSONParseSpec extends ParseSpec
35 | {
36 | private final ObjectMapper objectMapper;
37 | private final JSONPathSpec flattenSpec;
38 | private final Map featureSpec;
39 |
40 | @JsonCreator
41 | public JSONParseSpec(
42 | @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
43 | @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
44 | @JsonProperty("flattenSpec") JSONPathSpec flattenSpec,
45 | @JsonProperty("featureSpec") Map featureSpec
46 | )
47 | {
48 | super(timestampSpec, dimensionsSpec);
49 | this.objectMapper = new ObjectMapper();
50 | this.flattenSpec = flattenSpec != null ? flattenSpec : new JSONPathSpec(true, null);
51 | this.featureSpec = (featureSpec == null) ? new HashMap() : featureSpec;
52 | for (Map.Entry entry : this.featureSpec.entrySet()) {
53 | Feature feature = Feature.valueOf(entry.getKey());
54 | objectMapper.configure(feature, entry.getValue());
55 | }
56 | }
57 |
58 | @Deprecated
59 | public JSONParseSpec(TimestampSpec ts, DimensionsSpec dims)
60 | {
61 | this(ts, dims, null, null);
62 | }
63 |
64 | @Override
65 | public void verify(List usedCols)
66 | {
67 | }
68 |
69 | @Override
70 | public Parser makeParser()
71 | {
72 | return new JSONPathParser(
73 | convertFieldSpecs(flattenSpec.getFields()),
74 | flattenSpec.isUseFieldDiscovery(),
75 | objectMapper
76 | );
77 | }
78 |
79 | @Override
80 | public ParseSpec withTimestampSpec(TimestampSpec spec)
81 | {
82 | return new JSONParseSpec(spec, getDimensionsSpec(), getFlattenSpec(), getFeatureSpec());
83 | }
84 |
85 | @Override
86 | public ParseSpec withDimensionsSpec(DimensionsSpec spec)
87 | {
88 | return new JSONParseSpec(getTimestampSpec(), spec, getFlattenSpec(), getFeatureSpec());
89 | }
90 |
91 | @JsonProperty
92 | public JSONPathSpec getFlattenSpec()
93 | {
94 | return flattenSpec;
95 | }
96 |
97 | @JsonProperty
98 | public Map getFeatureSpec()
99 | {
100 | return featureSpec;
101 | }
102 |
103 | private List convertFieldSpecs(List druidFieldSpecs)
104 | {
105 | List newSpecs = new ArrayList<>();
106 | for (JSONPathFieldSpec druidSpec : druidFieldSpecs) {
107 | JSONPathParser.FieldType type;
108 | switch (druidSpec.getType()) {
109 | case ROOT:
110 | type = JSONPathParser.FieldType.ROOT;
111 | break;
112 | case PATH:
113 | type = JSONPathParser.FieldType.PATH;
114 | break;
115 | default:
116 | throw new IllegalArgumentException("Invalid type for field " + druidSpec.getName());
117 | }
118 |
119 | JSONPathParser.FieldSpec newSpec = new JSONPathParser.FieldSpec(
120 | type,
121 | druidSpec.getName(),
122 | druidSpec.getExpr()
123 | );
124 | newSpecs.add(newSpec);
125 | }
126 | return newSpecs;
127 | }
128 | }
129 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/JSONPathFieldSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonProperty;
24 |
25 | public class JSONPathFieldSpec
26 | {
27 | private final JSONPathFieldType type;
28 | private final String name;
29 | private final String expr;
30 |
31 | @JsonCreator
32 | public JSONPathFieldSpec(
33 | @JsonProperty("type") JSONPathFieldType type,
34 | @JsonProperty("name") String name,
35 | @JsonProperty("expr") String expr
36 | )
37 | {
38 | this.type = type;
39 | this.name = name;
40 | this.expr = expr;
41 | }
42 |
43 | @JsonProperty
44 | public JSONPathFieldType getType()
45 | {
46 | return type;
47 | }
48 |
49 | @JsonProperty
50 | public String getName()
51 | {
52 | return name;
53 | }
54 |
55 | @JsonProperty
56 | public String getExpr()
57 | {
58 | return expr;
59 | }
60 |
61 | @JsonCreator
62 | public static JSONPathFieldSpec fromString(String name)
63 | {
64 | return JSONPathFieldSpec.createRootField(name);
65 | }
66 |
67 | public static JSONPathFieldSpec createNestedField(String name, String expr)
68 | {
69 | return new JSONPathFieldSpec(JSONPathFieldType.PATH, name, expr);
70 | }
71 |
72 | public static JSONPathFieldSpec createRootField(String name)
73 | {
74 | return new JSONPathFieldSpec(JSONPathFieldType.ROOT, name, name);
75 | }
76 | }
77 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/JSONPathFieldType.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonValue;
24 |
25 | public enum JSONPathFieldType
26 | {
27 | ROOT,
28 | PATH;
29 |
30 | @JsonValue
31 | @Override
32 | public String toString()
33 | {
34 | return this.name().toLowerCase();
35 | }
36 |
37 | @JsonCreator
38 | public static JSONPathFieldType fromString(String name)
39 | {
40 | return valueOf(name.toUpperCase());
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/JSONPathSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonProperty;
24 | import com.google.common.collect.ImmutableList;
25 |
26 | import java.util.List;
27 |
28 | public class JSONPathSpec
29 | {
30 | private final boolean useFieldDiscovery;
31 | private final List fields;
32 |
33 | @JsonCreator
34 | public JSONPathSpec(
35 | @JsonProperty("useFieldDiscovery") Boolean useFieldDiscovery,
36 | @JsonProperty("fields") List fields
37 | )
38 | {
39 | this.useFieldDiscovery = useFieldDiscovery == null ? true : useFieldDiscovery;
40 | this.fields = fields == null ? ImmutableList.of() : fields;
41 | }
42 |
43 | @JsonProperty
44 | public boolean isUseFieldDiscovery()
45 | {
46 | return useFieldDiscovery;
47 | }
48 |
49 | @JsonProperty
50 | public List getFields()
51 | {
52 | return fields;
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/JavaScriptParseSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonProperty;
24 | import com.metamx.common.parsers.JavaScriptParser;
25 | import com.metamx.common.parsers.Parser;
26 |
27 | import java.util.List;
28 |
29 | /**
30 | */
31 | public class JavaScriptParseSpec extends ParseSpec
32 | {
33 | private final String function;
34 |
35 | @JsonCreator
36 | public JavaScriptParseSpec(
37 | @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
38 | @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
39 | @JsonProperty("function") String function
40 | )
41 | {
42 | super(timestampSpec, dimensionsSpec);
43 |
44 | this.function = function;
45 | }
46 |
47 | @JsonProperty("function")
48 | public String getFunction()
49 | {
50 | return function;
51 | }
52 |
53 | @Override
54 | public void verify(List usedCols)
55 | {
56 | }
57 |
58 | @Override
59 | public Parser makeParser()
60 | {
61 | return new JavaScriptParser(function);
62 | }
63 |
64 | @Override
65 | public ParseSpec withTimestampSpec(TimestampSpec spec)
66 | {
67 | return new JavaScriptParseSpec(spec, getDimensionsSpec(), function);
68 | }
69 |
70 | @Override
71 | public ParseSpec withDimensionsSpec(DimensionsSpec spec)
72 | {
73 | return new JavaScriptParseSpec(getTimestampSpec(), spec, function);
74 | }
75 |
76 | public ParseSpec withFunction(String fn)
77 | {
78 | return new JavaScriptParseSpec(getTimestampSpec(), getDimensionsSpec(), fn);
79 | }
80 | }
81 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/LongDimensionSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonIgnore;
24 | import com.fasterxml.jackson.annotation.JsonProperty;
25 |
26 | public class LongDimensionSchema extends DimensionSchema
27 | {
28 | @JsonCreator
29 | public LongDimensionSchema(
30 | @JsonProperty("name") String name
31 | )
32 | {
33 | super(name);
34 | }
35 |
36 | @Override
37 | public String getTypeName()
38 | {
39 | return DimensionSchema.LONG_TYPE_NAME;
40 | }
41 |
42 | @Override
43 | @JsonIgnore
44 | public ValueType getValueType()
45 | {
46 | return ValueType.LONG;
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/MapInputRowParser.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonCreator;
21 | import com.fasterxml.jackson.annotation.JsonProperty;
22 | import com.google.common.collect.Lists;
23 | import com.google.common.collect.Sets;
24 | import com.metamx.common.logger.Logger;
25 | import com.metamx.common.parsers.ParseException;
26 | import io.druid.data.input.InputRow;
27 | import io.druid.data.input.MapBasedInputRow;
28 | import org.joda.time.DateTime;
29 |
30 | import java.util.List;
31 | import java.util.Map;
32 |
33 | public class MapInputRowParser implements InputRowParser>
34 | {
35 | private final ParseSpec parseSpec;
36 |
37 | @JsonCreator
38 | public MapInputRowParser(
39 | @JsonProperty("parseSpec") ParseSpec parseSpec
40 | )
41 | {
42 | this.parseSpec = parseSpec;
43 | }
44 |
45 | @Override
46 | public InputRow parse(Map theMap)
47 | {
48 | final List dimensions = parseSpec.getDimensionsSpec().hasCustomDimensions()
49 | ? parseSpec.getDimensionsSpec().getDimensionNames()
50 | : Lists.newArrayList(
51 | Sets.difference(
52 | theMap.keySet(),
53 | parseSpec.getDimensionsSpec()
54 | .getDimensionExclusions()
55 | )
56 | );
57 |
58 | final DateTime timestamp;
59 | try {
60 | timestamp = parseSpec.getTimestampSpec().extractTimestamp(theMap);
61 | if (timestamp == null) {
62 | final String input = theMap.toString();
63 | throw new NullPointerException(
64 | String.format(
65 | "Null timestamp in input: %s",
66 | input.length() < 100 ? input : input.substring(0, 100) + "..."
67 | )
68 | );
69 | }
70 | }
71 | catch (Exception e) {
72 | throw new ParseException(e, "Unparseable timestamp found!");
73 | }
74 |
75 | return new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap);
76 | }
77 |
78 | @JsonProperty
79 | @Override
80 | public ParseSpec getParseSpec()
81 | {
82 | return parseSpec;
83 | }
84 |
85 | @Override
86 | public InputRowParser withParseSpec(ParseSpec parseSpec)
87 | {
88 | return new MapInputRowParser(parseSpec);
89 | }
90 | }
91 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/NewSpatialDimensionSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonCreator;
21 | import com.fasterxml.jackson.annotation.JsonIgnore;
22 | import com.fasterxml.jackson.annotation.JsonProperty;
23 |
24 | import java.util.List;
25 |
26 | /**
27 | * NOTE:
28 | * This class should be deprecated after Druid supports configurable index types on dimensions.
29 | * When that exists, this should be the implementation: https://github.com/druid-io/druid/issues/2622
30 | *
31 | * This is a stop-gap solution to consolidate the dimension specs and remove the separate spatial
32 | * section in DimensionsSpec.
33 | */
34 | public class NewSpatialDimensionSchema extends DimensionSchema
35 | {
36 | private final List dims;
37 |
38 | @JsonCreator
39 | public NewSpatialDimensionSchema(
40 | @JsonProperty("name") String name,
41 | @JsonProperty("dims") List dims
42 | )
43 | {
44 | super(name);
45 | this.dims = dims;
46 | }
47 |
48 | @JsonProperty
49 | public List getDims()
50 | {
51 | return dims;
52 | }
53 |
54 | @Override
55 | public String getTypeName()
56 | {
57 | return DimensionSchema.SPATIAL_TYPE_NAME;
58 | }
59 |
60 | @Override
61 | @JsonIgnore
62 | public ValueType getValueType()
63 | {
64 | return ValueType.STRING;
65 | }
66 |
67 | @Override
68 | public boolean equals(Object o)
69 | {
70 | if (this == o) {
71 | return true;
72 | }
73 | if (o == null || getClass() != o.getClass()) {
74 | return false;
75 | }
76 |
77 | NewSpatialDimensionSchema that = (NewSpatialDimensionSchema) o;
78 |
79 | return dims != null ? dims.equals(that.dims) : that.dims == null;
80 |
81 | }
82 |
83 | @Override
84 | public int hashCode()
85 | {
86 | return dims != null ? dims.hashCode() : 0;
87 | }
88 | }
89 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/NoopInputRowParser.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonProperty;
24 | import io.druid.data.input.InputRow;
25 |
26 | /**
27 | */
28 | public class NoopInputRowParser implements InputRowParser
29 | {
30 | private final ParseSpec parseSpec;
31 |
32 | @JsonCreator
33 | public NoopInputRowParser(
34 | @JsonProperty("parseSpec") ParseSpec parseSpec
35 | )
36 | {
37 | this.parseSpec = parseSpec != null ? parseSpec : new TimeAndDimsParseSpec(null, null);
38 | }
39 |
40 | @Override
41 | public InputRow parse(InputRow input)
42 | {
43 | return input;
44 | }
45 |
46 | @Override
47 | public ParseSpec getParseSpec()
48 | {
49 | return parseSpec;
50 | }
51 |
52 | @Override
53 | public InputRowParser withParseSpec(ParseSpec parseSpec)
54 | {
55 | return new NoopInputRowParser(parseSpec);
56 | }
57 |
58 | @Override
59 | public boolean equals(Object o)
60 | {
61 | if (this == o) {
62 | return true;
63 | }
64 | if (o == null || getClass() != o.getClass()) {
65 | return false;
66 | }
67 |
68 | NoopInputRowParser that = (NoopInputRowParser) o;
69 |
70 | return parseSpec.equals(that.parseSpec);
71 |
72 | }
73 |
74 | @Override
75 | public int hashCode()
76 | {
77 | return parseSpec.hashCode();
78 | }
79 | }
80 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/ParseSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonProperty;
21 | import com.fasterxml.jackson.annotation.JsonSubTypes;
22 | import com.fasterxml.jackson.annotation.JsonTypeInfo;
23 | import com.metamx.common.parsers.Parser;
24 |
25 | import java.util.List;
26 |
27 | /**
28 | */
29 | @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format", defaultImpl = DelimitedParseSpec.class)
30 | @JsonSubTypes(value = {
31 | @JsonSubTypes.Type(name = "json", value = JSONParseSpec.class),
32 | @JsonSubTypes.Type(name = "csv", value = CSVParseSpec.class),
33 | @JsonSubTypes.Type(name = "tsv", value = DelimitedParseSpec.class),
34 | @JsonSubTypes.Type(name = "jsonLowercase", value = JSONLowercaseParseSpec.class),
35 | @JsonSubTypes.Type(name = "timeAndDims", value = TimeAndDimsParseSpec.class),
36 | @JsonSubTypes.Type(name = "regex", value = RegexParseSpec.class),
37 | @JsonSubTypes.Type(name = "javascript", value = JavaScriptParseSpec.class)
38 |
39 | })
40 | public abstract class ParseSpec
41 | {
42 | private final TimestampSpec timestampSpec;
43 | private final DimensionsSpec dimensionsSpec;
44 |
45 | protected ParseSpec(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec)
46 | {
47 | this.timestampSpec = timestampSpec;
48 | this.dimensionsSpec = dimensionsSpec;
49 | }
50 |
51 | @JsonProperty
52 | public TimestampSpec getTimestampSpec()
53 | {
54 | return timestampSpec;
55 | }
56 |
57 | @JsonProperty
58 | public DimensionsSpec getDimensionsSpec()
59 | {
60 | return dimensionsSpec;
61 | }
62 |
63 | public void verify(List usedCols)
64 | {
65 | // do nothing
66 | }
67 |
68 | public Parser makeParser()
69 | {
70 | return null;
71 | }
72 |
73 | public ParseSpec withTimestampSpec(TimestampSpec spec)
74 | {
75 | throw new UnsupportedOperationException();
76 | }
77 |
78 | public ParseSpec withDimensionsSpec(DimensionsSpec spec)
79 | {
80 | throw new UnsupportedOperationException();
81 | }
82 |
83 | @Override
84 | public boolean equals(Object o)
85 | {
86 | if (this == o) {
87 | return true;
88 | }
89 | if (o == null || getClass() != o.getClass()) {
90 | return false;
91 | }
92 |
93 | ParseSpec parseSpec = (ParseSpec) o;
94 |
95 | if (timestampSpec != null ? !timestampSpec.equals(parseSpec.timestampSpec) : parseSpec.timestampSpec != null) {
96 | return false;
97 | }
98 | return !(dimensionsSpec != null
99 | ? !dimensionsSpec.equals(parseSpec.dimensionsSpec)
100 | : parseSpec.dimensionsSpec != null);
101 |
102 | }
103 |
104 | @Override
105 | public int hashCode()
106 | {
107 | int result = timestampSpec != null ? timestampSpec.hashCode() : 0;
108 | result = 31 * result + (dimensionsSpec != null ? dimensionsSpec.hashCode() : 0);
109 | return result;
110 | }
111 | }
112 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/RegexParseSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonProperty;
24 | import com.google.common.base.Optional;
25 | import com.google.common.base.Preconditions;
26 | import com.metamx.common.parsers.Parser;
27 | import com.metamx.common.parsers.RegexParser;
28 |
29 | import java.util.List;
30 |
31 | /**
32 | */
33 | public class RegexParseSpec extends ParseSpec
34 | {
35 | private final String listDelimiter;
36 | private final List columns;
37 | private final String pattern;
38 |
39 | @JsonCreator
40 | public RegexParseSpec(
41 | @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
42 | @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
43 | @JsonProperty("listDelimiter") String listDelimiter,
44 | @JsonProperty("columns") List columns,
45 | @JsonProperty("pattern") String pattern
46 | )
47 | {
48 | super(timestampSpec, dimensionsSpec);
49 |
50 | this.listDelimiter = listDelimiter;
51 | this.columns = columns;
52 | this.pattern = pattern;
53 |
54 | verify(dimensionsSpec.getDimensionNames());
55 | }
56 |
57 | @JsonProperty
58 | public String getListDelimiter()
59 | {
60 | return listDelimiter;
61 | }
62 |
63 | @JsonProperty("pattern")
64 | public String getPattern()
65 | {
66 | return pattern;
67 | }
68 |
69 | @JsonProperty
70 | public List getColumns()
71 | {
72 | return columns;
73 | }
74 |
75 | @Override
76 | public void verify(List usedCols)
77 | {
78 | if (columns != null) {
79 | for (String columnName : usedCols) {
80 | Preconditions.checkArgument(columns.contains(columnName), "column[%s] not in columns.", columnName);
81 | }
82 | }
83 | }
84 |
85 | @Override
86 | public Parser makeParser()
87 | {
88 | if (columns == null) {
89 | return new RegexParser(pattern, Optional.fromNullable(listDelimiter));
90 | }
91 | return new RegexParser(pattern, Optional.fromNullable(listDelimiter), columns);
92 | }
93 |
94 | @Override
95 | public ParseSpec withTimestampSpec(TimestampSpec spec)
96 | {
97 | return new RegexParseSpec(spec, getDimensionsSpec(), listDelimiter, columns, pattern);
98 | }
99 |
100 | @Override
101 | public ParseSpec withDimensionsSpec(DimensionsSpec spec)
102 | {
103 | return new RegexParseSpec(getTimestampSpec(), spec, listDelimiter, columns, pattern);
104 | }
105 |
106 | public ParseSpec withColumns(List cols)
107 | {
108 | return new RegexParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, cols, pattern);
109 | }
110 |
111 | public ParseSpec withPattern(String pat)
112 | {
113 | return new RegexParseSpec(getTimestampSpec(), getDimensionsSpec(), listDelimiter, columns, pat);
114 | }
115 | }
116 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/SpatialDimensionSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonCreator;
21 | import com.fasterxml.jackson.annotation.JsonIgnore;
22 | import com.fasterxml.jackson.annotation.JsonProperty;
23 |
24 | import java.util.List;
25 |
26 | /**
27 | */
28 | @Deprecated
29 | public class SpatialDimensionSchema
30 | {
31 | private final String dimName;
32 | private final List dims;
33 |
34 | @JsonCreator
35 | public SpatialDimensionSchema(
36 | @JsonProperty("dimName") String dimName,
37 | @JsonProperty("dims") List dims
38 | )
39 | {
40 | this.dimName = dimName;
41 | this.dims = dims;
42 | }
43 |
44 | @JsonProperty
45 | public String getDimName()
46 | {
47 | return dimName;
48 | }
49 |
50 | @JsonProperty
51 | public List getDims()
52 | {
53 | return dims;
54 | }
55 |
56 | @Override
57 | public boolean equals(Object o)
58 | {
59 | if (this == o) {
60 | return true;
61 | }
62 | if (o == null || getClass() != o.getClass()) {
63 | return false;
64 | }
65 |
66 | SpatialDimensionSchema that = (SpatialDimensionSchema) o;
67 |
68 | if (dimName != null ? !dimName.equals(that.dimName) : that.dimName != null) {
69 | return false;
70 | }
71 | return dims != null ? dims.equals(that.dims) : that.dims == null;
72 |
73 | }
74 |
75 | @Override
76 | public int hashCode()
77 | {
78 | int result = dimName != null ? dimName.hashCode() : 0;
79 | result = 31 * result + (dims != null ? dims.hashCode() : 0);
80 | return result;
81 | }
82 | }
83 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/StringDimensionSchema.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonIgnore;
24 | import com.fasterxml.jackson.annotation.JsonProperty;
25 |
26 | public class StringDimensionSchema extends DimensionSchema
27 | {
28 | @JsonCreator
29 | public static StringDimensionSchema create(String name) {
30 | return new StringDimensionSchema(name);
31 | }
32 |
33 | @JsonCreator
34 | public StringDimensionSchema(
35 | @JsonProperty("name") String name
36 | )
37 | {
38 | super(name);
39 | }
40 |
41 | @Override
42 | public String getTypeName()
43 | {
44 | return DimensionSchema.STRING_TYPE_NAME;
45 | }
46 |
47 | @Override
48 | @JsonIgnore
49 | public ValueType getValueType()
50 | {
51 | return ValueType.STRING;
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/StringInputRowParser.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonCreator;
21 | import com.fasterxml.jackson.annotation.JsonProperty;
22 | import com.google.common.base.Charsets;
23 | import com.metamx.common.parsers.ParseException;
24 | import com.metamx.common.parsers.Parser;
25 | import io.druid.data.input.ByteBufferInputRowParser;
26 | import io.druid.data.input.InputRow;
27 |
28 | import java.nio.ByteBuffer;
29 | import java.nio.CharBuffer;
30 | import java.nio.charset.Charset;
31 | import java.nio.charset.CoderResult;
32 | import java.nio.charset.CodingErrorAction;
33 | import java.util.Map;
34 |
35 | /**
36 | */
37 | public class StringInputRowParser implements ByteBufferInputRowParser
38 | {
39 | private static final Charset DEFAULT_CHARSET = Charsets.UTF_8;
40 |
41 | private final ParseSpec parseSpec;
42 | private final MapInputRowParser mapParser;
43 | private final Parser parser;
44 | private final Charset charset;
45 |
46 | private CharBuffer chars = null;
47 |
48 | @JsonCreator
49 | public StringInputRowParser(
50 | @JsonProperty("parseSpec") ParseSpec parseSpec,
51 | @JsonProperty("encoding") String encoding
52 | )
53 | {
54 | this.parseSpec = parseSpec;
55 | this.mapParser = new MapInputRowParser(parseSpec);
56 | this.parser = parseSpec.makeParser();
57 |
58 | if (encoding != null) {
59 | this.charset = Charset.forName(encoding);
60 | } else {
61 | this.charset = DEFAULT_CHARSET;
62 | }
63 | }
64 |
65 | @Deprecated
66 | public StringInputRowParser(ParseSpec parseSpec)
67 | {
68 | this(parseSpec, null);
69 | }
70 |
71 | @Override
72 | public InputRow parse(ByteBuffer input)
73 | {
74 | return parseMap(buildStringKeyMap(input));
75 | }
76 |
77 | @JsonProperty
78 | @Override
79 | public ParseSpec getParseSpec()
80 | {
81 | return parseSpec;
82 | }
83 |
84 | @JsonProperty
85 | public String getEncoding()
86 | {
87 | return charset.name();
88 | }
89 |
90 | @Override
91 | public StringInputRowParser withParseSpec(ParseSpec parseSpec)
92 | {
93 | return new StringInputRowParser(parseSpec, getEncoding());
94 | }
95 |
96 | private Map buildStringKeyMap(ByteBuffer input)
97 | {
98 | int payloadSize = input.remaining();
99 |
100 | if (chars == null || chars.remaining() < payloadSize) {
101 | chars = CharBuffer.allocate(payloadSize);
102 | }
103 |
104 | final CoderResult coderResult = charset.newDecoder()
105 | .onMalformedInput(CodingErrorAction.REPLACE)
106 | .onUnmappableCharacter(CodingErrorAction.REPLACE)
107 | .decode(input, chars, true);
108 |
109 | Map theMap;
110 | if (coderResult.isUnderflow()) {
111 | chars.flip();
112 | try {
113 | theMap = parseString(chars.toString());
114 | }
115 | finally {
116 | chars.clear();
117 | }
118 | } else {
119 | throw new ParseException("Failed with CoderResult[%s]", coderResult);
120 | }
121 | return theMap;
122 | }
123 |
124 | private Map parseString(String inputString)
125 | {
126 | return parser.parse(inputString);
127 | }
128 |
129 | public InputRow parse(String input)
130 | {
131 | return parseMap(parseString(input));
132 | }
133 |
134 | private InputRow parseMap(Map theMap)
135 | {
136 | return mapParser.parse(theMap);
137 | }
138 | }
139 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/TimeAndDimsParseSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonProperty;
24 | import com.metamx.common.parsers.Parser;
25 |
26 | import java.util.List;
27 | import java.util.Map;
28 |
29 | /**
30 | */
31 | public class TimeAndDimsParseSpec extends ParseSpec
32 | {
33 | @JsonCreator
34 | public TimeAndDimsParseSpec(
35 | @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
36 | @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec
37 | )
38 | {
39 | super(
40 | timestampSpec != null ? timestampSpec : new TimestampSpec(null, null, null),
41 | dimensionsSpec != null ? dimensionsSpec : new DimensionsSpec(null, null, null)
42 | );
43 | }
44 |
45 | public Parser makeParser()
46 | {
47 | return new Parser()
48 | {
49 | @Override
50 | public Map parse(String input)
51 | {
52 | throw new UnsupportedOperationException("not supported");
53 | }
54 |
55 | @Override
56 | public void setFieldNames(Iterable fieldNames)
57 | {
58 | throw new UnsupportedOperationException("not supported");
59 | }
60 |
61 | @Override
62 | public List getFieldNames()
63 | {
64 | throw new UnsupportedOperationException("not supported");
65 | }
66 | };
67 | }
68 |
69 | public ParseSpec withTimestampSpec(TimestampSpec spec)
70 | {
71 | return new TimeAndDimsParseSpec(spec, getDimensionsSpec());
72 | }
73 |
74 | public ParseSpec withDimensionsSpec(DimensionsSpec spec)
75 | {
76 | return new TimeAndDimsParseSpec(getTimestampSpec(), spec);
77 | }
78 | }
79 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/data/input/impl/TimestampSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Druid - a distributed column store.
3 | * Copyright 2012 - 2015 Metamarkets Group Inc.
4 | *
5 | * Licensed under the Apache License, Version 2.0 (the "License");
6 | * you may not use this file except in compliance with the License.
7 | * You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS,
13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | * See the License for the specific language governing permissions and
15 | * limitations under the License.
16 | */
17 |
18 | package io.druid.data.input.impl;
19 |
20 | import com.fasterxml.jackson.annotation.JsonCreator;
21 | import com.fasterxml.jackson.annotation.JsonProperty;
22 | import com.google.common.base.Function;
23 | import com.metamx.common.parsers.ParserUtils;
24 | import com.metamx.common.parsers.TimestampParser;
25 | import org.joda.time.DateTime;
26 |
27 | import java.util.Map;
28 |
29 | /**
30 | */
31 | public class TimestampSpec
32 | {
33 | private static final String DEFAULT_COLUMN = "timestamp";
34 | private static final String DEFAULT_FORMAT = "auto";
35 | private static final DateTime DEFAULT_MISSING_VALUE = null;
36 |
37 | private final String timestampColumn;
38 | private final String timestampFormat;
39 | private final Function timestampConverter;
40 | // this value should never be set for production data
41 | private final DateTime missingValue;
42 |
43 | @JsonCreator
44 | public TimestampSpec(
45 | @JsonProperty("column") String timestampColumn,
46 | @JsonProperty("format") String format,
47 | // this value should never be set for production data
48 | @JsonProperty("missingValue") DateTime missingValue
49 | )
50 | {
51 | this.timestampColumn = (timestampColumn == null) ? DEFAULT_COLUMN : timestampColumn;
52 | this.timestampFormat = format == null ? DEFAULT_FORMAT : format;
53 | this.timestampConverter = TimestampParser.createObjectTimestampParser(timestampFormat);
54 | this.missingValue = missingValue == null
55 | ? DEFAULT_MISSING_VALUE
56 | : missingValue;
57 | }
58 |
59 | @JsonProperty("column")
60 | public String getTimestampColumn()
61 | {
62 | return timestampColumn;
63 | }
64 |
65 | @JsonProperty("format")
66 | public String getTimestampFormat()
67 | {
68 | return timestampFormat;
69 | }
70 |
71 | @JsonProperty("missingValue")
72 | public DateTime getMissingValue()
73 | {
74 | return missingValue;
75 | }
76 |
77 | public DateTime extractTimestamp(Map input)
78 | {
79 | final Object o = input.get(timestampColumn);
80 |
81 | return o == null ? missingValue : timestampConverter.apply(o);
82 | }
83 |
84 | @Override
85 | public boolean equals(Object o)
86 | {
87 | if (this == o) {
88 | return true;
89 | }
90 | if (o == null || getClass() != o.getClass()) {
91 | return false;
92 | }
93 |
94 | TimestampSpec that = (TimestampSpec) o;
95 |
96 | if (!timestampColumn.equals(that.timestampColumn)) {
97 | return false;
98 | }
99 | if (!timestampFormat.equals(that.timestampFormat)) {
100 | return false;
101 | }
102 | return !(missingValue != null ? !missingValue.equals(that.missingValue) : that.missingValue != null);
103 |
104 | }
105 |
106 | @Override
107 | public int hashCode()
108 | {
109 | int result = timestampColumn.hashCode();
110 | result = 31 * result + timestampFormat.hashCode();
111 | result = 31 * result + (missingValue != null ? missingValue.hashCode() : 0);
112 | return result;
113 | }
114 | }
115 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/Binders.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.inject.Binder;
23 | import com.google.inject.Key;
24 | import com.google.inject.multibindings.MapBinder;
25 | import io.druid.segment.loading.DataSegmentArchiver;
26 | import io.druid.segment.loading.DataSegmentFinder;
27 | import io.druid.segment.loading.DataSegmentMover;
28 | import io.druid.segment.loading.DataSegmentKiller;
29 | import io.druid.segment.loading.DataSegmentPuller;
30 | import io.druid.segment.loading.DataSegmentPusher;
31 | import io.druid.tasklogs.TaskLogs;
32 |
33 | /**
34 | */
35 | public class Binders
36 | {
37 | public static MapBinder dataSegmentPullerBinder(Binder binder)
38 | {
39 | return MapBinder.newMapBinder(binder, String.class, DataSegmentPuller.class);
40 | }
41 |
42 | public static MapBinder dataSegmentKillerBinder(Binder binder)
43 | {
44 | return MapBinder.newMapBinder(binder, String.class, DataSegmentKiller.class);
45 | }
46 |
47 | public static MapBinder dataSegmentMoverBinder(Binder binder)
48 | {
49 | return MapBinder.newMapBinder(binder, String.class, DataSegmentMover.class);
50 | }
51 |
52 | public static MapBinder dataSegmentArchiverBinder(Binder binder)
53 | {
54 | return MapBinder.newMapBinder(binder, String.class, DataSegmentArchiver.class);
55 | }
56 |
57 | public static MapBinder dataSegmentPusherBinder(Binder binder)
58 | {
59 | return PolyBind.optionBinder(binder, Key.get(DataSegmentPusher.class));
60 | }
61 |
62 | public static MapBinder dataSegmentFinderBinder(Binder binder)
63 | {
64 | return PolyBind.optionBinder(binder, Key.get(DataSegmentFinder.class));
65 | }
66 |
67 | public static MapBinder taskLogsBinder(Binder binder)
68 | {
69 | return PolyBind.optionBinder(binder, Key.get(TaskLogs.class));
70 | }
71 | }
72 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/DruidGuiceExtensions.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.inject.Binder;
23 | import com.google.inject.Module;
24 |
25 | /**
26 | */
27 | public class DruidGuiceExtensions implements Module
28 | {
29 | @Override
30 | public void configure(Binder binder)
31 | {
32 | binder.bindScope(LazySingleton.class, DruidScopes.SINGLETON);
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/DruidScopes.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.inject.Key;
23 | import com.google.inject.Provider;
24 | import com.google.inject.Scope;
25 | import com.google.inject.Scopes;
26 |
27 | /**
28 | */
29 | public class DruidScopes
30 | {
31 | public static final Scope SINGLETON = new Scope()
32 | {
33 | @Override
34 | public Provider scope(Key key, Provider unscoped)
35 | {
36 | return Scopes.SINGLETON.scope(key, unscoped);
37 | }
38 |
39 | @Override
40 | public String toString()
41 | {
42 | return "DruidScopes.SINGLETON";
43 | }
44 | };
45 | }
46 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/Jerseys.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.inject.Binder;
23 | import com.google.inject.TypeLiteral;
24 | import com.google.inject.multibindings.Multibinder;
25 | import io.druid.guice.annotations.JSR311Resource;
26 |
27 | /**
28 | */
29 | public class Jerseys
30 | {
31 | public static void addResource(Binder binder, Class> resourceClazz)
32 | {
33 | Multibinder.newSetBinder(binder, new TypeLiteral>(){}, JSR311Resource.class)
34 | .addBinding()
35 | .toInstance(resourceClazz);
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/KeyHolder.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.inject.Key;
23 |
24 | /**
25 | */
26 | public class KeyHolder
27 | {
28 | private final Key extends T> key;
29 |
30 | public KeyHolder(
31 | Key extends T> key
32 | )
33 | {
34 | this.key = key;
35 | }
36 |
37 | public Key extends T> getKey()
38 | {
39 | return key;
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/LazySingleton.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.inject.ScopeAnnotation;
23 |
24 | import java.lang.annotation.ElementType;
25 | import java.lang.annotation.Retention;
26 | import java.lang.annotation.Target;
27 |
28 | import static java.lang.annotation.RetentionPolicy.RUNTIME;
29 |
30 | /**
31 | */
32 | @Target({ ElementType.TYPE, ElementType.METHOD })
33 | @Retention(RUNTIME)
34 | @ScopeAnnotation
35 | public @interface LazySingleton
36 | {
37 | }
38 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/LifecycleScope.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.common.collect.Lists;
23 | import com.google.inject.Key;
24 | import com.google.inject.Provider;
25 | import com.google.inject.Scope;
26 | import com.metamx.common.lifecycle.Lifecycle;
27 | import com.metamx.common.logger.Logger;
28 |
29 | import java.util.List;
30 |
31 | /**
32 | * A scope that adds objects to the Lifecycle. This is by definition also a lazy singleton scope.
33 | */
34 | public class LifecycleScope implements Scope
35 | {
36 | private static final Logger log = new Logger(LifecycleScope.class);
37 | private final Lifecycle.Stage stage;
38 |
39 | private Lifecycle lifecycle;
40 | private List instances = Lists.newLinkedList();
41 |
42 | public LifecycleScope(Lifecycle.Stage stage)
43 | {
44 | this.stage = stage;
45 | }
46 |
47 | public void setLifecycle(Lifecycle lifecycle)
48 | {
49 | this.lifecycle = lifecycle;
50 | synchronized (instances) {
51 | for (Object instance : instances) {
52 | lifecycle.addManagedInstance(instance);
53 | }
54 | }
55 | }
56 |
57 | @Override
58 | public Provider scope(final Key key, final Provider unscoped)
59 | {
60 | return new Provider()
61 | {
62 | private volatile T value = null;
63 |
64 | @Override
65 | public synchronized T get()
66 | {
67 | if (value == null) {
68 | final T retVal = unscoped.get();
69 |
70 | synchronized (instances) {
71 | if (lifecycle == null) {
72 | instances.add(retVal);
73 | }
74 | else {
75 | try {
76 | lifecycle.addMaybeStartManagedInstance(retVal, stage);
77 | }
78 | catch (Exception e) {
79 | log.warn(e, "Caught exception when trying to create a[%s]", key);
80 | return null;
81 | }
82 | }
83 | }
84 |
85 | value = retVal;
86 | }
87 |
88 | return value;
89 | }
90 | };
91 | }
92 | }
93 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/ManageLifecycle.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.inject.ScopeAnnotation;
23 |
24 | import java.lang.annotation.ElementType;
25 | import java.lang.annotation.Retention;
26 | import java.lang.annotation.Target;
27 |
28 | import static java.lang.annotation.RetentionPolicy.RUNTIME;
29 |
30 | /**
31 | * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle}
32 | *
33 | * This Scope gets defined by {@link io.druid.guice.LifecycleModule}
34 | */
35 | @Target({ ElementType.TYPE, ElementType.METHOD })
36 | @Retention(RUNTIME)
37 | @ScopeAnnotation
38 | public @interface ManageLifecycle
39 | {
40 | }
41 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/ManageLifecycleLast.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.inject.ScopeAnnotation;
23 |
24 | import java.lang.annotation.ElementType;
25 | import java.lang.annotation.Retention;
26 | import java.lang.annotation.Target;
27 |
28 | import static java.lang.annotation.RetentionPolicy.RUNTIME;
29 |
30 | /**
31 | * Marks the object to be managed by {@link com.metamx.common.lifecycle.Lifecycle} and set to be on Stage.LAST
32 | *
33 | * This Scope gets defined by {@link io.druid.guice.LifecycleModule}
34 | */
35 | @Target({ ElementType.TYPE, ElementType.METHOD })
36 | @Retention(RUNTIME)
37 | @ScopeAnnotation
38 | public @interface ManageLifecycleLast
39 | {
40 | }
41 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/SupplierProvider.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.common.base.Supplier;
23 | import com.google.inject.Inject;
24 | import com.google.inject.Injector;
25 | import com.google.inject.Key;
26 | import com.google.inject.Provider;
27 |
28 | /**
29 | */
30 | public class SupplierProvider implements Provider
31 | {
32 | private final Key> supplierKey;
33 |
34 | private Provider> supplierProvider;
35 |
36 | public SupplierProvider(
37 | Key> supplierKey
38 | )
39 | {
40 | this.supplierKey = supplierKey;
41 | }
42 |
43 | @Inject
44 | public void configure(Injector injector)
45 | {
46 | this.supplierProvider = injector.getProvider(supplierKey);
47 | }
48 |
49 | @Override
50 | public T get()
51 | {
52 | return supplierProvider.get().get();
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/annotations/Global.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice.annotations;
21 |
22 | import com.google.inject.BindingAnnotation;
23 |
24 | import java.lang.annotation.ElementType;
25 | import java.lang.annotation.Retention;
26 | import java.lang.annotation.RetentionPolicy;
27 | import java.lang.annotation.Target;
28 |
29 | /**
30 | */
31 | @BindingAnnotation
32 | @Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
33 | @Retention(RetentionPolicy.RUNTIME)
34 | public @interface Global
35 | {
36 | }
37 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/annotations/JSR311Resource.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice.annotations;
21 |
22 | import com.google.inject.BindingAnnotation;
23 |
24 | import java.lang.annotation.ElementType;
25 | import java.lang.annotation.Retention;
26 | import java.lang.annotation.RetentionPolicy;
27 | import java.lang.annotation.Target;
28 |
29 | /**
30 | */
31 | @BindingAnnotation
32 | @Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
33 | @Retention(RetentionPolicy.RUNTIME)
34 | public @interface JSR311Resource
35 | {
36 | }
37 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/annotations/Json.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice.annotations;
21 |
22 | import com.google.inject.BindingAnnotation;
23 |
24 | import java.lang.annotation.ElementType;
25 | import java.lang.annotation.Retention;
26 | import java.lang.annotation.RetentionPolicy;
27 | import java.lang.annotation.Target;
28 |
29 | /**
30 | */
31 | @Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
32 | @Retention(RetentionPolicy.RUNTIME)
33 | @BindingAnnotation
34 | public @interface Json
35 | {
36 | }
37 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/annotations/Self.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice.annotations;
21 |
22 | import com.google.inject.BindingAnnotation;
23 |
24 | import java.lang.annotation.ElementType;
25 | import java.lang.annotation.Retention;
26 | import java.lang.annotation.RetentionPolicy;
27 | import java.lang.annotation.Target;
28 |
29 | /**
30 | */
31 | @Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
32 | @Retention(RetentionPolicy.RUNTIME)
33 | @BindingAnnotation
34 | public @interface Self
35 | {
36 | }
37 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/guice/annotations/Smile.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice.annotations;
21 |
22 | import com.google.inject.BindingAnnotation;
23 |
24 | import java.lang.annotation.ElementType;
25 | import java.lang.annotation.Retention;
26 | import java.lang.annotation.RetentionPolicy;
27 | import java.lang.annotation.Target;
28 |
29 | /**
30 | */
31 | @Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
32 | @Retention(RetentionPolicy.RUNTIME)
33 | @BindingAnnotation
34 | public @interface Smile
35 | {
36 | }
37 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/initialization/DruidModule.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.initialization;
21 |
22 | import com.fasterxml.jackson.databind.Module;
23 |
24 | import java.util.List;
25 |
26 | /**
27 | */
28 | public interface DruidModule extends com.google.inject.Module
29 | {
30 | public List extends Module> getJacksonModules();
31 | }
32 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/jackson/CommaListJoinDeserializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.jackson;
21 |
22 | import com.fasterxml.jackson.core.JsonParser;
23 | import com.fasterxml.jackson.core.JsonProcessingException;
24 | import com.fasterxml.jackson.databind.DeserializationContext;
25 | import com.fasterxml.jackson.databind.deser.std.StdScalarDeserializer;
26 |
27 | import java.io.IOException;
28 | import java.util.Arrays;
29 | import java.util.List;
30 |
31 | /**
32 | */
33 | public class CommaListJoinDeserializer extends StdScalarDeserializer>
34 | {
35 | protected CommaListJoinDeserializer()
36 | {
37 | super(List.class);
38 | }
39 |
40 | @Override
41 | public List deserialize(JsonParser jsonParser, DeserializationContext deserializationContext)
42 | throws IOException, JsonProcessingException
43 | {
44 | return Arrays.asList(jsonParser.getText().split(","));
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/jackson/CommaListJoinSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.jackson;
21 |
22 | import com.fasterxml.jackson.core.JsonGenerationException;
23 | import com.fasterxml.jackson.core.JsonGenerator;
24 | import com.fasterxml.jackson.databind.SerializerProvider;
25 | import com.fasterxml.jackson.databind.ser.std.StdScalarSerializer;
26 | import com.google.common.base.Joiner;
27 |
28 | import java.io.IOException;
29 | import java.util.List;
30 |
31 | /**
32 | */
33 | public class CommaListJoinSerializer extends StdScalarSerializer>
34 | {
35 | private static final Joiner joiner = Joiner.on(",");
36 |
37 | protected CommaListJoinSerializer()
38 | {
39 | super(List.class, true);
40 | }
41 |
42 | @Override
43 | public void serialize(List value, JsonGenerator jgen, SerializerProvider provider)
44 | throws IOException, JsonGenerationException
45 | {
46 | jgen.writeString(joiner.join(value));
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/query/SegmentDescriptor.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.query;
21 |
22 | import com.fasterxml.jackson.annotation.JsonCreator;
23 | import com.fasterxml.jackson.annotation.JsonProperty;
24 | import org.joda.time.Interval;
25 |
26 | /**
27 | */
28 | public class SegmentDescriptor
29 | {
30 | private final Interval interval;
31 | private final String version;
32 | private final int partitionNumber;
33 |
34 | @JsonCreator
35 | public SegmentDescriptor(
36 | @JsonProperty("itvl") Interval interval,
37 | @JsonProperty("ver") String version,
38 | @JsonProperty("part") int partitionNumber
39 | )
40 | {
41 | this.interval = interval;
42 | this.version = version;
43 | this.partitionNumber = partitionNumber;
44 | }
45 |
46 | @JsonProperty("itvl")
47 | public Interval getInterval()
48 | {
49 | return interval;
50 | }
51 |
52 | @JsonProperty("ver")
53 | public String getVersion()
54 | {
55 | return version;
56 | }
57 |
58 | @JsonProperty("part")
59 | public int getPartitionNumber()
60 | {
61 | return partitionNumber;
62 | }
63 |
64 | @Override
65 | public boolean equals(Object o)
66 | {
67 | if (this == o) {
68 | return true;
69 | }
70 | if (o == null || getClass() != o.getClass()) {
71 | return false;
72 | }
73 |
74 | SegmentDescriptor that = (SegmentDescriptor) o;
75 |
76 | if (partitionNumber != that.partitionNumber) {
77 | return false;
78 | }
79 | if (interval != null ? !interval.equals(that.interval) : that.interval != null) {
80 | return false;
81 | }
82 | if (version != null ? !version.equals(that.version) : that.version != null) {
83 | return false;
84 | }
85 |
86 | return true;
87 | }
88 |
89 | @Override
90 | public int hashCode()
91 | {
92 | int result = interval != null ? interval.hashCode() : 0;
93 | result = 31 * result + (version != null ? version.hashCode() : 0);
94 | result = 31 * result + partitionNumber;
95 | return result;
96 | }
97 |
98 | @Override
99 | public String toString()
100 | {
101 | return "SegmentDescriptor{" +
102 | "interval=" + interval +
103 | ", version='" + version + '\'' +
104 | ", partitionNumber=" + partitionNumber +
105 | '}';
106 | }
107 | }
108 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/SegmentUtils.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment;
21 |
22 | import com.google.common.io.Files;
23 | import com.google.common.primitives.Ints;
24 |
25 | import java.io.File;
26 | import java.io.FileInputStream;
27 | import java.io.IOException;
28 | import java.io.InputStream;
29 |
30 | /**
31 | */
32 | public class SegmentUtils
33 | {
34 | public static int getVersionFromDir(File inDir) throws IOException
35 | {
36 | File versionFile = new File(inDir, "version.bin");
37 | if (versionFile.exists()) {
38 | return Ints.fromByteArray(Files.toByteArray(versionFile));
39 | }
40 |
41 | final File indexFile = new File(inDir, "index.drd");
42 | int version;
43 | try (InputStream in = new FileInputStream(indexFile)) {
44 | version = in.read();
45 | }
46 | return version;
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/DataSegmentArchiver.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import io.druid.timeline.DataSegment;
23 |
24 | public interface DataSegmentArchiver
25 | {
26 | public DataSegment archive(DataSegment segment) throws SegmentLoadingException;
27 | public DataSegment restore(DataSegment segment) throws SegmentLoadingException;
28 | }
29 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/DataSegmentFinder.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import io.druid.timeline.DataSegment;
23 |
24 | import java.util.Set;
25 |
26 | /**
27 | * A DataSegmentFinder is responsible for finding Druid segments underneath a specified directory and optionally updates
28 | * all descriptor.json files on deep storage with correct loadSpec.
29 | */
30 | public interface DataSegmentFinder
31 | {
32 | /**
33 | * This method should first recursively look for descriptor.json underneath workingDirPath and then verify that
34 | * index.zip exists in the same folder. If not, it should throw SegmentLoadingException to let the caller know that
35 | * descriptor.json exists while index.zip doesn't. If a segment is found and updateDescriptor is set, then this method
36 | * should update the loadSpec in descriptor.json to reflect the location from where it was found. After the search,
37 | * this method should return the set of segments that were found.
38 | *
39 | * @param workingDirPath the String representation of the working directory path
40 | * @param updateDescriptor if true, update loadSpec in descriptor.json if loadSpec's location is different from where
41 | * desciptor.json was found
42 | *
43 | * @return a set of segments that were found underneath workingDirPath
44 | */
45 | Set findSegments(String workingDirPath, boolean updateDescriptor) throws SegmentLoadingException;
46 | }
47 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/DataSegmentKiller.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import io.druid.timeline.DataSegment;
23 |
24 | /**
25 | */
26 | public interface DataSegmentKiller
27 | {
28 | public void kill(DataSegment segments) throws SegmentLoadingException;
29 | }
30 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/DataSegmentMover.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import io.druid.timeline.DataSegment;
23 |
24 | import java.util.Map;
25 |
26 | public interface DataSegmentMover
27 | {
28 | public DataSegment move(DataSegment segment, Map targetLoadSpec) throws SegmentLoadingException;
29 | }
30 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/DataSegmentPuller.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import io.druid.timeline.DataSegment;
23 |
24 | import java.io.File;
25 |
26 | /**
27 | * A DataSegmentPuller is responsible for pulling data for a particular segment into a particular directory
28 | */
29 | public interface DataSegmentPuller
30 | {
31 | /**
32 | * Pull down segment files for the given DataSegment and put them in the given directory.
33 | *
34 | * @param segment The segment to pull down files for
35 | * @param dir The directory to store the files in
36 | *
37 | * @throws SegmentLoadingException if there are any errors
38 | */
39 | public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException;
40 | }
41 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/DataSegmentPusher.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import io.druid.timeline.DataSegment;
23 |
24 | import java.io.File;
25 | import java.io.IOException;
26 |
27 | public interface DataSegmentPusher
28 | {
29 | public String getPathForHadoop(String dataSource);
30 | public DataSegment push(File file, DataSegment segment) throws IOException;
31 | }
32 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/DataSegmentPusherUtil.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import com.google.common.base.Joiner;
23 | import io.druid.timeline.DataSegment;
24 | import org.joda.time.format.ISODateTimeFormat;
25 |
26 | /**
27 | */
28 | public class DataSegmentPusherUtil
29 | {
30 | private static final Joiner JOINER = Joiner.on("/").skipNulls();
31 |
32 | // Note: storage directory structure format = .../dataSource/interval/version/partitionNumber/
33 | // If above format is ever changed, make sure to change it appropriately in other places
34 | // e.g. HDFSDataSegmentKiller uses this information to clean the version, interval and dataSource directories
35 | // on segment deletion if segment being deleted was the only segment
36 | public static String getStorageDir(DataSegment segment)
37 | {
38 | return JOINER.join(
39 | segment.getDataSource(),
40 | String.format(
41 | "%s_%s",
42 | segment.getInterval().getStart(),
43 | segment.getInterval().getEnd()
44 | ),
45 | segment.getVersion(),
46 | segment.getShardSpec().getPartitionNum()
47 | );
48 | }
49 |
50 | /**
51 | * Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in
52 | * path names. So we format paths differently for HDFS.
53 | */
54 | public static String getHdfsStorageDir(DataSegment segment)
55 | {
56 | return JOINER.join(
57 | segment.getDataSource(),
58 | String.format(
59 | "%s_%s",
60 | segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()),
61 | segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime())
62 | ),
63 | segment.getVersion().replaceAll(":", "_"),
64 | segment.getShardSpec().getPartitionNum()
65 | );
66 | }
67 | }
68 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/LoadSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import com.fasterxml.jackson.annotation.JsonTypeInfo;
23 |
24 | import java.io.File;
25 |
26 | /**
27 | * A means of pulling segment files into a destination directory
28 | */
29 | @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
30 | public interface LoadSpec
31 | {
32 | /**
33 | * Method should put the segment files in the directory passed
34 | * @param destDir The destination directory
35 | * @return The byte count of data put in the destination directory
36 | */
37 | public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException;
38 |
39 | // Hold interesting data about the results of the segment load
40 | public static class LoadSpecResult{
41 | private final long size;
42 | public LoadSpecResult(long size){
43 | this.size = size;
44 | }
45 | public long getSize(){
46 | return this.size;
47 | }
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/SegmentLoadingException.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | /**
23 | */
24 | public class SegmentLoadingException extends Exception
25 | {
26 | public SegmentLoadingException(
27 | String formatString,
28 | Object... objs
29 | )
30 | {
31 | super(String.format(formatString, objs));
32 | }
33 |
34 | public SegmentLoadingException(
35 | Throwable cause,
36 | String formatString,
37 | Object... objs
38 | )
39 | {
40 | super(String.format(formatString, objs), cause);
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/segment/loading/URIDataPuller.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import com.google.common.base.Predicate;
23 |
24 | import java.io.IOException;
25 | import java.io.InputStream;
26 | import java.net.URI;
27 |
28 | /**
29 | * A URIDataPuller has handlings for URI based data
30 | */
31 | public interface URIDataPuller
32 | {
33 | /**
34 | * Create a new InputStream based on the URI
35 | *
36 | * @param uri The URI to open an Input Stream to
37 | *
38 | * @return A new InputStream which streams the URI in question
39 | *
40 | * @throws IOException
41 | */
42 | public InputStream getInputStream(URI uri) throws IOException;
43 |
44 | /**
45 | * Returns an abstract "version" for the URI. The exact meaning of the version is left up to the implementation.
46 | *
47 | * @param uri The URI to check
48 | *
49 | * @return A "version" as interpreted by the URIDataPuller implementation
50 | *
51 | * @throws IOException on error
52 | */
53 | public String getVersion(URI uri) throws IOException;
54 |
55 | /**
56 | * Evaluates a Throwable to see if it is recoverable. This is expected to be used in conjunction with the other methods
57 | * to determine if anything thrown from the method should be retried.
58 | *
59 | * @return Predicate function indicating if the Throwable is recoverable
60 | */
61 | public Predicate shouldRetryPredicate();
62 | }
63 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/tasklogs/NoopTaskLogs.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.tasklogs;
21 |
22 | import com.google.common.base.Optional;
23 | import com.google.common.io.ByteSource;
24 | import com.google.common.io.InputSupplier;
25 | import com.metamx.common.logger.Logger;
26 |
27 | import java.io.File;
28 | import java.io.IOException;
29 | import java.io.InputStream;
30 |
31 | public class NoopTaskLogs implements TaskLogs
32 | {
33 | private final Logger log = new Logger(TaskLogs.class);
34 |
35 | @Override
36 | public Optional streamTaskLog(String taskid, long offset) throws IOException
37 | {
38 | return Optional.absent();
39 | }
40 |
41 | @Override
42 | public void pushTaskLog(String taskid, File logFile) throws IOException
43 | {
44 | log.info("Not pushing logs for task: %s", taskid);
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/tasklogs/TaskLogPusher.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.tasklogs;
21 |
22 | import java.io.File;
23 | import java.io.IOException;
24 |
25 | /**
26 | * Something that knows how to persist local task logs to some form of long-term storage.
27 | */
28 | public interface TaskLogPusher
29 | {
30 | public void pushTaskLog(String taskid, File logFile) throws IOException;
31 | }
32 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/tasklogs/TaskLogStreamer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.tasklogs;
21 |
22 | import com.google.common.base.Optional;
23 | import com.google.common.io.ByteSource;
24 | import com.google.common.io.InputSupplier;
25 |
26 | import java.io.IOException;
27 | import java.io.InputStream;
28 |
29 | /**
30 | * Something that knows how to stream logs for tasks.
31 | */
32 | public interface TaskLogStreamer
33 | {
34 | /**
35 | * Stream log for a task.
36 | *
37 | * @param offset If zero, stream the entire log. If positive, attempt to read from this position onwards. If
38 | * negative, attempt to read this many bytes from the end of the file (like tail -n ).
39 | *
40 | * @return input supplier for this log, if available from this provider
41 | */
42 | public Optional streamTaskLog(String taskid, long offset) throws IOException;
43 | }
44 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/tasklogs/TaskLogs.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.tasklogs;
21 |
22 | public interface TaskLogs extends TaskLogStreamer, TaskLogPusher
23 | {
24 | }
25 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/timeline/partition/NoneShardSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.timeline.partition;
21 |
22 | import com.google.common.base.Preconditions;
23 | import com.google.common.collect.Iterables;
24 | import com.metamx.common.ISE;
25 | import io.druid.data.input.InputRow;
26 |
27 | import java.util.Collections;
28 | import java.util.List;
29 | import java.util.Set;
30 |
31 | /**
32 | */
33 | public class NoneShardSpec implements ShardSpec
34 | {
35 | @Override
36 | public PartitionChunk createChunk(T obj)
37 | {
38 | return new SingleElementPartitionChunk(obj);
39 | }
40 |
41 | @Override
42 | public boolean isInChunk(long timestamp, InputRow inputRow)
43 | {
44 | return true;
45 | }
46 |
47 | @Override
48 | public int getPartitionNum()
49 | {
50 | return 0;
51 | }
52 |
53 | @Override
54 | public ShardSpecLookup getLookup(final List shardSpecs)
55 | {
56 |
57 | return new ShardSpecLookup()
58 | {
59 | @Override
60 | public ShardSpec getShardSpec(long timestamp, InputRow row)
61 | {
62 | return shardSpecs.get(0);
63 | }
64 | };
65 | }
66 |
67 | @Override
68 | public boolean equals(Object obj)
69 | {
70 | return obj instanceof NoneShardSpec;
71 | }
72 |
73 | @Override
74 | public int hashCode()
75 | {
76 | return 0;
77 | }
78 |
79 | @Override
80 | public String toString()
81 | {
82 | return "NoneShardSpec";
83 | }
84 | }
85 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/timeline/partition/PartitionChunk.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.timeline.partition;
21 |
22 | /**
23 | * A PartitionChunk represents a chunk of a partitioned(sharded) space. It has knowledge of whether it is
24 | * the start of the domain of partitions, the end of the domain, if it abuts another partition and where it stands
25 | * inside of a sorted collection of partitions.
26 | *
27 | * The ordering of PartitionChunks is based entirely upon the partition boundaries defined inside the concrete
28 | * PartitionChunk class. That is, the payload (the object returned by getObject()) should *not* be involved in
29 | * comparisons between PartitionChunk objects.
30 | */
31 | public interface PartitionChunk extends Comparable>
32 | {
33 | /**
34 | * Returns the payload, generally an object that can be used to perform some action against the shard.
35 | *
36 | * @return the payload
37 | */
38 | public T getObject();
39 |
40 | /**
41 | * Determines if this PartitionChunk abuts another PartitionChunk. A sequence of abutting PartitionChunks should
42 | * start with an object where isStart() == true and eventually end with an object where isEnd() == true.
43 | *
44 | * @param chunk input chunk
45 | * @return true if this chunk abuts the input chunk
46 | */
47 | public boolean abuts(PartitionChunk chunk);
48 |
49 | /**
50 | * Returns true if this chunk is the beginning of the partition. Most commonly, that means it represents the range
51 | * [-infinity, X) for some concrete X.
52 | *
53 | * @return true if the chunk is the beginning of the partition
54 | */
55 | public boolean isStart();
56 |
57 | /**
58 | * Returns true if this chunk is the end of the partition. Most commonly, that means it represents the range
59 | * [X, infinity] for some concrete X.
60 | *
61 | * @return true if the chunk is the beginning of the partition
62 | */
63 | public boolean isEnd();
64 |
65 | /**
66 | * Returns the partition chunk number of this PartitionChunk. I.e. if there are 4 partitions in total and this
67 | * is the 3rd partition, it will return 2
68 | *
69 | * @return the sequential numerical id of this partition chunk
70 | */
71 | public int getChunkNumber();
72 | }
73 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/timeline/partition/ShardSpec.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.timeline.partition;
21 |
22 | import com.fasterxml.jackson.annotation.JsonSubTypes;
23 | import com.fasterxml.jackson.annotation.JsonTypeInfo;
24 | import io.druid.data.input.InputRow;
25 |
26 | import java.util.List;
27 |
28 | /**
29 | * A Marker interface that exists to combine ShardSpec objects together for Jackson
30 | */
31 | @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
32 | @JsonSubTypes({
33 | @JsonSubTypes.Type(name = "none", value = NoneShardSpec.class),
34 | })
35 | public interface ShardSpec
36 | {
37 | public PartitionChunk createChunk(T obj);
38 |
39 | public boolean isInChunk(long timestamp, InputRow inputRow);
40 |
41 | public int getPartitionNum();
42 |
43 | public ShardSpecLookup getLookup(List shardSpecs);
44 | }
45 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/timeline/partition/ShardSpecLookup.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.timeline.partition;
21 |
22 | import io.druid.data.input.InputRow;
23 |
24 | public interface ShardSpecLookup
25 | {
26 | ShardSpec getShardSpec(long timestamp, InputRow row);
27 | }
28 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/timeline/partition/SingleElementPartitionChunk.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.timeline.partition;
21 |
22 | /**
23 | */
24 | public class SingleElementPartitionChunk implements PartitionChunk
25 | {
26 | private final T element;
27 |
28 | public SingleElementPartitionChunk
29 | (
30 | T element
31 | )
32 | {
33 | this.element = element;
34 | }
35 |
36 | @Override
37 | public T getObject()
38 | {
39 | return element;
40 | }
41 |
42 | @Override
43 | public boolean abuts(PartitionChunk tPartitionChunk)
44 | {
45 | return false;
46 | }
47 |
48 | @Override
49 | public boolean isStart()
50 | {
51 | return true;
52 | }
53 |
54 | @Override
55 | public boolean isEnd()
56 | {
57 | return true;
58 | }
59 |
60 | @Override
61 | public int getChunkNumber()
62 | {
63 | return 0;
64 | }
65 |
66 | /**
67 | * The ordering of PartitionChunks is determined entirely by the partition boundaries and has nothing to do
68 | * with the object. Thus, if there are two SingleElementPartitionChunks, they are equal because they both
69 | * represent the full partition space.
70 | *
71 | * SingleElementPartitionChunks are currently defined as less than every other type of PartitionChunk. There
72 | * is no good reason for it, nor is there a bad reason, that's just the way it is. This is subject to change.
73 | *
74 | * @param chunk
75 | * @return
76 | */
77 | @Override
78 | public int compareTo(PartitionChunk chunk)
79 | {
80 | return chunk instanceof SingleElementPartitionChunk ? 0 : -1;
81 | }
82 |
83 | @Override
84 | public boolean equals(Object o)
85 | {
86 | if (this == o) {
87 | return true;
88 | }
89 | if (o == null || getClass() != o.getClass()) {
90 | return false;
91 | }
92 |
93 | return true;
94 | }
95 |
96 | @Override
97 | public int hashCode()
98 | {
99 | return element != null ? element.hashCode() : 0;
100 | }
101 |
102 | @Override
103 | public String toString()
104 | {
105 | return "SingleElementPartitionChunk{" +
106 | "element=" + element +
107 | '}';
108 | }
109 | }
110 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/utils/CompressionUtils.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.utils;
21 |
22 | import com.metamx.common.logger.Logger;
23 |
24 | import java.io.File;
25 | import java.io.IOException;
26 | import java.io.InputStream;
27 | import java.io.OutputStream;
28 |
29 | /**
30 | */
31 | public class CompressionUtils
32 | {
33 | private static final Logger log = new Logger(CompressionUtils.class);
34 |
35 |
36 | @Deprecated // Use com.metamx.common.CompressionUtils.zip
37 | public static long zip(File directory, File outputZipFile) throws IOException
38 | {
39 | return com.metamx.common.CompressionUtils.zip(directory, outputZipFile);
40 | }
41 |
42 |
43 | @Deprecated // Use com.metamx.common.CompressionUtils.zip
44 | public static long zip(File directory, OutputStream out) throws IOException
45 | {
46 | return com.metamx.common.CompressionUtils.zip(directory, out);
47 | }
48 |
49 | @Deprecated // Use com.metamx.common.CompressionUtils.unzip
50 | public static void unzip(File pulledFile, File outDir) throws IOException
51 | {
52 | com.metamx.common.CompressionUtils.unzip(pulledFile, outDir);
53 | }
54 |
55 | @Deprecated // Use com.metamx.common.CompressionUtils.unzip
56 | public static void unzip(InputStream in, File outDir) throws IOException
57 | {
58 | com.metamx.common.CompressionUtils.unzip(in, outDir);
59 | }
60 |
61 | /**
62 | * Uncompress using a gzip uncompress algorithm from the `pulledFile` to the `outDir`.
63 | * Unlike `com.metamx.common.CompressionUtils.gunzip`, this function takes an output *DIRECTORY* and tries to guess the file name.
64 | * It is recommended that the caller use `com.metamx.common.CompressionUtils.gunzip` and specify the output file themselves to ensure names are as expected
65 | *
66 | * @param pulledFile The source file
67 | * @param outDir The destination directory to put the resulting file
68 | *
69 | * @throws IOException on propogated IO exception, IAE if it cannot determine the proper new name for `pulledFile`
70 | */
71 | @Deprecated // See description for alternative
72 | public static void gunzip(File pulledFile, File outDir) throws IOException
73 | {
74 | final File outFile = new File(outDir, com.metamx.common.CompressionUtils.getGzBaseName(pulledFile.getName()));
75 | com.metamx.common.CompressionUtils.gunzip(pulledFile, outFile);
76 | if (!pulledFile.delete()) {
77 | log.error("Could not delete tmpFile[%s].", pulledFile);
78 | }
79 | }
80 |
81 | }
82 |
--------------------------------------------------------------------------------
/src/main/java/io/druid/utils/Runnables.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.utils;
21 |
22 | /**
23 | */
24 | public class Runnables
25 | {
26 | public static Runnable getNoopRunnable(){
27 | return new Runnable(){
28 | public void run(){}
29 | };
30 | }
31 | }
32 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/TestObjectMapper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid;
21 |
22 | import com.fasterxml.jackson.core.JsonParser;
23 | import com.fasterxml.jackson.core.JsonProcessingException;
24 | import com.fasterxml.jackson.databind.DeserializationContext;
25 | import com.fasterxml.jackson.databind.DeserializationFeature;
26 | import com.fasterxml.jackson.databind.MapperFeature;
27 | import com.fasterxml.jackson.databind.ObjectMapper;
28 | import com.fasterxml.jackson.databind.SerializationFeature;
29 | import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
30 | import com.fasterxml.jackson.databind.module.SimpleModule;
31 | import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
32 | import org.joda.time.Interval;
33 |
34 | import java.io.IOException;
35 |
36 | /**
37 | */
38 | public class TestObjectMapper extends ObjectMapper
39 | {
40 | public TestObjectMapper()
41 | {
42 | configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
43 | configure(MapperFeature.AUTO_DETECT_GETTERS, false);
44 | configure(MapperFeature.AUTO_DETECT_FIELDS, false);
45 | configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false);
46 | configure(MapperFeature.AUTO_DETECT_SETTERS, false);
47 | configure(SerializationFeature.INDENT_OUTPUT, false);
48 | registerModule(new TestModule());
49 | }
50 |
51 | public static class TestModule extends SimpleModule
52 | {
53 | TestModule()
54 | {
55 | addSerializer(Interval.class, ToStringSerializer.instance);
56 | addDeserializer(
57 | Interval.class, new StdDeserializer(Interval.class)
58 | {
59 | @Override
60 | public Interval deserialize(
61 | JsonParser jsonParser, DeserializationContext deserializationContext
62 | ) throws IOException, JsonProcessingException
63 | {
64 | return new Interval(jsonParser.getText());
65 | }
66 | }
67 | );
68 | }
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/MapBasedRowTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input;
21 |
22 | import org.joda.time.DateTime;
23 | import org.junit.Assert;
24 | import org.junit.Test;
25 |
26 | import com.google.common.collect.ImmutableMap;
27 |
28 | public class MapBasedRowTest
29 | {
30 | @Test
31 | public void testGetLongMetricFromString()
32 | {
33 | MapBasedRow row = new MapBasedRow(
34 | new DateTime(),
35 | ImmutableMap.builder()
36 | .put("k0", "-1.2")
37 | .put("k1", "1.23")
38 | .put("k2", "1.8")
39 | .put("k3", "1e5")
40 | .put("k4", "9223372036854775806")
41 | .put("k5", "-9223372036854775807")
42 | .put("k6", "+9223372036854775802")
43 | .build()
44 | );
45 |
46 | Assert.assertEquals(-1, row.getLongMetric("k0"));
47 | Assert.assertEquals(1, row.getLongMetric("k1"));
48 | Assert.assertEquals(1, row.getLongMetric("k2"));
49 | Assert.assertEquals(100000, row.getLongMetric("k3"));
50 | Assert.assertEquals(9223372036854775806L, row.getLongMetric("k4"));
51 | Assert.assertEquals(-9223372036854775807L, row.getLongMetric("k5"));
52 | Assert.assertEquals(9223372036854775802L, row.getLongMetric("k6"));
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/CSVParseSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.google.common.collect.Lists;
23 | import com.metamx.common.parsers.ParseException;
24 | import org.junit.Test;
25 |
26 | import java.util.Arrays;
27 |
28 | public class CSVParseSpecTest
29 | {
30 | @Test(expected = IllegalArgumentException.class)
31 | public void testColumnMissing() throws Exception
32 | {
33 | final ParseSpec spec = new CSVParseSpec(
34 | new TimestampSpec(
35 | "timestamp",
36 | "auto",
37 | null
38 | ),
39 | new DimensionsSpec(
40 | DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b")),
41 | Lists.newArrayList(),
42 | Lists.newArrayList()
43 | ),
44 | ",",
45 | Arrays.asList("a")
46 | );
47 | }
48 |
49 | @Test(expected = IllegalArgumentException.class)
50 | public void testComma() throws Exception
51 | {
52 | final ParseSpec spec = new CSVParseSpec(
53 | new TimestampSpec(
54 | "timestamp",
55 | "auto",
56 | null
57 | ),
58 | new DimensionsSpec(
59 | DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b")),
60 | Lists.newArrayList(),
61 | Lists.newArrayList()
62 | ),
63 | ",",
64 | Arrays.asList("a")
65 | );
66 | }
67 | }
68 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/DelimitedParseSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.databind.ObjectMapper;
23 | import com.google.common.collect.Lists;
24 | import io.druid.TestObjectMapper;
25 | import org.junit.Assert;
26 | import org.junit.Test;
27 |
28 | import javax.validation.constraints.Null;
29 | import java.io.IOException;
30 | import java.util.Arrays;
31 |
32 | public class DelimitedParseSpecTest
33 | {
34 | private final ObjectMapper jsonMapper = new TestObjectMapper();
35 |
36 | @Test
37 | public void testSerde() throws IOException
38 | {
39 | DelimitedParseSpec spec = new DelimitedParseSpec(
40 | new TimestampSpec("abc", "iso", null),
41 | new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null),
42 | "\u0001",
43 | "\u0002",
44 | Arrays.asList("abc")
45 | );
46 | final DelimitedParseSpec serde = jsonMapper.readValue(
47 | jsonMapper.writeValueAsString(spec),
48 | DelimitedParseSpec.class
49 | );
50 | Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn());
51 | Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
52 |
53 | Assert.assertEquals(Arrays.asList("abc"), serde.getColumns());
54 | Assert.assertEquals("\u0001", serde.getDelimiter());
55 | Assert.assertEquals("\u0002", serde.getListDelimiter());
56 | Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames());
57 | }
58 |
59 | @Test(expected = IllegalArgumentException.class)
60 | public void testColumnMissing() throws Exception
61 | {
62 | final ParseSpec spec = new DelimitedParseSpec(
63 | new TimestampSpec(
64 | "timestamp",
65 | "auto",
66 | null
67 | ),
68 | new DimensionsSpec(
69 | DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b")),
70 | Lists.newArrayList(),
71 | Lists.newArrayList()
72 | ),
73 | ",",
74 | " ",
75 | Arrays.asList("a")
76 | );
77 | }
78 |
79 | @Test(expected = IllegalArgumentException.class)
80 | public void testComma() throws Exception
81 | {
82 | final ParseSpec spec = new DelimitedParseSpec(
83 | new TimestampSpec(
84 | "timestamp",
85 | "auto",
86 | null
87 | ),
88 | new DimensionsSpec(
89 | DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b")),
90 | Lists.newArrayList(),
91 | Lists.newArrayList()
92 | ),
93 | ",",
94 | null,
95 | Arrays.asList("a")
96 | );
97 | }
98 |
99 | @Test(expected = NullPointerException.class)
100 | public void testDefaultColumnList(){
101 | final DelimitedParseSpec spec = new DelimitedParseSpec(
102 | new TimestampSpec(
103 | "timestamp",
104 | "auto",
105 | null
106 | ),
107 | new DimensionsSpec(
108 | DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b")),
109 | Lists.newArrayList(),
110 | Lists.newArrayList()
111 | ),
112 | ",",
113 | null,
114 | // pass null columns not allowed
115 | null
116 | );
117 | }
118 | }
119 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/DimensionsSpecSerdeTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.databind.ObjectMapper;
23 | import junit.framework.Assert;
24 | import org.junit.Test;
25 |
26 | import java.util.Arrays;
27 | import java.util.List;
28 |
29 | /**
30 | */
31 | public class DimensionsSpecSerdeTest
32 | {
33 | private final ObjectMapper mapper = new ObjectMapper();
34 |
35 | @Test
36 | public void testDimensionsSpecSerde() throws Exception
37 | {
38 | DimensionsSpec expected = new DimensionsSpec(
39 | Arrays.asList(
40 | new StringDimensionSchema("AAA"),
41 | new StringDimensionSchema("BBB"),
42 | new FloatDimensionSchema("C++"),
43 | new NewSpatialDimensionSchema("DDT", null),
44 | new LongDimensionSchema("EEE"),
45 | new NewSpatialDimensionSchema("DDT2", Arrays.asList("A", "B")),
46 | new NewSpatialDimensionSchema("IMPR", Arrays.asList("S", "P", "Q", "R"))
47 | ),
48 | Arrays.asList("FOO", "HAR"),
49 | null
50 | );
51 |
52 | String jsonStr = "{\"dimensions\":"
53 | + "[\"AAA\", \"BBB\","
54 | + "{\"name\":\"C++\", \"type\":\"float\"},"
55 | + "{\"name\":\"DDT\", \"type\":\"spatial\"},"
56 | + "{\"name\":\"EEE\", \"type\":\"long\"},"
57 | + "{\"name\":\"DDT2\", \"type\": \"spatial\", \"dims\":[\"A\", \"B\"]}],"
58 | + "\"dimensionExclusions\": [\"FOO\", \"HAR\"],"
59 | + "\"spatialDimensions\": [{\"dimName\":\"IMPR\", \"dims\":[\"S\",\"P\",\"Q\",\"R\"]}]"
60 | + "}";
61 |
62 | DimensionsSpec actual = mapper.readValue(
63 | mapper.writeValueAsString(
64 | mapper.readValue(jsonStr, DimensionsSpec.class)
65 | ),
66 | DimensionsSpec.class
67 | );
68 |
69 | List expectedSpatials = Arrays.asList(
70 | new SpatialDimensionSchema("DDT", null),
71 | new SpatialDimensionSchema("DDT2", Arrays.asList("A","B")),
72 | new SpatialDimensionSchema("IMPR", Arrays.asList("S","P","Q","R"))
73 | );
74 |
75 | Assert.assertEquals(expected, actual);
76 | Assert.assertEquals(expectedSpatials, actual.getSpatialDimensions());
77 | }
78 | }
79 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/FileIteratingFirehoseTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.google.common.base.Function;
23 | import com.google.common.base.Joiner;
24 | import com.google.common.collect.ImmutableList;
25 | import com.google.common.collect.Lists;
26 | import com.metamx.common.Pair;
27 | import junit.framework.Assert;
28 | import org.apache.commons.io.LineIterator;
29 | import org.junit.Test;
30 |
31 | import java.io.StringReader;
32 | import java.util.Arrays;
33 | import java.util.List;
34 |
35 | public class FileIteratingFirehoseTest
36 | {
37 | private static final List>> fixtures = ImmutableList.of(
38 | Pair.of(new String[]{"2000,foo"}, ImmutableList.of("foo")),
39 | Pair.of(new String[]{"2000,foo\n2000,bar\n"}, ImmutableList.of("foo", "bar")),
40 | Pair.of(new String[]{"2000,foo\n2000,bar\n", "2000,baz"}, ImmutableList.of("foo", "bar", "baz")),
41 | Pair.of(new String[]{"2000,foo\n2000,bar\n", "", "2000,baz"}, ImmutableList.of("foo", "bar", "baz")),
42 | Pair.of(new String[]{"2000,foo\n2000,bar\n", "", "2000,baz", ""}, ImmutableList.of("foo", "bar", "baz")),
43 | Pair.of(new String[]{""}, ImmutableList.of()),
44 | Pair.of(new String[]{}, ImmutableList.of())
45 | );
46 |
47 | @Test
48 | public void testFirehose() throws Exception
49 | {
50 | for (Pair> fixture : fixtures) {
51 | final List lineIterators = Lists.transform(
52 | Arrays.asList(fixture.lhs),
53 | new Function()
54 | {
55 | @Override
56 | public LineIterator apply(String s)
57 | {
58 | return new LineIterator(new StringReader(s));
59 | }
60 | }
61 | );
62 |
63 | final StringInputRowParser parser = new StringInputRowParser(
64 | new CSVParseSpec(
65 | new TimestampSpec("ts", "auto", null),
66 | new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null),
67 | ",",
68 | ImmutableList.of("ts", "x")
69 | )
70 | );
71 |
72 | final FileIteratingFirehose firehose = new FileIteratingFirehose(lineIterators.iterator(), parser);
73 | final List results = Lists.newArrayList();
74 |
75 | while (firehose.hasMore()) {
76 | results.add(Joiner.on("|").join(firehose.nextRow().getDimension("x")));
77 | }
78 |
79 | Assert.assertEquals(fixture.rhs, results);
80 | }
81 | }
82 | }
83 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/JSONLowercaseParseSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.google.common.collect.Lists;
23 | import com.metamx.common.parsers.JSONToLowerParser;
24 | import com.metamx.common.parsers.Parser;
25 | import junit.framework.Assert;
26 | import org.junit.Test;
27 |
28 | import java.util.Arrays;
29 | import java.util.Map;
30 |
31 | public class JSONLowercaseParseSpecTest
32 | {
33 | @Test
34 | public void testLowercasing() throws Exception
35 | {
36 | JSONLowercaseParseSpec spec = new JSONLowercaseParseSpec(
37 | new TimestampSpec(
38 | "timestamp",
39 | "auto",
40 | null
41 | ),
42 | new DimensionsSpec(
43 | DimensionsSpec.getDefaultSchemas(Arrays.asList("A", "B")),
44 | Lists.newArrayList(),
45 | Lists.newArrayList()
46 | )
47 | );
48 | Parser parser = spec.makeParser();
49 | Map event = parser.parse("{\"timestamp\":\"2015-01-01\",\"A\":\"foo\"}");
50 | Assert.assertEquals("foo", event.get("a"));
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/JSONParseSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import io.druid.TestObjectMapper;
23 |
24 | import java.io.IOException;
25 | import java.util.Arrays;
26 | import java.util.HashMap;
27 |
28 | import org.junit.Assert;
29 | import org.junit.Test;
30 |
31 | import com.fasterxml.jackson.databind.ObjectMapper;
32 | import com.google.common.collect.ImmutableList;
33 |
34 | public class JSONParseSpecTest {
35 | private final ObjectMapper jsonMapper = new TestObjectMapper();
36 |
37 | @Test
38 | public void testSerde() throws IOException
39 | {
40 | HashMap feature = new HashMap();
41 | feature.put("ALLOW_UNQUOTED_CONTROL_CHARS", true);
42 | JSONParseSpec spec = new JSONParseSpec(
43 | new TimestampSpec("timestamp", "iso", null),
44 | new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")), null, null),
45 | null,
46 | feature
47 | );
48 |
49 | final JSONParseSpec serde = jsonMapper.readValue(
50 | jsonMapper.writeValueAsString(spec),
51 | JSONParseSpec.class
52 | );
53 | Assert.assertEquals("timestamp", serde.getTimestampSpec().getTimestampColumn());
54 | Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
55 |
56 | Assert.assertEquals(Arrays.asList("bar", "foo"), serde.getDimensionsSpec().getDimensionNames());
57 | Assert.assertEquals(feature, serde.getFeatureSpec());
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/JSONPathSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.databind.ObjectMapper;
23 | import com.google.common.collect.Lists;
24 | import io.druid.TestObjectMapper;
25 | import org.junit.Assert;
26 | import org.junit.Test;
27 |
28 | import java.io.IOException;
29 | import java.util.ArrayList;
30 | import java.util.Arrays;
31 | import java.util.List;
32 |
33 | public class JSONPathSpecTest
34 | {
35 | private final ObjectMapper jsonMapper = new TestObjectMapper();
36 |
37 | @Test
38 | public void testSerde() throws IOException
39 | {
40 | List fields = new ArrayList<>();
41 | fields.add(JSONPathFieldSpec.createNestedField("foobar1", "$.foo.bar1"));
42 | fields.add(JSONPathFieldSpec.createNestedField("baz0", "$.baz[0]"));
43 | fields.add(JSONPathFieldSpec.createNestedField("hey0barx", "$.hey[0].barx"));
44 | fields.add(JSONPathFieldSpec.createRootField("timestamp"));
45 | fields.add(JSONPathFieldSpec.createRootField("foo.bar1"));
46 |
47 | JSONPathSpec flattenSpec = new JSONPathSpec(true, fields);
48 |
49 | final JSONPathSpec serde = jsonMapper.readValue(
50 | jsonMapper.writeValueAsString(flattenSpec),
51 | JSONPathSpec.class
52 | );
53 | Assert.assertTrue(serde.isUseFieldDiscovery());
54 | List serdeFields = serde.getFields();
55 | JSONPathFieldSpec foobar1 = serdeFields.get(0);
56 | JSONPathFieldSpec baz0 = serdeFields.get(1);
57 | JSONPathFieldSpec hey0barx = serdeFields.get(2);
58 | JSONPathFieldSpec timestamp = serdeFields.get(3);
59 | JSONPathFieldSpec foodotbar1 = serdeFields.get(4);
60 |
61 | Assert.assertEquals(JSONPathFieldType.PATH, foobar1.getType());
62 | Assert.assertEquals("foobar1", foobar1.getName());
63 | Assert.assertEquals("$.foo.bar1", foobar1.getExpr());
64 |
65 | Assert.assertEquals(JSONPathFieldType.PATH, baz0.getType());
66 | Assert.assertEquals("baz0", baz0.getName());
67 | Assert.assertEquals("$.baz[0]", baz0.getExpr());
68 |
69 | Assert.assertEquals(JSONPathFieldType.PATH, hey0barx.getType());
70 | Assert.assertEquals("hey0barx", hey0barx.getName());
71 | Assert.assertEquals("$.hey[0].barx", hey0barx.getExpr());
72 |
73 | Assert.assertEquals(JSONPathFieldType.ROOT, timestamp.getType());
74 | Assert.assertEquals("timestamp", timestamp.getName());
75 | Assert.assertEquals("timestamp", timestamp.getExpr());
76 |
77 | Assert.assertEquals(JSONPathFieldType.ROOT, foodotbar1.getType());
78 | Assert.assertEquals("foo.bar1", foodotbar1.getName());
79 | Assert.assertEquals("foo.bar1", foodotbar1.getExpr());
80 | }
81 | }
82 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/JavaScriptParseSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.databind.ObjectMapper;
23 | import io.druid.TestObjectMapper;
24 | import org.junit.Assert;
25 | import org.junit.Test;
26 |
27 | import java.io.IOException;
28 | import java.util.Arrays;
29 |
30 | /**
31 | */
32 | public class JavaScriptParseSpecTest
33 | {
34 | private final ObjectMapper jsonMapper = new TestObjectMapper();
35 |
36 | @Test
37 | public void testSerde() throws IOException
38 | {
39 | JavaScriptParseSpec spec = new JavaScriptParseSpec(
40 | new TimestampSpec("abc", "iso", null),
41 | new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null),
42 | "abc"
43 | );
44 | final JavaScriptParseSpec serde = jsonMapper.readValue(
45 | jsonMapper.writeValueAsString(spec),
46 | JavaScriptParseSpec.class
47 | );
48 | Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn());
49 | Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
50 |
51 | Assert.assertEquals("abc", serde.getFunction());
52 | Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames());
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/NoopInputRowParserTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.databind.ObjectMapper;
23 | import com.google.common.collect.ImmutableList;
24 | import junit.framework.Assert;
25 | import org.junit.Test;
26 |
27 | /**
28 | */
29 | public class NoopInputRowParserTest
30 | {
31 | private final ObjectMapper mapper = new ObjectMapper();
32 |
33 | @Test
34 | public void testSerdeWithNullParseSpec() throws Exception
35 | {
36 | String jsonStr = "{ \"type\":\"noop\" }";
37 |
38 | InputRowParser actual = mapper.readValue(
39 | mapper.writeValueAsString(
40 | mapper.readValue(jsonStr, InputRowParser.class)
41 | ),
42 | InputRowParser.class
43 | );
44 |
45 | Assert.assertEquals(new NoopInputRowParser(null), actual);
46 | }
47 |
48 | @Test
49 | public void testSerdeWithNonNullParseSpec() throws Exception
50 | {
51 | String jsonStr = "{"
52 | + "\"type\":\"noop\","
53 | + "\"parseSpec\":{ \"format\":\"timeAndDims\", \"dimensionsSpec\": { \"dimensions\": [\"host\"] } }"
54 | + "}";
55 |
56 | InputRowParser actual = mapper.readValue(
57 | mapper.writeValueAsString(
58 | mapper.readValue(jsonStr, InputRowParser.class)
59 | ),
60 | InputRowParser.class
61 | );
62 |
63 | Assert.assertEquals(
64 | new NoopInputRowParser(
65 | new TimeAndDimsParseSpec(
66 | null,
67 | new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null)
68 | )
69 | ),
70 | actual
71 | );
72 | }
73 | }
74 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/ParseSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.google.common.collect.Lists;
23 | import com.metamx.common.parsers.ParseException;
24 | import org.junit.Test;
25 |
26 | import java.util.Arrays;
27 |
28 | public class ParseSpecTest
29 | {
30 | @Test(expected = ParseException.class)
31 | public void testDuplicateNames() throws Exception
32 | {
33 | final ParseSpec spec = new DelimitedParseSpec(
34 | new TimestampSpec(
35 | "timestamp",
36 | "auto",
37 | null
38 | ),
39 | new DimensionsSpec(
40 | DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b", "a")),
41 | Lists.newArrayList(),
42 | Lists.newArrayList()
43 | ),
44 | ",",
45 | " ",
46 | Arrays.asList("a", "b")
47 | );
48 | }
49 |
50 | @Test(expected = IllegalArgumentException.class)
51 | public void testDimAndDimExcluOverlap() throws Exception
52 | {
53 | final ParseSpec spec = new DelimitedParseSpec(
54 | new TimestampSpec(
55 | "timestamp",
56 | "auto",
57 | null
58 | ),
59 | new DimensionsSpec(
60 | DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "B")),
61 | Lists.newArrayList("B"),
62 | Lists.newArrayList()
63 | ),
64 | ",",
65 | null,
66 | Arrays.asList("a", "B")
67 | );
68 | }
69 |
70 | @Test
71 | public void testDimExclusionDuplicate() throws Exception
72 | {
73 | final ParseSpec spec = new DelimitedParseSpec(
74 | new TimestampSpec(
75 | "timestamp",
76 | "auto",
77 | null
78 | ),
79 | new DimensionsSpec(
80 | DimensionsSpec.getDefaultSchemas(Arrays.asList("a")),
81 | Lists.newArrayList("B", "B"),
82 | Lists.newArrayList()
83 | ),
84 | ",",
85 | null,
86 | Arrays.asList("a", "B")
87 | );
88 | }
89 | }
90 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/RegexParseSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.databind.ObjectMapper;
23 | import io.druid.TestObjectMapper;
24 | import org.junit.Assert;
25 | import org.junit.Test;
26 |
27 | import java.io.IOException;
28 | import java.util.Arrays;
29 |
30 | /**
31 | */
32 | public class RegexParseSpecTest
33 | {
34 | private final ObjectMapper jsonMapper = new TestObjectMapper();
35 |
36 | @Test
37 | public void testSerde() throws IOException
38 | {
39 | RegexParseSpec spec = new RegexParseSpec(
40 | new TimestampSpec("abc", "iso", null),
41 | new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null),
42 | "\u0001",
43 | Arrays.asList("abc"),
44 | "abc"
45 | );
46 | final RegexParseSpec serde = jsonMapper.readValue(
47 | jsonMapper.writeValueAsString(spec),
48 | RegexParseSpec.class
49 | );
50 | Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn());
51 | Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
52 |
53 | Assert.assertEquals("abc", serde.getPattern());
54 | Assert.assertEquals("\u0001", serde.getListDelimiter());
55 | Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames());
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/TimeAndDimsParseSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.fasterxml.jackson.databind.ObjectMapper;
23 | import com.google.common.collect.ImmutableList;
24 | import junit.framework.Assert;
25 | import org.junit.Test;
26 |
27 | /**
28 | */
29 | public class TimeAndDimsParseSpecTest
30 | {
31 | private final ObjectMapper mapper = new ObjectMapper();
32 |
33 | @Test
34 | public void testSerdeWithNulls() throws Exception
35 | {
36 | String jsonStr = "{ \"format\":\"timeAndDims\" }";
37 |
38 | ParseSpec actual = mapper.readValue(
39 | mapper.writeValueAsString(
40 | mapper.readValue(jsonStr, ParseSpec.class)
41 | ),
42 | ParseSpec.class
43 | );
44 |
45 | Assert.assertEquals(new TimeAndDimsParseSpec(null, null), actual);
46 | }
47 |
48 | @Test
49 | public void testSerdeWithNonNulls() throws Exception
50 | {
51 | String jsonStr = "{"
52 | + "\"format\":\"timeAndDims\","
53 | + "\"timestampSpec\": { \"column\": \"tcol\" },"
54 | + "\"dimensionsSpec\": { \"dimensions\": [\"host\"] }"
55 | + "}";
56 |
57 | ParseSpec actual = mapper.readValue(
58 | mapper.writeValueAsString(
59 | mapper.readValue(jsonStr, ParseSpec.class)
60 | ),
61 | ParseSpec.class
62 | );
63 |
64 | Assert.assertEquals(
65 | new TimeAndDimsParseSpec(
66 | new TimestampSpec("tcol", null, null),
67 | new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null)
68 | ),
69 | actual
70 | );
71 | }
72 | }
73 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/data/input/impl/TimestampSpecTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.data.input.impl;
21 |
22 | import com.google.common.collect.ImmutableMap;
23 | import junit.framework.Assert;
24 | import org.joda.time.DateTime;
25 | import org.junit.Test;
26 |
27 | public class TimestampSpecTest
28 | {
29 | @Test
30 | public void testExtractTimestamp() throws Exception
31 | {
32 | TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null);
33 | Assert.assertEquals(
34 | new DateTime("2014-03-01"),
35 | spec.extractTimestamp(ImmutableMap.of("TIMEstamp", "2014-03-01"))
36 | );
37 | }
38 |
39 | @Test
40 | public void testExtractTimestampWithMissingTimestampColumn() throws Exception
41 | {
42 | TimestampSpec spec = new TimestampSpec(null, null, new DateTime(0));
43 | Assert.assertEquals(
44 | new DateTime("1970-01-01"),
45 | spec.extractTimestamp(ImmutableMap.of("dim", "foo"))
46 | );
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/guice/PolyBindTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.guice;
21 |
22 | import com.google.common.collect.Iterables;
23 | import com.google.inject.Binder;
24 | import com.google.inject.Guice;
25 | import com.google.inject.Injector;
26 | import com.google.inject.Key;
27 | import com.google.inject.Module;
28 | import com.google.inject.ProvisionException;
29 | import com.google.inject.multibindings.MapBinder;
30 | import com.google.inject.name.Names;
31 | import org.junit.Assert;
32 | import org.junit.Test;
33 |
34 | import java.util.Arrays;
35 | import java.util.Properties;
36 |
37 | /**
38 | */
39 | public class PolyBindTest
40 | {
41 | private Properties props;
42 | private Injector injector;
43 |
44 | public void setUp(Module... modules) throws Exception
45 | {
46 | props = new Properties();
47 | injector = Guice.createInjector(
48 | Iterables.concat(
49 | Arrays.asList(
50 | new Module()
51 | {
52 | @Override
53 | public void configure(Binder binder)
54 | {
55 | binder.bind(Properties.class).toInstance(props);
56 | PolyBind.createChoice(binder, "billy", Key.get(Gogo.class), Key.get(GoA.class));
57 | PolyBind.createChoiceWithDefault(binder, "sally", Key.get(GogoSally.class), null, "b");
58 |
59 | }
60 | }
61 | ),
62 | Arrays.asList(modules)
63 | )
64 | );
65 | }
66 |
67 | @Test
68 | public void testSanity() throws Exception
69 | {
70 | setUp(
71 | new Module()
72 | {
73 | @Override
74 | public void configure(Binder binder)
75 | {
76 | final MapBinder gogoBinder = PolyBind.optionBinder(binder, Key.get(Gogo.class));
77 | gogoBinder.addBinding("a").to(GoA.class);
78 | gogoBinder.addBinding("b").to(GoB.class);
79 |
80 | final MapBinder gogoSallyBinder = PolyBind.optionBinder(binder, Key.get(GogoSally.class));
81 | gogoSallyBinder.addBinding("a").to(GoA.class);
82 | gogoSallyBinder.addBinding("b").to(GoB.class);
83 |
84 | PolyBind.createChoice(
85 | binder, "billy", Key.get(Gogo.class, Names.named("reverse")), Key.get(GoB.class)
86 | );
87 | final MapBinder annotatedGogoBinder = PolyBind.optionBinder(
88 | binder, Key.get(Gogo.class, Names.named("reverse"))
89 | );
90 | annotatedGogoBinder.addBinding("a").to(GoB.class);
91 | annotatedGogoBinder.addBinding("b").to(GoA.class);
92 | }
93 | }
94 | );
95 |
96 |
97 | Assert.assertEquals("A", injector.getInstance(Gogo.class).go());
98 | Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go());
99 | props.setProperty("billy", "b");
100 | Assert.assertEquals("B", injector.getInstance(Gogo.class).go());
101 | Assert.assertEquals("A", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go());
102 | props.setProperty("billy", "a");
103 | Assert.assertEquals("A", injector.getInstance(Gogo.class).go());
104 | Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go());
105 | props.setProperty("billy", "b");
106 | Assert.assertEquals("B", injector.getInstance(Gogo.class).go());
107 | Assert.assertEquals("A", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go());
108 | props.setProperty("billy", "c");
109 | Assert.assertEquals("A", injector.getInstance(Gogo.class).go());
110 | Assert.assertEquals("B", injector.getInstance(Key.get(Gogo.class, Names.named("reverse"))).go());
111 |
112 | // test default property value
113 | Assert.assertEquals("B", injector.getInstance(GogoSally.class).go());
114 | props.setProperty("sally", "a");
115 | Assert.assertEquals("A", injector.getInstance(GogoSally.class).go());
116 | props.setProperty("sally", "b");
117 | Assert.assertEquals("B", injector.getInstance(GogoSally.class).go());
118 | props.setProperty("sally", "c");
119 | try {
120 | injector.getInstance(GogoSally.class).go();
121 | Assert.fail(); // should never be reached
122 | } catch(Exception e) {
123 | Assert.assertTrue(e instanceof ProvisionException);
124 | Assert.assertTrue(e.getMessage().contains("Unknown provider[c] of Key[type=io.druid.guice.PolyBindTest$GogoSally"));
125 | }
126 | }
127 |
128 | public static interface Gogo
129 | {
130 | public String go();
131 | }
132 |
133 | public static interface GogoSally
134 | {
135 | public String go();
136 | }
137 |
138 | public static class GoA implements Gogo, GogoSally
139 | {
140 | @Override
141 | public String go()
142 | {
143 | return "A";
144 | }
145 | }
146 |
147 | public static class GoB implements Gogo, GogoSally
148 | {
149 | @Override
150 | public String go()
151 | {
152 | return "B";
153 | }
154 | }
155 | }
156 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/segment/loading/DataSegmentPusherUtilTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to Metamarkets Group Inc. (Metamarkets) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. Metamarkets licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing,
13 | * software distributed under the License is distributed on an
14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15 | * KIND, either express or implied. See the License for the
16 | * specific language governing permissions and limitations
17 | * under the License.
18 | */
19 |
20 | package io.druid.segment.loading;
21 |
22 | import com.google.common.collect.ImmutableMap;
23 | import io.druid.timeline.DataSegment;
24 | import io.druid.timeline.partition.NoneShardSpec;
25 | import org.joda.time.Interval;
26 | import org.junit.Assert;
27 | import org.junit.Test;
28 |
29 | import java.util.Arrays;
30 |
31 | public class DataSegmentPusherUtilTest
32 | {
33 | @Test
34 | public void shouldNotHaveColonsInHdfsStorageDir() throws Exception {
35 |
36 | Interval interval = new Interval("2011-10-01/2011-10-02");
37 | ImmutableMap loadSpec = ImmutableMap.of("something", "or_other");
38 |
39 | DataSegment segment = new DataSegment(
40 | "something",
41 | interval,
42 | "brand:new:version",
43 | loadSpec,
44 | Arrays.asList("dim1", "dim2"),
45 | Arrays.asList("met1", "met2"),
46 | new NoneShardSpec(),
47 | null,
48 | 1
49 | );
50 |
51 | String storageDir = DataSegmentPusherUtil.getHdfsStorageDir(segment);
52 | Assert.assertEquals("something/20111001T000000.000Z_20111002T000000.000Z/brand_new_version/0", storageDir);
53 |
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/src/test/java/io/druid/timeline/partition/NoneShardSpecTest.java:
--------------------------------------------------------------------------------
1 | package io.druid.timeline.partition;
2 |
3 | import org.junit.Assert;
4 | import org.junit.Test;
5 |
6 | public class NoneShardSpecTest
7 | {
8 | @Test
9 | public void testEqualsAndHashCode()
10 | {
11 | final ShardSpec one = new NoneShardSpec();
12 | final ShardSpec two = new NoneShardSpec();
13 | Assert.assertEquals(one, two);
14 | Assert.assertEquals(one.hashCode(), two.hashCode());
15 | }
16 | }
17 |
--------------------------------------------------------------------------------