mirror of https://github.com/apache/druid.git
Merge remote-tracking branch 'metamx/master'
This commit is contained in:
commit
276a62a0ce
|
@ -20,15 +20,15 @@
|
|||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid.extensions</groupId>
|
||||
<artifactId>druid-cassandra-storage</artifactId>
|
||||
<name>druid-cassandra-storage</name>
|
||||
<description>druid-cassandra-storage</description>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -20,15 +20,15 @@
|
|||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-common</artifactId>
|
||||
<name>druid-common</name>
|
||||
<description>druid-common</description>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.base.Supplier;
|
|||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.concurrent.ScheduledExecutors;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
|
@ -38,6 +39,7 @@ import org.skife.jdbi.v2.tweak.ResultSetMapper;
|
|||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
@ -76,7 +78,7 @@ public class ConfigManager
|
|||
final String configTable = dbTables.get().getConfigTable();
|
||||
|
||||
this.selectStatement = String.format("SELECT payload FROM %s WHERE name = :name", configTable);
|
||||
insertStatement = String.format(
|
||||
this.insertStatement = String.format(
|
||||
"INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload",
|
||||
configTable
|
||||
);
|
||||
|
@ -186,19 +188,29 @@ public class ConfigManager
|
|||
@Override
|
||||
public byte[] withHandle(Handle handle) throws Exception
|
||||
{
|
||||
return handle.createQuery(selectStatement)
|
||||
.bind("name", key)
|
||||
.map(
|
||||
new ResultSetMapper<byte[]>()
|
||||
{
|
||||
@Override
|
||||
public byte[] map(int index, ResultSet r, StatementContext ctx) throws SQLException
|
||||
{
|
||||
return r.getBytes("payload");
|
||||
}
|
||||
}
|
||||
)
|
||||
.first();
|
||||
List<byte[]> matched = handle.createQuery(selectStatement)
|
||||
.bind("name", key)
|
||||
.map(
|
||||
new ResultSetMapper<byte[]>()
|
||||
{
|
||||
@Override
|
||||
public byte[] map(int index, ResultSet r, StatementContext ctx)
|
||||
throws SQLException
|
||||
{
|
||||
return r.getBytes("payload");
|
||||
}
|
||||
}
|
||||
).list();
|
||||
|
||||
if (matched.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (matched.size() > 1) {
|
||||
throw new ISE("Error! More than one matching entry[%d] found for [%s]?!", matched.size(), key);
|
||||
}
|
||||
|
||||
return matched.get(0);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
|
|
@ -0,0 +1,725 @@
|
|||
<?xml version="1.0" encoding="UTF-8" standalone="no"?>
|
||||
<svg
|
||||
xmlns:dc="http://purl.org/dc/elements/1.1/"
|
||||
xmlns:cc="http://creativecommons.org/ns#"
|
||||
xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
|
||||
xmlns:svg="http://www.w3.org/2000/svg"
|
||||
xmlns="http://www.w3.org/2000/svg"
|
||||
xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
|
||||
xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
|
||||
version="1.1"
|
||||
viewBox="40 36 466.7041 318.99965"
|
||||
width="583.38013"
|
||||
height="398.74957"
|
||||
id="svg2"
|
||||
inkscape:version="0.48.2 r9819"
|
||||
sodipodi:docname="druid-dataflow.svg"
|
||||
inkscape:export-filename="/Users/xavier/mmx/druid/docs/_graphics/druid-dataflow@2x.png"
|
||||
inkscape:export-xdpi="244.5267"
|
||||
inkscape:export-ydpi="244.5267">
|
||||
<sodipodi:namedview
|
||||
pagecolor="#ffffff"
|
||||
bordercolor="#666666"
|
||||
borderopacity="1"
|
||||
objecttolerance="10"
|
||||
gridtolerance="10"
|
||||
guidetolerance="10"
|
||||
inkscape:pageopacity="0"
|
||||
inkscape:pageshadow="2"
|
||||
inkscape:window-width="1616"
|
||||
inkscape:window-height="949"
|
||||
id="namedview318"
|
||||
showgrid="false"
|
||||
fit-margin-top="0"
|
||||
fit-margin-left="0"
|
||||
fit-margin-right="0"
|
||||
fit-margin-bottom="0"
|
||||
inkscape:zoom="1.3366372"
|
||||
inkscape:cx="248.71225"
|
||||
inkscape:cy="133.85839"
|
||||
inkscape:window-x="0"
|
||||
inkscape:window-y="0"
|
||||
inkscape:window-maximized="0"
|
||||
inkscape:current-layer="layer1" />
|
||||
<metadata
|
||||
id="metadata4">
|
||||
<dc:date>2013-07-10 16:52Z</dc:date>
|
||||
<!-- Produced by OmniGraffle Professional 5.4.4 -->
|
||||
<rdf:RDF>
|
||||
<cc:Work
|
||||
rdf:about="">
|
||||
<dc:format>image/svg+xml</dc:format>
|
||||
<dc:type
|
||||
rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
|
||||
<dc:title></dc:title>
|
||||
</cc:Work>
|
||||
</rdf:RDF>
|
||||
</metadata>
|
||||
<defs
|
||||
id="defs6">
|
||||
<filter
|
||||
id="Shadow"
|
||||
filterUnits="userSpaceOnUse"
|
||||
color-interpolation-filters="sRGB">
|
||||
<feOffset
|
||||
in="SourceAlpha"
|
||||
result="offset"
|
||||
dx="0"
|
||||
dy="2"
|
||||
id="feOffset9" />
|
||||
<feFlood
|
||||
flood-color="black"
|
||||
flood-opacity=".12"
|
||||
result="flood"
|
||||
id="feFlood11" />
|
||||
<feComposite
|
||||
in="flood"
|
||||
in2="offset"
|
||||
operator="in"
|
||||
id="feComposite13" />
|
||||
</filter>
|
||||
<font-face
|
||||
font-family="Open Sans"
|
||||
font-size="12"
|
||||
panose-1="2 11 7 6 3 8 4 2 2 4"
|
||||
units-per-em="1000"
|
||||
underline-position="-75.195312"
|
||||
underline-thickness="49.804688"
|
||||
slope="0"
|
||||
x-height="549.8047"
|
||||
cap-height="724.1211"
|
||||
ascent="1068.8477"
|
||||
descent="-292.96875"
|
||||
font-weight="bold"
|
||||
id="font-face15"
|
||||
stemv="0"
|
||||
stemh="0"
|
||||
accent-height="0"
|
||||
ideographic="0"
|
||||
alphabetic="0"
|
||||
mathematical="0"
|
||||
hanging="0"
|
||||
v-ideographic="0"
|
||||
v-alphabetic="0"
|
||||
v-mathematical="0"
|
||||
v-hanging="0"
|
||||
strikethrough-position="0"
|
||||
strikethrough-thickness="0"
|
||||
overline-position="0"
|
||||
overline-thickness="0">
|
||||
<font-face-src
|
||||
id="font-face-src17">
|
||||
<font-face-name
|
||||
name="OpenSans-Semibold"
|
||||
id="font-face-name19" />
|
||||
</font-face-src>
|
||||
</font-face>
|
||||
<marker
|
||||
orient="auto"
|
||||
overflow="visible"
|
||||
markerUnits="strokeWidth"
|
||||
id="FilledArrow_Marker"
|
||||
viewBox="-1 -3 5 6"
|
||||
markerWidth="5"
|
||||
markerHeight="6"
|
||||
style="color:#7f95a7;overflow:visible">
|
||||
<g
|
||||
id="g22">
|
||||
<path
|
||||
d="M 2.8800001,0 0,-1.08 0,1.08 z"
|
||||
id="path24"
|
||||
inkscape:connector-curvature="0"
|
||||
style="fill:currentColor;stroke:currentColor;stroke-width:1" />
|
||||
</g>
|
||||
</marker>
|
||||
<marker
|
||||
orient="auto"
|
||||
overflow="visible"
|
||||
markerUnits="strokeWidth"
|
||||
id="FilledArrow_Marker_2"
|
||||
viewBox="-4 -3 5 6"
|
||||
markerWidth="5"
|
||||
markerHeight="6"
|
||||
style="color:#7f95a7;overflow:visible">
|
||||
<g
|
||||
id="g27">
|
||||
<path
|
||||
d="M -2.8800001,0 0,1.08 0,-1.08 z"
|
||||
id="path29"
|
||||
inkscape:connector-curvature="0"
|
||||
style="fill:currentColor;stroke:currentColor;stroke-width:1" />
|
||||
</g>
|
||||
</marker>
|
||||
<font-face
|
||||
font-family="Open Sans"
|
||||
font-size="18"
|
||||
panose-1="2 11 6 6 3 5 4 2 2 4"
|
||||
units-per-em="1000"
|
||||
underline-position="-75.195312"
|
||||
underline-thickness="49.804688"
|
||||
slope="0"
|
||||
x-height="544.92188"
|
||||
cap-height="724.1211"
|
||||
ascent="1068.8477"
|
||||
descent="-292.96875"
|
||||
font-weight="500"
|
||||
id="font-face31"
|
||||
stemv="0"
|
||||
stemh="0"
|
||||
accent-height="0"
|
||||
ideographic="0"
|
||||
alphabetic="0"
|
||||
mathematical="0"
|
||||
hanging="0"
|
||||
v-ideographic="0"
|
||||
v-alphabetic="0"
|
||||
v-mathematical="0"
|
||||
v-hanging="0"
|
||||
strikethrough-position="0"
|
||||
strikethrough-thickness="0"
|
||||
overline-position="0"
|
||||
overline-thickness="0">
|
||||
<font-face-src
|
||||
id="font-face-src33">
|
||||
<font-face-name
|
||||
name="OpenSans"
|
||||
id="font-face-name35" />
|
||||
</font-face-src>
|
||||
</font-face>
|
||||
<font-face
|
||||
font-family="Open Sans"
|
||||
font-size="10"
|
||||
panose-1="2 11 6 6 3 5 4 2 2 4"
|
||||
units-per-em="1000"
|
||||
underline-position="-75.195312"
|
||||
underline-thickness="49.804688"
|
||||
slope="0"
|
||||
x-height="544.92188"
|
||||
cap-height="724.1211"
|
||||
ascent="1068.8477"
|
||||
descent="-292.96875"
|
||||
font-weight="500"
|
||||
id="font-face37"
|
||||
stemv="0"
|
||||
stemh="0"
|
||||
accent-height="0"
|
||||
ideographic="0"
|
||||
alphabetic="0"
|
||||
mathematical="0"
|
||||
hanging="0"
|
||||
v-ideographic="0"
|
||||
v-alphabetic="0"
|
||||
v-mathematical="0"
|
||||
v-hanging="0"
|
||||
strikethrough-position="0"
|
||||
strikethrough-thickness="0"
|
||||
overline-position="0"
|
||||
overline-thickness="0">
|
||||
<font-face-src
|
||||
id="font-face-src39">
|
||||
<font-face-name
|
||||
name="OpenSans"
|
||||
id="font-face-name41" />
|
||||
</font-face-src>
|
||||
</font-face>
|
||||
<filter
|
||||
inkscape:collect="always"
|
||||
id="filter5034"
|
||||
x="-0.12"
|
||||
width="1.24"
|
||||
y="-0.12"
|
||||
height="1.24"
|
||||
color-interpolation-filters="sRGB">
|
||||
<feGaussianBlur
|
||||
inkscape:collect="always"
|
||||
stdDeviation="1.8"
|
||||
id="feGaussianBlur5036" />
|
||||
</filter>
|
||||
<filter
|
||||
inkscape:collect="always"
|
||||
id="filter5038"
|
||||
x="-0.12"
|
||||
width="1.24"
|
||||
y="-0.12"
|
||||
height="1.24"
|
||||
color-interpolation-filters="sRGB">
|
||||
<feGaussianBlur
|
||||
inkscape:collect="always"
|
||||
stdDeviation="1.8"
|
||||
id="feGaussianBlur5040" />
|
||||
</filter>
|
||||
<filter
|
||||
inkscape:collect="always"
|
||||
id="filter5042"
|
||||
x="-0.12"
|
||||
width="1.24"
|
||||
y="-0.12"
|
||||
height="1.24"
|
||||
color-interpolation-filters="sRGB">
|
||||
<feGaussianBlur
|
||||
inkscape:collect="always"
|
||||
stdDeviation="1.8"
|
||||
id="feGaussianBlur5044" />
|
||||
</filter>
|
||||
<filter
|
||||
inkscape:collect="always"
|
||||
id="filter5046"
|
||||
x="-0.12"
|
||||
width="1.24"
|
||||
y="-0.12"
|
||||
height="1.24"
|
||||
color-interpolation-filters="sRGB">
|
||||
<feGaussianBlur
|
||||
inkscape:collect="always"
|
||||
stdDeviation="1.8"
|
||||
id="feGaussianBlur5048" />
|
||||
</filter>
|
||||
<filter
|
||||
inkscape:collect="always"
|
||||
id="filter5050"
|
||||
x="-0.12"
|
||||
width="1.24"
|
||||
y="-0.12"
|
||||
height="1.24"
|
||||
color-interpolation-filters="sRGB">
|
||||
<feGaussianBlur
|
||||
inkscape:collect="always"
|
||||
stdDeviation="1.8"
|
||||
id="feGaussianBlur5052" />
|
||||
</filter>
|
||||
<filter
|
||||
inkscape:collect="always"
|
||||
id="filter5054"
|
||||
x="-0.12"
|
||||
width="1.24"
|
||||
y="-0.12"
|
||||
height="1.24"
|
||||
color-interpolation-filters="sRGB">
|
||||
<feGaussianBlur
|
||||
inkscape:collect="always"
|
||||
stdDeviation="1.8"
|
||||
id="feGaussianBlur5056" />
|
||||
</filter>
|
||||
<filter
|
||||
inkscape:collect="always"
|
||||
id="filter5058"
|
||||
x="-0.18000001"
|
||||
width="1.36"
|
||||
y="-0.18000001"
|
||||
height="1.36"
|
||||
color-interpolation-filters="sRGB">
|
||||
<feGaussianBlur
|
||||
inkscape:collect="always"
|
||||
stdDeviation="2.7"
|
||||
id="feGaussianBlur5060" />
|
||||
</filter>
|
||||
<filter
|
||||
inkscape:collect="always"
|
||||
id="filter5062"
|
||||
x="-0.18000001"
|
||||
width="1.36"
|
||||
y="-0.18000001"
|
||||
height="1.36"
|
||||
color-interpolation-filters="sRGB">
|
||||
<feGaussianBlur
|
||||
inkscape:collect="always"
|
||||
stdDeviation="2.7"
|
||||
id="feGaussianBlur5064" />
|
||||
</filter>
|
||||
</defs>
|
||||
<g
|
||||
inkscape:groupmode="layer"
|
||||
id="layer1"
|
||||
inkscape:label="Layer2"
|
||||
style="display:inline"
|
||||
transform="translate(-1.2070312,-2.461015e-6)">
|
||||
<text
|
||||
sodipodi:linespacing="125%"
|
||||
y="53.362823"
|
||||
x="68.675781"
|
||||
id="text109"
|
||||
style="font-size:12px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:center;line-height:125%;writing-mode:lr-tb;text-anchor:middle;fill:#33424f;stroke:none;font-family:Open Sans;-inkscape-font-specification:Open Sans">
|
||||
<tspan
|
||||
y="53.362823"
|
||||
x="68.675781"
|
||||
id="tspan4550"
|
||||
sodipodi:role="line">streaming</tspan>
|
||||
<tspan
|
||||
y="68.362823"
|
||||
x="68.675781"
|
||||
id="tspan4552"
|
||||
sodipodi:role="line">data</tspan>
|
||||
</text>
|
||||
<text
|
||||
sodipodi:linespacing="125%"
|
||||
y="145.23154"
|
||||
x="487.89328"
|
||||
id="text115"
|
||||
style="font-size:12px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:center;line-height:125%;writing-mode:lr-tb;text-anchor:middle;fill:#33424f;stroke:none;font-family:Open Sans;-inkscape-font-specification:Open Sans">
|
||||
<tspan
|
||||
font-size="12"
|
||||
font-weight="bold"
|
||||
x="493.09863"
|
||||
y="158.23154"
|
||||
textLength="43.283203"
|
||||
id="tspan117"
|
||||
style="font-size:12px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:center;line-height:125%;writing-mode:lr-tb;text-anchor:middle;fill:#33424f;font-family:Open Sans;-inkscape-font-specification:Open Sans">client</tspan>
|
||||
</text>
|
||||
<line
|
||||
x1="106.93243"
|
||||
y1="56.438995"
|
||||
x2="132.23264"
|
||||
y2="56.438995"
|
||||
id="line119"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-end:url(#FilledArrow_Marker)" />
|
||||
<text
|
||||
sodipodi:linespacing="125%"
|
||||
y="253.6384"
|
||||
x="64.488861"
|
||||
id="text121"
|
||||
style="font-size:12px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:center;line-height:125%;writing-mode:lr-tb;text-anchor:middle;fill:#33424f;stroke:none;font-family:Open Sans;-inkscape-font-specification:Open Sans">
|
||||
<tspan
|
||||
font-size="12"
|
||||
font-weight="bold"
|
||||
x="68.760742"
|
||||
y="266.6384"
|
||||
textLength="32.71289"
|
||||
id="tspan123"
|
||||
style="font-size:12px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:center;line-height:125%;writing-mode:lr-tb;text-anchor:middle;fill:#33424f;font-family:Open Sans;-inkscape-font-specification:Open Sans">batch</tspan>
|
||||
<tspan
|
||||
font-size="12"
|
||||
font-weight="bold"
|
||||
x="72.05957"
|
||||
y="283.6384"
|
||||
textLength="26.115234"
|
||||
id="tspan125"
|
||||
style="font-size:12px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:center;line-height:125%;writing-mode:lr-tb;text-anchor:middle;fill:#33424f;font-family:Open Sans;-inkscape-font-specification:Open Sans">data</tspan>
|
||||
</text>
|
||||
<line
|
||||
x1="107.43243"
|
||||
y1="270.6384"
|
||||
x2="132.73264"
|
||||
y2="270.6384"
|
||||
id="line127"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-end:url(#FilledArrow_Marker)" />
|
||||
<line
|
||||
x1="159.79263"
|
||||
y1="74.938965"
|
||||
x2="159.79263"
|
||||
y2="243.0784"
|
||||
id="line129"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-end:url(#FilledArrow_Marker)" />
|
||||
<path
|
||||
sodipodi:nodetypes="cc"
|
||||
inkscape:connector-curvature="0"
|
||||
id="line131"
|
||||
d="m 185.06378,57.75783 214.03664,87.3856"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-start:url(#FilledArrow_Marker_2);marker-end:url(#FilledArrow_Marker)" />
|
||||
<path
|
||||
sodipodi:nodetypes="cc"
|
||||
inkscape:connector-curvature="0"
|
||||
id="line133"
|
||||
d="m 178.29264,270.63837 121.01422,0"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-end:url(#FilledArrow_Marker)" />
|
||||
<line
|
||||
x1="355.03683"
|
||||
y1="245.13406"
|
||||
x2="405.93555"
|
||||
y2="180.64258"
|
||||
id="line135"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-start:url(#FilledArrow_Marker_2);marker-end:url(#FilledArrow_Marker)" />
|
||||
<line
|
||||
x1="353.40796"
|
||||
y1="155.13821"
|
||||
x2="398.50443"
|
||||
y2="155.13821"
|
||||
id="line137"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-end:url(#FilledArrow_Marker)" />
|
||||
<line
|
||||
x1="243.03348"
|
||||
y1="155.9173"
|
||||
x2="243.0354"
|
||||
y2="167.45596"
|
||||
id="line139"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-end:url(#FilledArrow_Marker)" />
|
||||
<path
|
||||
sodipodi:nodetypes="cc"
|
||||
inkscape:connector-curvature="0"
|
||||
id="line141"
|
||||
d="m 277.8411,176.62586 30.29726,-9.86829"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-start:url(#FilledArrow_Marker_2);marker-end:url(#FilledArrow_Marker)" />
|
||||
<path
|
||||
sodipodi:nodetypes="cc"
|
||||
inkscape:connector-curvature="0"
|
||||
id="line143"
|
||||
d="M 184.42804,68.73894 308.5797,140.29898"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-start:url(#FilledArrow_Marker_2);marker-end:url(#FilledArrow_Marker)" />
|
||||
<line
|
||||
x1="177.49472"
|
||||
y1="73.660522"
|
||||
x2="218.83446"
|
||||
y2="113.87802"
|
||||
id="line145"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-end:url(#FilledArrow_Marker)" />
|
||||
<line
|
||||
x1="334.90796"
|
||||
y1="243.0784"
|
||||
x2="334.90796"
|
||||
y2="182.69818"
|
||||
id="line147"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-start:url(#FilledArrow_Marker_2);marker-end:url(#FilledArrow_Marker)" />
|
||||
<path
|
||||
style="fill:#89d735;fill-opacity:1;stroke:none"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path177"
|
||||
d="m 142.63817,36.49612 30,0 c 1.65685,0 3,1.34314 3,3 l 0,30 c 0,1.65686 -1.34315,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34314 -3,-3 l 0,-30 c 0,-1.65686 1.34315,-3 3,-3 z" />
|
||||
<path
|
||||
style="fill:none;stroke:#60902c;stroke-width:1;stroke-linecap:round;stroke-linejoin:round;stroke-opacity:1"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path179"
|
||||
d="m 142.63817,36.49611 30,0 c 1.65685,0 3,1.34315 3,3 l 0,30 c 0,1.65686 -1.34315,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34314 -3,-3 l 0,-30 c 0,-1.65685 1.34315,-3 3,-3 z" />
|
||||
<text
|
||||
style="font-size:8px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#ffffff;stroke:none;font-family:Open Sans;-inkscape-font-specification:Open Sans"
|
||||
sodipodi:linespacing="125%"
|
||||
id="text3436"
|
||||
x="142.0069"
|
||||
y="59.447937">
|
||||
<tspan
|
||||
y="59.447937"
|
||||
x="142.0069"
|
||||
id="tspan4273"
|
||||
sodipodi:role="line">realtime</tspan>
|
||||
<tspan
|
||||
y="69.447937"
|
||||
x="142.0069"
|
||||
id="tspan4275"
|
||||
sodipodi:role="line">nodes</tspan>
|
||||
</text>
|
||||
<path
|
||||
sodipodi:nodetypes="sssssssss"
|
||||
style="fill:#687de9;stroke:none"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path186"
|
||||
d="m 311.90795,252.6384 46.0002,0 c 1.65685,0 3,1.34315 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -46.0002,0 c -1.65686,0 -3,-1.34315 -3,-3 l 0,-30 c 0,-1.65685 1.34314,-3 3,-3 z" />
|
||||
<path
|
||||
sodipodi:nodetypes="sssssssss"
|
||||
style="fill:none;stroke:#3446b0;stroke-width:1;stroke-linecap:round;stroke-linejoin:round"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path188"
|
||||
d="m 311.90795,252.6384 46.0002,0 c 1.65685,0 3,1.34315 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -46.0002,0 c -1.65686,0 -3,-1.34315 -3,-3 l 0,-30 c 0,-1.65685 1.34314,-3 3,-3 z" />
|
||||
<text
|
||||
sodipodi:linespacing="125%"
|
||||
y="274.90717"
|
||||
x="311.92163"
|
||||
style="font-size:8px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#ffffff;stroke:none;font-family:Sans;-inkscape-font-specification:Sans"
|
||||
id="text190">
|
||||
<tspan
|
||||
y="274.90717"
|
||||
x="311.92163"
|
||||
id="tspan4477"
|
||||
sodipodi:role="line">historical</tspan>
|
||||
<tspan
|
||||
y="284.90717"
|
||||
x="311.92163"
|
||||
id="tspan4479"
|
||||
sodipodi:role="line">nodes</tspan>
|
||||
</text>
|
||||
<path
|
||||
style="opacity:0.5;fill:#fd664a;stroke:none;filter:url(#filter5046)"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path195"
|
||||
d="m 228.0304,119.4173 30,0 c 1.65685,0 3,1.34315 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34315 -3,-3 l 0,-30 c 0,-1.65685 1.34315,-3 3,-3 z"
|
||||
transform="translate(-2.6747754e-6,-3.30513e-8)" />
|
||||
<path
|
||||
style="opacity:0.5;fill:none;stroke:#f3472c;stroke-width:1;stroke-linecap:round;stroke-linejoin:round;filter:url(#filter5042)"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path197"
|
||||
d="m 228.0304,119.4173 30,0 c 1.65685,0 3,1.34315 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34315 -3,-3 l 0,-30 c 0,-1.65685 1.34315,-3 3,-3 z"
|
||||
transform="translate(-2.6747754e-6,-3.30513e-8)" />
|
||||
<text
|
||||
sodipodi:linespacing="125%"
|
||||
style="font-size:8px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#ffffff;stroke:none;font-family:Sans;-inkscape-font-specification:Sans"
|
||||
id="text199"
|
||||
x="227.10298"
|
||||
y="151.71964">
|
||||
<tspan
|
||||
y="151.71964"
|
||||
x="227.10298"
|
||||
id="tspan4291"
|
||||
sodipodi:role="line">MySQL</tspan>
|
||||
</text>
|
||||
<path
|
||||
sodipodi:nodetypes="sssssssss"
|
||||
style="fill:#3fbab2;stroke:none"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path204"
|
||||
d="m 220.03055,177.01595 45.99985,0 c 1.65685,0 3,1.34314 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -45.99985,0 c -1.65685,0 -3,-1.34315 -3,-3 l 0,-30 c 0,-1.65686 1.34315,-3 3,-3 z" />
|
||||
<path
|
||||
sodipodi:nodetypes="sssssssscsss"
|
||||
style="fill:none;stroke:#1e9189;stroke-width:1;stroke-linecap:round;stroke-linejoin:round"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path206"
|
||||
d="m 220.03055,177.01595 45.99985,0 c 1.65685,0 3,1.34314 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -45.99985,0 c -0.82843,0 -1.57843,-0.33579 -2.12132,-0.87868 -0.54289,-0.54289 -0.87868,-1.29289 -0.87868,-2.12132 l 0,-15 0,-15 c 0,-0.82843 0.33579,-1.57843 0.87868,-2.12132 0.54289,-0.5429 1.29289,-0.87868 2.12132,-0.87868 z" />
|
||||
<text
|
||||
y="200.61603"
|
||||
x="218.88231"
|
||||
style="font-size:8px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#ffffff;stroke:none;font-family:Sans;-inkscape-font-specification:Sans"
|
||||
id="text208"
|
||||
sodipodi:linespacing="125%">
|
||||
<tspan
|
||||
y="200.61603"
|
||||
x="218.88231"
|
||||
id="tspan4489"
|
||||
sodipodi:role="line">coordinator</tspan>
|
||||
<tspan
|
||||
y="210.61603"
|
||||
x="218.88231"
|
||||
id="tspan4491"
|
||||
sodipodi:role="line">nodes</tspan>
|
||||
</text>
|
||||
<path
|
||||
style="opacity:0.5;fill:#191f7d;fill-opacity:1;stroke:none;filter:url(#filter5038)"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path213"
|
||||
d="m 144.79263,252.63839 30,0 c 1.65685,0 3,1.34315 3,3 l 0,30 c 0,1.65686 -1.34315,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34314 -3,-3 l 0,-30 c 0,-1.65685 1.34315,-3 3,-3 z"
|
||||
transform="translate(-2.6747754e-6,-3.30513e-8)" />
|
||||
<path
|
||||
style="opacity:0.5;fill:none;stroke:#080b3e;stroke-width:1;stroke-linecap:round;stroke-linejoin:round;stroke-opacity:1;filter:url(#filter5034)"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path215"
|
||||
d="m 144.79263,252.63839 30,0 c 1.65685,0 3,1.34315 3,3 l 0,30 c 0,1.65686 -1.34315,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34314 -3,-3 l 0,-30 c 0,-1.65685 1.34315,-3 3,-3 z"
|
||||
transform="translate(-2.6747754e-6,-3.30513e-8)" />
|
||||
<text
|
||||
sodipodi:linespacing="125%"
|
||||
y="274.63846"
|
||||
x="144.0856"
|
||||
style="font-size:8px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#ffffff;stroke:none;font-family:Sans;-inkscape-font-specification:Sans"
|
||||
id="text217">
|
||||
<tspan
|
||||
y="274.63846"
|
||||
x="144.0856"
|
||||
id="tspan4485"
|
||||
sodipodi:role="line">deep</tspan>
|
||||
<tspan
|
||||
y="284.63846"
|
||||
x="144.0856"
|
||||
id="tspan4487"
|
||||
sodipodi:role="line">storage</tspan>
|
||||
</text>
|
||||
<path
|
||||
style="opacity:0.5;fill:#d443a4;stroke:none;filter:url(#filter5054)"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path222"
|
||||
d="m 319.90796,137.13819 30,0 c 1.65686,0 3,1.34315 3,3 l 0,30 c 0,1.65686 -1.34314,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34314 -3,-3 l 0,-30 c 0,-1.65685 1.34315,-3 3,-3 z"
|
||||
transform="translate(-2.6747754e-6,-3.30513e-8)" />
|
||||
<path
|
||||
style="opacity:0.5;fill:none;stroke:#ad3184;stroke-width:1;stroke-linecap:round;stroke-linejoin:round;filter:url(#filter5050)"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path224"
|
||||
d="m 319.90796,137.13819 30,0 c 1.65686,0 3,1.34315 3,3 l 0,30 c 0,1.65686 -1.34314,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34314 -3,-3 l 0,-30 c 0,-1.65685 1.34315,-3 3,-3 z"
|
||||
transform="translate(-2.6747754e-6,-3.30513e-8)" />
|
||||
<text
|
||||
y="159.69537"
|
||||
x="318.52225"
|
||||
style="font-size:8px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#ffffff;stroke:none;font-family:Sans;-inkscape-font-specification:Sans"
|
||||
id="text226"
|
||||
sodipodi:linespacing="125%">
|
||||
<tspan
|
||||
y="159.69537"
|
||||
x="318.52225"
|
||||
id="tspan4309"
|
||||
sodipodi:role="line">Zoo</tspan>
|
||||
<tspan
|
||||
y="169.69537"
|
||||
x="318.52225"
|
||||
id="tspan4311"
|
||||
sodipodi:role="line">Keeper</tspan>
|
||||
</text>
|
||||
<line
|
||||
x1="453.12442"
|
||||
y1="154.63824"
|
||||
x2="467.89215"
|
||||
y2="154.63828"
|
||||
id="line293"
|
||||
style="fill:none;stroke:#7f95a7;stroke-width:2;stroke-linecap:round;stroke-linejoin:round;marker-start:url(#FilledArrow_Marker_2);marker-end:url(#FilledArrow_Marker)" />
|
||||
<path
|
||||
style="fill:#fbae4e;stroke:none"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path296"
|
||||
d="m 411.06442,137.13821 30,0 c 1.65685,0 3,1.34314 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -30,0 c -1.65687,0 -3,-1.34315 -3,-3 l 0,-30 c 0,-1.65686 1.34313,-3 3,-3 z" />
|
||||
<path
|
||||
style="fill:none;stroke:#e48819;stroke-width:1;stroke-linecap:round;stroke-linejoin:round"
|
||||
inkscape:connector-curvature="0"
|
||||
id="path298"
|
||||
d="m 411.06442,137.13821 30,0 c 1.65685,0 3,1.34314 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -30,0 c -1.65687,0 -3,-1.34315 -3,-3 l 0,-30 c 0,-1.65686 1.34313,-3 3,-3 z" />
|
||||
<text
|
||||
sodipodi:linespacing="125%"
|
||||
y="159.77753"
|
||||
x="409.95114"
|
||||
style="font-size:8px;font-style:normal;font-variant:normal;font-weight:normal;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#ffffff;stroke:none;font-family:Sans;-inkscape-font-specification:Sans"
|
||||
id="text300">
|
||||
<tspan
|
||||
y="159.77753"
|
||||
x="409.95114"
|
||||
id="tspan4481"
|
||||
sodipodi:role="line">broker</tspan>
|
||||
<tspan
|
||||
y="169.77753"
|
||||
x="409.95114"
|
||||
id="tspan4483"
|
||||
sodipodi:role="line">nodes</tspan>
|
||||
</text>
|
||||
<path
|
||||
d="m 424.40234,240.03369 30,0 c 1.65685,0 3,1.34315 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34315 -3,-3 l 0,-30 c 0,-1.65685 1.34315,-3 3,-3 z"
|
||||
id="path5004"
|
||||
inkscape:connector-curvature="0"
|
||||
style="opacity:0.5;fill:#a3a3a3;fill-opacity:1;stroke:none;filter:url(#filter5062)"
|
||||
transform="matrix(0.5,0,0,0.5,170.45068,213.49402)" />
|
||||
<path
|
||||
d="m 424.40234,240.03369 30,0 c 1.65685,0 3,1.34315 3,3 l 0,30 c 0,1.65685 -1.34315,3 -3,3 l -30,0 c -1.65685,0 -3,-1.34315 -3,-3 l 0,-30 c 0,-1.65685 1.34315,-3 3,-3 z"
|
||||
id="path5006"
|
||||
inkscape:connector-curvature="0"
|
||||
style="opacity:0.5;fill:none;stroke:#8f8f8f;stroke-width:1;stroke-linecap:round;stroke-linejoin:round;stroke-opacity:1;filter:url(#filter5058)"
|
||||
transform="matrix(0.5,0,0,0.5,170.45068,213.49402)" />
|
||||
<text
|
||||
style="font-size:12px;font-style:normal;font-variant:normal;font-weight:600;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#33424f;stroke:none;font-family:Open Sans;-inkscape-font-specification:Open Sans Semi-Bold"
|
||||
id="text5012"
|
||||
x="399.03949"
|
||||
y="315.00461"
|
||||
sodipodi:linespacing="125%">
|
||||
<tspan
|
||||
style="font-size:9.60000038px;font-style:normal;font-variant:normal;font-weight:300;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#33424f;font-family:Open Sans;-inkscape-font-specification:Open Sans Light"
|
||||
id="tspan5016"
|
||||
textLength="26.115234"
|
||||
y="345.00461"
|
||||
x="406.6102"
|
||||
font-weight="bold"
|
||||
font-size="12">external dependencies</tspan>
|
||||
</text>
|
||||
<path
|
||||
d="m 382.65185,305.62966 15,0 c 0.82843,0 1.5,0.67157 1.5,1.5 l 0,15 c 0,0.82842 -0.67157,1.5 -1.5,1.5 l -15,0 c -0.82844,0 -1.5,-0.67158 -1.5,-1.5 l 0,-15 c 0,-0.82843 0.67156,-1.5 1.5,-1.5 z"
|
||||
id="path5026"
|
||||
inkscape:connector-curvature="0"
|
||||
style="fill:#a5a5a5;fill-opacity:1;stroke:none" />
|
||||
<path
|
||||
d="m 382.65185,305.62966 15,0 c 0.82843,0 1.5,0.67157 1.5,1.5 l 0,15 c 0,0.82842 -0.67157,1.5 -1.5,1.5 l -15,0 c -0.82844,0 -1.5,-0.67158 -1.5,-1.5 l 0,-15 c 0,-0.82843 0.67156,-1.5 1.5,-1.5 z"
|
||||
id="path5028"
|
||||
inkscape:connector-curvature="0"
|
||||
style="fill:none;stroke:#7f7f7f;stroke-width:0.5;stroke-linecap:round;stroke-linejoin:round;stroke-opacity:1" />
|
||||
<text
|
||||
sodipodi:linespacing="125%"
|
||||
y="287.12341"
|
||||
x="399.03949"
|
||||
id="text5030"
|
||||
style="font-size:12px;font-style:normal;font-variant:normal;font-weight:600;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#33424f;stroke:none;font-family:Open Sans;-inkscape-font-specification:Open Sans Semi-Bold">
|
||||
<tspan
|
||||
font-size="12"
|
||||
font-weight="bold"
|
||||
x="406.6102"
|
||||
y="317.12341"
|
||||
textLength="26.115234"
|
||||
id="tspan5032"
|
||||
style="font-size:9.60000038px;font-style:normal;font-variant:normal;font-weight:300;font-stretch:normal;text-align:start;line-height:125%;writing-mode:lr-tb;text-anchor:start;fill:#33424f;font-family:Open Sans;-inkscape-font-specification:Open Sans Light">druid components</tspan>
|
||||
</text>
|
||||
</g>
|
||||
</svg>
|
After Width: | Height: | Size: 28 KiB |
|
@ -0,0 +1,41 @@
|
|||
// dot -Gnewrank -Tpng indexing_service.dot > indexing_service.png
|
||||
|
||||
digraph g {
|
||||
node [ fontname = "Helvetica Neue" ]
|
||||
node [ fontname = "Helvetica Neue" ]
|
||||
edge [ fontname = "Helvetica Neue Light Italic" fontsize = 12]
|
||||
|
||||
new_task[shape="plaintext" fontname="Helvetica Neue Light Italic"]
|
||||
overlord[shape="box" label="Overlord"]
|
||||
|
||||
new_task -> overlord
|
||||
overlord -> zk_tasks:mm1:n [label = "new_task"]
|
||||
zk_tasks:mm1 -> mm1 [label = "new_task"]
|
||||
|
||||
subgraph cluster_0 {
|
||||
style = "dotted"
|
||||
label = "ZooKeeper"
|
||||
fontname = "Helvetica Neue"
|
||||
|
||||
zk_status -> zk_tasks [style="invis"]
|
||||
|
||||
zk_status [fontname="Source Code Pro" shape = record label = "<status> /status | { <new_task> /new_task }"]
|
||||
zk_tasks [fontname="Source Code Pro" shape=record label="<tasks> /tasks | {<mm1> /mm1 | <mm2> /mm2 | <mm3> /mm3}"]
|
||||
|
||||
{ rank = same; zk_status zk_tasks }
|
||||
}
|
||||
|
||||
subgraph cluster_mm1 {
|
||||
style="dotted"
|
||||
mm1 [shape = "box" label = "Middle Manager 1" ]
|
||||
peon_11[label = "peon"]
|
||||
peon_12[label = "peon"]
|
||||
peon_13[label = "peon"]
|
||||
|
||||
mm1 -> peon_11 [label = "new_task"]
|
||||
mm1 -> { peon_12; peon_13 }
|
||||
}
|
||||
|
||||
zk_status:new_task:s -> peon_11:e [label = "new_task_status" dir = back]
|
||||
overlord:e -> zk_status:new_task:n [dir=back label="new_task_status"]
|
||||
}
|
|
@ -11,9 +11,6 @@
|
|||
<a href="mailto:info@druid.io">info@druid.io</a>
|
||||
</address>
|
||||
<address>
|
||||
<strong>Metamarkets</strong>
|
||||
625 2nd Street, Suite #230<br>
|
||||
San Francisco, CA 94017<br>
|
||||
<div class="soc">
|
||||
<a href="https://twitter.com/druidio"></a>
|
||||
<a href="https://github.com/metamx/druid" class="github"></a>
|
||||
|
@ -25,7 +22,7 @@
|
|||
<li><a href="/"><strong>DRUID</strong></a></li>
|
||||
<li><a href="/druid.html">What is Druid?</a></li>
|
||||
<li><a href="/downloads.html">Downloads</a></li>
|
||||
<li><a target="_blank" href="https://github.com/metamx/druid/wiki">Documentation</a></li>
|
||||
<li><a target="_blank" href="Home.html">Documentation</a></li>
|
||||
</ul>
|
||||
<ul class="col-md-4 list-unstyled">
|
||||
<li><a href="/community.html"><strong>SUPPORT</strong></a></li>
|
||||
|
|
|
@ -1,88 +1,113 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Batch Data Ingestion
|
||||
====================
|
||||
|
||||
There are two choices for batch data ingestion to your Druid cluster, you can use the [Indexing service](Indexing-service.html) or you can use the `HadoopDruidIndexer`. This page describes how to use the `HadoopDruidIndexer`.
|
||||
There are two choices for batch data ingestion to your Druid cluster, you can use the [Indexing service](Indexing-service.html) or you can use the `HadoopDruidIndexer`.
|
||||
|
||||
Which should I use?
|
||||
-------------------
|
||||
|
||||
The [Indexing service](Indexing-service.html) is a node that can run as part of your Druid cluster and can accomplish a number of different types of indexing tasks. Even if all you care about is batch indexing, it provides for the encapsulation of things like the Database that is used for segment metadata and other things, so that your indexing tasks do not need to include such information. Long-term, the indexing service is going to be the preferred method of ingesting data.
|
||||
The [Indexing service](Indexing-service.html) is a node that can run as part of your Druid cluster and can accomplish a number of different types of indexing tasks. Even if all you care about is batch indexing, it provides for the encapsulation of things like the [database](MySQL.html) that is used for segment metadata and other things, so that your indexing tasks do not need to include such information. The indexing service was created such that external systems could programmatically interact with it and run periodic indexing tasks. Long-term, the indexing service is going to be the preferred method of ingesting data.
|
||||
|
||||
The `HadoopDruidIndexer` runs hadoop jobs in order to separate and index data segments. It takes advantage of Hadoop as a job scheduling and distributed job execution platform. It is a simple method if you already have Hadoop running and don’t want to spend the time configuring and deploying the [Indexing service](Indexing service.html) just yet.
|
||||
|
||||
HadoopDruidIndexer
|
||||
------------------
|
||||
Batch Ingestion using the HadoopDruidIndexer
|
||||
--------------------------------------------
|
||||
|
||||
The HadoopDruidIndexer can be run like so:
|
||||
|
||||
```
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath hadoop_config_path:`echo lib/* | tr ' ' ':'` io.druid.cli.Main index hadoop <config_file>
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:<hadoop_config_path> io.druid.cli.Main index hadoop <config_file>
|
||||
```
|
||||
|
||||
The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Time_intervals) of the data you are processing. The config\_file is a path to a file (the "specFile") that contains JSON and an example looks like:
|
||||
|
||||
```
|
||||
```json
|
||||
{
|
||||
"dataSource": "the_data_source",
|
||||
"timestampColumn": "ts",
|
||||
"timestampFormat": "<iso, millis, posix, auto or any Joda time format>",
|
||||
"timestampSpec" : {
|
||||
"timestampColumn": "ts",
|
||||
"timestampFormat": "<iso, millis, posix, auto or any Joda time format>"
|
||||
},
|
||||
"dataSpec": {
|
||||
"format": "<csv, tsv, or json>",
|
||||
"columns": ["ts", "column_1", "column_2", "column_3", "column_4", "column_5"],
|
||||
"dimensions": ["column_1", "column_2", "column_3"]
|
||||
"columns": [
|
||||
"ts",
|
||||
"column_1",
|
||||
"column_2",
|
||||
"column_3",
|
||||
"column_4",
|
||||
"column_5"
|
||||
],
|
||||
"dimensions": [
|
||||
"column_1",
|
||||
"column_2",
|
||||
"column_3"
|
||||
]
|
||||
},
|
||||
"granularitySpec": {
|
||||
"type":"uniform",
|
||||
"intervals":["<ISO8601 interval:http://en.wikipedia.org/wiki/ISO_8601#Time_intervals>"],
|
||||
"gran":"day"
|
||||
"type": "uniform",
|
||||
"intervals": [
|
||||
"<ISO8601 interval:http:\/\/en.wikipedia.org\/wiki\/ISO_8601#Time_intervals>"
|
||||
],
|
||||
"gran": "day"
|
||||
},
|
||||
"pathSpec": { "type": "granularity",
|
||||
"dataGranularity": "hour",
|
||||
"inputPath": "s3n://billy-bucket/the/data/is/here",
|
||||
"filePattern": ".*" },
|
||||
"rollupSpec": { "aggs": [
|
||||
{ "type": "count", "name":"event_count" },
|
||||
{ "type": "doubleSum", "fieldName": "column_4", "name": "revenue" },
|
||||
{ "type": "longSum", "fieldName" : "column_5", "name": "clicks" }
|
||||
],
|
||||
"rollupGranularity": "minute"},
|
||||
"workingPath": "/tmp/path/on/hdfs",
|
||||
"segmentOutputPath": "s3n://billy-bucket/the/segments/go/here",
|
||||
"pathSpec": {
|
||||
"type": "granularity",
|
||||
"dataGranularity": "hour",
|
||||
"inputPath": "s3n:\/\/billy-bucket\/the\/data\/is\/here",
|
||||
"filePattern": ".*"
|
||||
},
|
||||
"rollupSpec": {
|
||||
"aggs": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "event_count"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"fieldName": "column_4",
|
||||
"name": "revenue"
|
||||
},
|
||||
{
|
||||
"type": "longSum",
|
||||
"fieldName": "column_5",
|
||||
"name": "clicks"
|
||||
}
|
||||
],
|
||||
"rollupGranularity": "minute"
|
||||
},
|
||||
"workingPath": "\/tmp\/path\/on\/hdfs",
|
||||
"segmentOutputPath": "s3n:\/\/billy-bucket\/the\/segments\/go\/here",
|
||||
"leaveIntermediate": "false",
|
||||
"partitionsSpec": {
|
||||
"targetPartitionSize": 5000000
|
||||
},
|
||||
"updaterJobSpec": {
|
||||
"type":"db",
|
||||
"connectURI":"jdbc:mysql://localhost:7980/test_db",
|
||||
"user":"username",
|
||||
"password":"passmeup",
|
||||
"segmentTable":"segments"
|
||||
"type": "db",
|
||||
"connectURI": "jdbc:mysql:\/\/localhost:7980\/test_db",
|
||||
"user": "username",
|
||||
"password": "passmeup",
|
||||
"segmentTable": "segments"
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
### Hadoop indexer config
|
||||
### Hadoop Index Config
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|dataSource|name of the dataSource the data will belong to|yes|
|
||||
|timestampColumn|the column that is to be used as the timestamp column|yes|
|
||||
|timestampFormat|the format of timestamps; auto = either iso or millis, Joda time formats:http://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html|yes|
|
||||
|dataSpec|a specification of the data format and an array that names all of the columns in the input data|yes|
|
||||
|dimensions|the columns that are to be used as dimensions|yes|
|
||||
|granularitySpec|the time granularity and interval to chunk segments up into|yes|
|
||||
|timestampSpec|includes the column that is to be used as the timestamp column and the format of the timestamps; auto = either iso or millis, Joda time formats can be found [here](http://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html).|yes|
|
||||
|dataSpec|a specification of the data format and an array that names all of the columns in the input data.|yes|
|
||||
|dimensions|the columns that are to be used as dimensions.|yes|
|
||||
|granularitySpec|the time granularity and interval to chunk segments up into.|yes|
|
||||
|pathSpec|a specification of where to pull the data in from|yes|
|
||||
|rollupSpec|a specification of the rollup to perform while processing the data|yes|
|
||||
|workingPath|the working path to use for intermediate results (results between Hadoop jobs)|yes|
|
||||
|segmentOutputPath|the path to dump segments into|yes|
|
||||
|leaveIntermediate|leave behind files in the workingPath when job completes or fails (debugging tool)|no|
|
||||
|partitionsSpec|a specification of how to partition each time bucket into segments, absence of this property means no partitioning will occur|no|
|
||||
|updaterJobSpec|a specification of how to update the metadata for the druid cluster these segments belong to|yes|
|
||||
|registererers|a list of serde handler classnames|no|
|
||||
|workingPath|the working path to use for intermediate results (results between Hadoop jobs).|yes|
|
||||
|segmentOutputPath|the path to dump segments into.|yes|
|
||||
|leaveIntermediate|leave behind files in the workingPath when job completes or fails (debugging tool).|no|
|
||||
|partitionsSpec|a specification of how to partition each time bucket into segments, absence of this property means no partitioning will occur.|no|
|
||||
|updaterJobSpec|a specification of how to update the metadata for the druid cluster these segments belong to.|yes|
|
||||
|
||||
### Path specification
|
||||
|
||||
|
@ -94,9 +119,9 @@ Is a type of data loader that expects data to be laid out in a specific path for
|
|||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|dataGranularity|specifies the granularity to expect the data at, e.g. hour means to expect directories `y=XXXX/m=XX/d=XX/H=XX`|yes|
|
||||
|inputPath|Base path to append the expected time path to|yes|
|
||||
|filePattern|Pattern that files should match to be included|yes|
|
||||
|dataGranularity|specifies the granularity to expect the data at, e.g. hour means to expect directories `y=XXXX/m=XX/d=XX/H=XX`.|yes|
|
||||
|inputPath|Base path to append the expected time path to.|yes|
|
||||
|filePattern|Pattern that files should match to be included.|yes|
|
||||
|
||||
For example, if the sample config were run with the interval 2012-06-01/2012-06-02, it would expect data at the paths
|
||||
|
||||
|
@ -114,7 +139,7 @@ The indexing process has the ability to roll data up as it processes the incomin
|
|||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|aggs|specifies a list of aggregators to aggregate for each bucket (a bucket is defined by the tuple of the truncated timestamp and the dimensions). Aggregators available here are the same as available when querying.|yes|
|
||||
|rollupGranularity|The granularity to use when truncating incoming timestamps for bucketization|yes|
|
||||
|rollupGranularity|The granularity to use when truncating incoming timestamps for bucketization.|yes|
|
||||
|
||||
### Partitioning specification
|
||||
|
||||
|
@ -134,10 +159,92 @@ This is a specification of the properties that tell the job how to update metada
|
|||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|"db" is the only value available|yes|
|
||||
|connectURI|a valid JDBC url to MySQL|yes|
|
||||
|user|username for db|yes|
|
||||
|password|password for db|yes|
|
||||
|segmentTable|table to use in DB|yes|
|
||||
|type|"db" is the only value available.|yes|
|
||||
|connectURI|A valid JDBC url to MySQL.|yes|
|
||||
|user|Username for db.|yes|
|
||||
|password|password for db.|yes|
|
||||
|segmentTable|Table to use in DB.|yes|
|
||||
|
||||
These properties should parrot what you have configured for your [Coordinator](Coordinator.html).
|
||||
|
||||
Batch Ingestion Using the Indexing Service
|
||||
------------------------------------------
|
||||
|
||||
Batch ingestion for the indexing service is done by submitting a [Hadoop Index Task](Tasks.html). The indexing service can be started by issuing:
|
||||
|
||||
```
|
||||
java -Xmx2g -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/overlord io.druid.cli.Main server overlord
|
||||
```
|
||||
|
||||
This will start up a very simple local indexing service. For more complex deployments of the indexing service, see [here](Indexing-Service.html).
|
||||
|
||||
The schema of the Hadoop Index Task contains a task "type" and a Hadoop Index Config. A sample Hadoop index task is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "index_hadoop",
|
||||
"config": {
|
||||
"dataSource" : "example",
|
||||
"timestampSpec" : {
|
||||
"timestampColumn" : "timestamp",
|
||||
"timestampFormat" : "auto"
|
||||
},
|
||||
"dataSpec" : {
|
||||
"format" : "json",
|
||||
"dimensions" : ["dim1","dim2","dim3"]
|
||||
},
|
||||
"granularitySpec" : {
|
||||
"type" : "uniform",
|
||||
"gran" : "DAY",
|
||||
"intervals" : [ "2013-08-31/2013-09-01" ]
|
||||
},
|
||||
"pathSpec" : {
|
||||
"type" : "static",
|
||||
"paths" : "data.json"
|
||||
},
|
||||
"targetPartitionSize" : 5000000,
|
||||
"rollupSpec" : {
|
||||
"aggs": [{
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "added",
|
||||
"fieldName" : "added"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "deleted",
|
||||
"fieldName" : "deleted"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "delta",
|
||||
"fieldName" : "delta"
|
||||
}],
|
||||
"rollupGranularity" : "none"
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This should be "index_hadoop".|yes|
|
||||
|config|A Hadoop Index Config (see above).|yes|
|
||||
|hadoopCoordinates|The Maven <groupId>:<artifactId>:<version> of Hadoop to use. The default is "org.apache.hadoop:hadoop-core:1.0.3".|no|
|
||||
|
||||
The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopBatchIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `updaterJobSpec`. The Indexing Service takes care of setting these fields internally.
|
||||
|
||||
To run the task:
|
||||
|
||||
```
|
||||
curl -X 'POST' -H 'Content-Type:application/json' -d @example_index_hadoop_task.json localhost:8087/druid/indexer/v1/task
|
||||
```
|
||||
|
||||
If the task succeeds, you should see in the logs of the indexing service:
|
||||
|
||||
```
|
||||
2013-10-16 16:38:31,945 INFO [pool-6-thread-1] io.druid.indexing.overlord.exec.TaskConsumer - Task SUCCESS: HadoopIndexTask...
|
||||
```
|
||||
|
||||
Having Problems?
|
||||
----------------
|
||||
Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-development).
|
||||
|
|
|
@ -3,7 +3,7 @@ layout: doc_page
|
|||
---
|
||||
# Booting a Single Node Cluster #
|
||||
|
||||
[Loading Your Data](Loading-Your-Data.html) and [Querying Your Data](Querying-Your-Data.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.0-bin.tar.gz).
|
||||
[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.0-bin.tar.gz).
|
||||
|
||||
The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables:
|
||||
|
||||
|
|
|
@ -6,6 +6,64 @@ Broker
|
|||
|
||||
The Broker is the node to route queries to if you want to run a distributed cluster. It understands the metadata published to ZooKeeper about what segments exist on what nodes and routes queries such that they hit the right nodes. This node also merges the result sets from all of the individual nodes together.
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server broker
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=broker
|
||||
druid.port=8080
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
```
|
||||
|
||||
JVM Configuration
|
||||
-----------------
|
||||
|
||||
The broker module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.broker.cache.type`|Choices: local, memcache. The type of cache to use for queries.|local|
|
||||
|
||||
#### Local Cache
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.broker.cache.sizeInBytes`|Maximum size of the cache. If this is zero, cache is disabled.|0|
|
||||
|`druid.broker.cache.initialSize`|The initial size of the cache in bytes.|500000|
|
||||
|`druid.broker.cache.logEvictionCount`|If this is non-zero, there will be an eviction of entries.|0|
|
||||
|
||||
#### Memcache
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.broker.cache.expiration`|Memcache [expiration time ](https://code.google.com/p/memcached/wiki/NewCommands#Standard_Protocol).|2592000 (30 days)|
|
||||
|`druid.broker.cache.timeout`|Maximum time in milliseconds to wait for a response from Memcache.|500|
|
||||
|`druid.broker.cache.hosts`|Memcache hosts.|none|
|
||||
|`druid.broker.cache.maxObjectSize`|Maximum object size in bytes for a Memcache object.|52428800 (50 MB)|
|
||||
|`druid.broker.cache.memcachedPrefix`|Key prefix for all keys in Memcache.|druid|
|
||||
|
||||
Running
|
||||
-------
|
||||
|
||||
```
|
||||
io.druid.cli.Main server broker
|
||||
```
|
||||
|
||||
|
||||
Forwarding Queries
|
||||
------------------
|
||||
|
||||
|
@ -16,15 +74,5 @@ To determine which nodes to forward queries to, the Broker node first builds a v
|
|||
Caching
|
||||
-------
|
||||
|
||||
Broker nodes employ a distributed cache with a LRU cache invalidation strategy. The broker cache stores per segment results. The cache can be local to each broker node or shared across multiple nodes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker node receives a query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment results that do not exist in the cache, the broker node will forward the query to the
|
||||
historical nodes. Once the historical nodes return their results, the broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and caching the results would be unreliable.
|
||||
|
||||
Running
|
||||
-------
|
||||
|
||||
Broker nodes can be run using the `com.metamx.druid.http.BrokerMain` class.
|
||||
|
||||
Configuration
|
||||
-------------
|
||||
|
||||
See [Configuration](Configuration.html).
|
||||
Broker nodes employ a cache with a LRU cache invalidation strategy. The broker cache stores per segment results. The cache can be local to each broker node or shared across multiple nodes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker node receives a query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment results that do not exist in the cache, the broker node will forward the query to the
|
||||
historical nodes. Once the historical nodes return their results, the broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and caching the results would be unreliable.
|
|
@ -0,0 +1,118 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
A Druid cluster consists of various node types that need to be set up depending on your use case. See our [Design](Design.html) docs for a description of the different node types.
|
||||
|
||||
Setup Scripts
|
||||
-------------
|
||||
|
||||
One of our community members, [housejester](https://github.com/housejester/), contributed some scripts to help with setting up a cluster. Checkout the [github](https://github.com/housejester/druid-test-harness) and [wiki](https://github.com/housejester/druid-test-harness/wiki/Druid-Test-Harness).
|
||||
|
||||
Minimum Physical Layout: Absolute Minimum
|
||||
-----------------------------------------
|
||||
|
||||
As a special case, the absolute minimum setup is one of the standalone examples for real-time ingestion and querying; see [Examples](Examples.html) that can easily run on one machine with one core and 1GB RAM. This layout can be set up to try some basic queries with Druid.
|
||||
|
||||
Minimum Physical Layout: Experimental Testing with 4GB of RAM
|
||||
-------------------------------------------------------------
|
||||
|
||||
This layout can be used to load some data from deep storage onto a Druid compute node for the first time. A minimal physical layout for a 1 or 2 core machine with 4GB of RAM is:
|
||||
|
||||
1. node1: [Coordinator](Coordinator.html) + metadata service + zookeeper + [Historical](Historical.html)
|
||||
2. transient nodes: [indexer](Batch-ingestion.html)
|
||||
|
||||
This setup is only reasonable to prove that a configuration works. It would not be worthwhile to use this layout for performance measurement.
|
||||
|
||||
Comfortable Physical Layout: Pilot Project with Multiple Machines
|
||||
-----------------------------------------------------------------
|
||||
|
||||
The machine size "flavors" are using AWS/EC2 terminology for descriptive purposes only and is not meant to imply that AWS/EC2 is required or recommended. Another cloud provider or your own hardware can also work.
|
||||
|
||||
A minimal physical layout not constrained by cores that demonstrates parallel querying and realtime, using AWS-EC2 "small"/m1.small (one core, with 1.7GB of RAM) or larger, no real-time, is:
|
||||
|
||||
1. node1: [Coordinator](Coordinator.html) (m1.small)
|
||||
2. node2: metadata service (m1.small)
|
||||
3. node3: zookeeper (m1.small)
|
||||
4. node4: [[Broker]] (m1.small or m1.medium or m1.large)
|
||||
5. node5: [Historical](Historical.html) (m1.small or m1.medium or m1.large)
|
||||
6. node6: [Historical](Historical.html) (m1.small or m1.medium or m1.large)
|
||||
7. node7: [[Realtime]] (m1.small or m1.medium or m1.large)
|
||||
8. transient nodes: [indexer](Batch-ingestion.html)
|
||||
|
||||
This layout naturally lends itself to adding more RAM and core to Compute nodes, and to adding many more Compute nodes. Depending on the actual load, the Master, metadata server, and Zookeeper might need to use larger machines.
|
||||
|
||||
High Availability Physical Layout
|
||||
---------------------------------
|
||||
|
||||
The machine size "flavors" are using AWS/EC2 terminology for descriptive purposes only and is not meant to imply that AWS/EC2 is required or recommended. Another cloud provider or your own hardware can also work.
|
||||
|
||||
An HA layout allows full rolling restarts and heavy volume:
|
||||
|
||||
1. node1: [Coordinator](Coordinator.html) (m1.small or m1.medium or m1.large)
|
||||
2. node2: [Coordinator](Coordinator.html) (m1.small or m1.medium or m1.large) (backup)
|
||||
3. node3: metadata service (c1.medium or m1.large)
|
||||
4. node4: metadata service (c1.medium or m1.large) (backup)
|
||||
5. node5: zookeeper (c1.medium)
|
||||
6. node6: zookeeper (c1.medium)
|
||||
7. node7: zookeeper (c1.medium)
|
||||
8. node8: [[Broker]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge)
|
||||
9. node9: [[Broker]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge) (backup)
|
||||
10. node10: [Historical](Historical.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge)
|
||||
11. node11: [Historical](Historical.html) (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge)
|
||||
12. node12: [[Realtime]] (m1.small or m1.medium or m1.large or m2.xlarge or m2.2xlarge or m2.4xlarge)
|
||||
13. transient nodes: [indexer](Batch-ingestion.html)
|
||||
|
||||
Sizing for Cores and RAM
|
||||
------------------------
|
||||
|
||||
The Compute and Broker nodes will use as many cores as are available, depending on usage, so it is best to keep these on dedicated machines. The upper limit of effectively utilized cores is not well characterized yet and would depend on types of queries, query load, and the schema. Compute daemons should have a heap a size of at least 1GB per core for normal usage, but could be squeezed into a smaller heap for testing. Since in-memory caching is essential for good performance, even more RAM is better. Broker nodes will use RAM for caching, so they do more than just route queries.
|
||||
|
||||
The effective utilization of cores by Zookeeper, MySQL, and Master nodes is likely to be between 1 and 2 for each process/daemon, so these could potentially share a machine with lots of cores. These daemons work with heap a size between 500MB and 1GB.
|
||||
|
||||
Storage
|
||||
-------
|
||||
|
||||
Indexed segments should be kept in a permanent store accessible by all nodes like AWS S3 or HDFS or equivalent. Currently Druid supports S3, but this will be extended soon.
|
||||
|
||||
Local disk ("ephemeral" on AWS EC2) for caching is recommended over network mounted storage (example of mounted: AWS EBS, Elastic Block Store) in order to avoid network delays during times of heavy usage. If your data center is suitably provisioned for networked storage, perhaps with separate LAN/NICs just for storage, then mounted might work fine.
|
||||
|
||||
Setup
|
||||
-----
|
||||
|
||||
Setting up a cluster is essentially just firing up all of the nodes you want with the proper [[configuration]]. One thing to be aware of is that there are a few properties in the configuration that potentially need to be set individually for each process:
|
||||
|
||||
```
|
||||
druid.server.type=historical|realtime
|
||||
druid.host=someHostOrIPaddrWithPort
|
||||
druid.port=8080
|
||||
```
|
||||
|
||||
`druid.server.type` should be set to "historical" for your compute nodes and realtime for the realtime nodes. The master will only assign segments to a "historical" node and the broker has some intelligence around its ability to cache results when talking to a realtime node. This does not need to be set for the master or the broker.
|
||||
|
||||
`druid.host` should be set to the hostname and port that can be used to talk to the given server process. Basically, someone should be able to send a request to http://${druid.host}/ and actually talk to the process.
|
||||
|
||||
`druid.port` should be set to the port that the server should listen on. In the vast majority of cases, this port should be the same as what is on `druid.host`.
|
||||
|
||||
Build/Run
|
||||
---------
|
||||
|
||||
The simplest way to build and run from the repository is to run `mvn package` from the base directory and then take `druid-services/target/druid-services-*-selfcontained.jar` and push that around to your machines; the jar does not need to be expanded, and since it contains the main() methods for each kind of service, it is *not* invoked with java -jar. It can be run from a normal java command-line by just including it on the classpath and then giving it the main class that you want to run. For example one instance of the Historical node/service can be started like this:
|
||||
|
||||
```
|
||||
java -Duser.timezone=UTC -Dfile.encoding=UTF-8 -cp services/target/druid-services-*-selfcontained.jar io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
All Druid server nodes can be started with:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server <node_type>
|
||||
```
|
||||
|
||||
The table below show the program arguments for the different node types.
|
||||
|
||||
|service|program arguments|
|
||||
|-------|----------------|
|
||||
|Realtime|realtime|
|
||||
|Coordinator|coordinator|
|
||||
|Broker|broker|
|
||||
|Historical|historical|
|
|
@ -12,150 +12,317 @@ There are three JVM parameters that we set on all of our processes:
|
|||
2. `-Dfile.encoding=UTF-8` This is similar to timezone, we test assuming UTF-8. Local encodings might work, but they also might result in weird and interesting bugs
|
||||
3. `-Djava.io.tmpdir=<a path>` Various parts of the system that interact with the file system do it via temporary files, these files can get somewhat large. Many production systems are setup to have small (but fast) `/tmp` directories, these can be problematic with Druid so we recommend pointing the JVM’s tmp directory to something with a little more meat.
|
||||
|
||||
Basic Service Configuration
|
||||
===========================
|
||||
Modules
|
||||
=======
|
||||
|
||||
Configuration of the various nodes is done via Java properties. These can either be provided as `-D` system properties on the java command line or they can be passed in via a file called `runtime.properties` that exists on the classpath. Note: as a future item, I’d like to consolidate all of the various configuration into a yaml/JSON based configuration files.
|
||||
As of Druid v0.6.0, most core Druid functionality has been compartmentalized into modules. There are a set of default modules that may apply to any node type, and there are specific modules for the different node types. Default modules are __lazily instantiated__. Each module has its own set of configuration. This page will describe the configuration of the default modules.
|
||||
|
||||
The periodic time intervals (like "PT1M") are [ISO8601 intervals](http://en.wikipedia.org/wiki/ISO_8601#Time_intervals)
|
||||
Configuration of the various modules is done via Java properties. These can either be provided as `-D` system properties on the java command line or they can be passed in via a file called `runtime.properties` that exists on the classpath.
|
||||
|
||||
An example runtime.properties is as follows:
|
||||
Note: as a future item, we’d like to consolidate all of the various configuration into a yaml/JSON based configuration files.
|
||||
|
||||
```
|
||||
# S3 access
|
||||
com.metamx.aws.accessKey=<S3 access key>
|
||||
com.metamx.aws.secretKey=<S3 secret_key>
|
||||
### Emitter Module
|
||||
|
||||
# thread pool size for servicing queries
|
||||
druid.client.http.connections=30
|
||||
|
||||
# JDBC connection string for metadata database
|
||||
druid.database.connectURI=
|
||||
druid.database.user=user
|
||||
druid.database.password=password
|
||||
# time between polling for metadata database
|
||||
druid.database.poll.duration=PT1M
|
||||
druid.database.segmentTable=prod_segments
|
||||
|
||||
# Path on local FS for storage of segments; dir will be created if needed
|
||||
druid.paths.indexCache=/tmp/druid/indexCache
|
||||
# Path on local FS for storage of segment metadata; dir will be created if needed
|
||||
druid.paths.segmentInfoCache=/tmp/druid/segmentInfoCache
|
||||
|
||||
druid.request.logging.dir=/tmp/druid/log
|
||||
|
||||
druid.server.maxSize=300000000000
|
||||
|
||||
# ZK quorum IPs
|
||||
druid.zk.service.host=
|
||||
# ZK path prefix for Druid-usage of zookeeper, Druid will create multiple paths underneath this znode
|
||||
druid.zk.paths.base=/druid
|
||||
# ZK path for discovery, the only path not to default to anything
|
||||
druid.zk.paths.discoveryPath=/druid/discoveryPath
|
||||
|
||||
# the host:port as advertised to clients
|
||||
druid.host=someHostOrIPaddrWithPort
|
||||
# the port on which to listen, this port should line up with the druid.host value
|
||||
druid.port=8080
|
||||
|
||||
com.metamx.emitter.logging=true
|
||||
com.metamx.emitter.logging.level=debug
|
||||
|
||||
druid.processing.formatString=processing_%s
|
||||
druid.processing.numThreads=3
|
||||
|
||||
|
||||
druid.computation.buffer.size=100000000
|
||||
|
||||
# S3 dest for realtime indexer
|
||||
druid.storage.s3.bucket=
|
||||
druid.storage.s3.baseKey=
|
||||
|
||||
druid.bard.cache.sizeInBytes=40000000
|
||||
```
|
||||
|
||||
Configuration groupings
|
||||
-----------------------
|
||||
|
||||
### S3 Access
|
||||
|
||||
These properties are for connecting with S3 and using it to pull down segments. In the future, we plan on being able to use other deep storage file systems as well, like HDFS. The file system is actually only accessed by the [Historical](Historical.html), [Realtime](Realtime.html) and [Indexing service](Indexing service.html) nodes.
|
||||
The Druid servers emit various metrics and alerts via something we call an Emitter. There are two emitter implementations included with the code, one that just logs to log4j and one that does POSTs of JSON events to a server. The properties for using the logging emitter are described below.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`com.metamx.aws.accessKey`|The access key to use to access S3.|none|
|
||||
|`com.metamx.aws.secretKey`|The secret key to use to access S3.|none|
|
||||
|`druid.storage.s3.bucket`|The bucket to store segments, this is used by Realtime and the Indexing service.|none|
|
||||
|`druid.storage.s3.baseKey`|The base key to use when storing segments, this is used by Realtime and the Indexing service|none|
|
||||
|`druid.emitter`|Appending "logging" or "http" to this config will instantialize one of the emitter modules.|logging|
|
||||
|
||||
### JDBC connection
|
||||
|
||||
These properties specify the jdbc connection and other configuration around the "segments table" database. The only processes that connect to the DB with these properties are the [Coordinator](Coordinator.html) and [Indexing service](Indexing-service.html). This is tested on MySQL.
|
||||
#### Logging Emitter Module
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.database.connectURI`|The jdbc connection uri|none|
|
||||
|`druid.database.user`|The username to connect with|none|
|
||||
|`druid.database.password`|The password to connect with|none|
|
||||
|`druid.database.poll.duration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments|PT1M|
|
||||
|`druid.database.segmentTable`|The table to use to look for segments.|none|
|
||||
|`druid.database.ruleTable`|The table to use to look for segment load/drop rules.|none|
|
||||
|`druid.database.configTable`|The table to use to look for configs.|none|
|
||||
|`druid.emitter.logging.loggerClass`|Choices: HttpPostEmitter, LoggingEmitter, NoopServiceEmitter, ServiceEmitter. The class used for logging.|LoggingEmitter|
|
||||
|`druid.emitter.logging.logLevel`|Choices: debug, info, warn, error. The log level at which message are logged.|info|
|
||||
|
||||
### Coordinator properties
|
||||
#### Http Emitter Module
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.coordinator.period`|The run period for the coordinator. The coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs|PT60S|
|
||||
|`druid.coordinator.removedSegmentLifetime`|When a node disappears, the coordinator can provide a grace period for how long it waits before deciding that the node really isn’t going to come back and it really should declare that all segments from that node are no longer available. This sets that grace period in number of runs of the coordinator.|1|
|
||||
|`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data|PT600S|
|
||||
|`druid.emitter.http.timeOut`|The timeout for data reads.|PT5M|
|
||||
|`druid.emitter.http.flushMillis`|How often to internal message buffer is flushed (data is sent).|60000|
|
||||
|`druid.emitter.http.flushCount`|How many messages can the internal message buffer hold before flushing (sending).|500|
|
||||
|`druid.emitter.http.recipientBaseUrl`|The base URL to emit messages to.|none|
|
||||
|
||||
### Zk properties
|
||||
### Http Client Module
|
||||
|
||||
See [ZooKeeper](ZooKeeper.html) for a description of these properties.
|
||||
|
||||
### Service properties
|
||||
|
||||
These are properties that define various service/HTTP server aspects
|
||||
This is the HTTP client used by [Broker](Broker.html) nodes.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to historical and real-time nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|5|
|
||||
|`druid.broker.http.readTimeout`|The timeout for data reads.|none|
|
||||
|
||||
### Curator Module
|
||||
|
||||
Druid uses [Curator](http://curator.incubator.apache.org/) for all [Zookeeper](http://zookeeper.apache.org/) interactions.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.zk.service.host`|The Zookeeper hosts to connect to.|none|
|
||||
|`druid.zk.service.sessionTimeoutMs`|Zookeeper session timeout.|30000|
|
||||
|`druid.curator.compress`|Boolean flag for whether or not created Znodes should be compressed.|false|
|
||||
|
||||
### Announcer Module
|
||||
|
||||
The announcer module is used to announce and unannounce Znodes in Zookeeper (using Curator).
|
||||
|
||||
#### Zookeeper Paths
|
||||
|
||||
See [Zookeeper](Zookeeper.html).
|
||||
|
||||
#### Data Segment Announcer
|
||||
|
||||
Data segment announcers are used to announce segments.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.announcer.type`|Choices: legacy or batch. The type of data segment announcer to use.|legacy|
|
||||
|
||||
#### Single Data Segment Announcer
|
||||
|
||||
In legacy Druid, each segment served by a node would be announced as an individual Znode.
|
||||
|
||||
#### Batch Data Segment Announcer
|
||||
|
||||
In current Druid, multiple data segments may be announced under the same Znode.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.announcer.segmentsPerNode`|Each Znode contains info for up to this many segments.|50|
|
||||
|`druid.announcer.maxBytesPerNode`|Max byte size for Znode.|524288|
|
||||
|
||||
### Druid Processing Module
|
||||
|
||||
This module contains query processing functionality.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)|
|
||||
|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|processing-%s|
|
||||
|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, this means that even under heavy load there will still be one core available to do background tasks like talking with ZK and pulling down segments.|1|
|
||||
|
||||
### AWS Module
|
||||
|
||||
This module is used to interact with S3.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.s3.accessKey`|The access key to use to access S3.|none|
|
||||
|`druid.s3.secretKey`|The secret key to use to access S3.|none|
|
||||
|
||||
### Metrics Module
|
||||
|
||||
The metrics module is used to track Druid metrics.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.monitoring.emissionPeriod`|How often metrics are emitted.|PT1m|
|
||||
|`druid.monitoring.monitors`|List of Druid monitors.|none|
|
||||
|
||||
### Server Module
|
||||
|
||||
This module is used for Druid server nodes.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.client.http.connections`|Size of connection pool for the Broker to connect to historical nodes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|none|
|
||||
|`druid.paths.indexCache`|Segments assigned to a historical node are first stored on the local file system and then served by the historical node. This path defines where that local cache resides. Directory will be created if needed|none|
|
||||
|`druid.paths.segmentInfoCache`|historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed|none|
|
||||
|`druid.http.numThreads`|The number of HTTP worker threads.|10|
|
||||
|`druid.http.maxIdleTimeMillis`|The amount of time a connection can remain idle before it is terminated|300000 (5 min)|
|
||||
|`druid.request.logging.dir`|Historical, Realtime and Broker nodes maintain request logs of all of the requests they get (interacton is via POST, so normal request logs don’t generally capture information about the actual query), this specifies the directory to store the request logs in|none|
|
||||
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|none|
|
||||
|`druid.port`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|none|
|
||||
|`druid.processing.formatString`|Realtime and historical nodes use this format string to name their processing threads.|none|
|
||||
|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, this means that even under heavy load there will still be one core available to do background tasks like talking with ZK and pulling down segments.|none|
|
||||
|`druid.computation.buffer.size`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|1073741824 (1GB)|
|
||||
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|none|
|
||||
|`druid.bard.cache.sizeInBytes`|The Broker (called Bard internally) instance has the ability to store results of queries in an in-memory cache. This specifies the number of bytes to use for that cache|none|
|
||||
|
||||
### Historical Node Properties
|
||||
### Storage Node Module
|
||||
|
||||
These are properties that the historical nodes use
|
||||
This module is used by nodes that store data (historical and real-time nodes).
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.server.maxSize`|The maximum number of bytes worth of segment that the node wants assigned to it. This is not a limit that the historical nodes actually enforce, they just publish it to the coordinator and trust the coordinator to do the right thing|none|
|
||||
|`druid.server.type`|Specifies the type of the node. This is published via ZK and depending on the value the node will be treated specially by the Coordinator/Broker. Allowed values are "realtime" or "historical". This is a configuration parameter because the plan is to allow for a more configurable cluster composition. At the current time, all realtime nodes should just be "realtime" and all historical nodes should just be "historical"|none|
|
||||
|`druid.server.maxSize`|The maximum number of bytes worth of segments that the node wants assigned to it. This is not a limit that the historical nodes actually enforce, they just publish it to the coordinator and trust the coordinator to do the right thing|0|
|
||||
|`druid.server.tier`|Druid server host port.|none|
|
||||
|
||||
### Emitter Properties
|
||||
#### Segment Cache
|
||||
|
||||
The Druid servers emit various metrics and alerts via something we call an [Emitter](Emitter.html). There are two emitter implementations included with the code, one that just logs to log4j and one that does POSTs of JSON events to a server. More information can be found on the [Emitter](Emitter.html) page. The properties for using the logging emitter are described below.
|
||||
Druid storage nodes maintain information about segments they have already downloaded.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`com.metamx.emitter.logging`|Set to "true" to use the logging emitter|none|
|
||||
|`com.metamx.emitter.logging.level`|Sets the level to log at|debug|
|
||||
|`com.metamx.emitter.logging.class`|Sets the class to log at|com.metamx.emiter.core.LoggingEmitter|
|
||||
|`druid.segmentCache.locations`|Segments assigned to a historical node are first stored on the local file system and then served by the historical node. These locations defines where that local cache resides|none|
|
||||
|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true|
|
||||
|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir|
|
||||
|
||||
### Realtime Properties
|
||||
### Jetty Server Module
|
||||
|
||||
Druid uses Jetty to serve HTTP requests.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.realtime.specFile`|The file with realtime specifications in it. See [Realtime](Realtime.html).|none|
|
||||
|`druid.server.http.numThreads`|Number of threads for HTTP requests.|10|
|
||||
|`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5m|
|
||||
|
||||
### Queryable Module
|
||||
|
||||
This module is used by all nodes that can serve queries.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.request.logging.type`|Choices: noop, file, emitter. How to log every request.|noop|
|
||||
|
||||
#### File Request Logging
|
||||
|
||||
Daily request logs are stored on disk.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.request.logging.dir`|Historical, Realtime and Broker nodes maintain request logs of all of the requests they get (interacton is via POST, so normal request logs don’t generally capture information about the actual query), this specifies the directory to store the request logs in|none|
|
||||
|
||||
#### Emitter Request Logging
|
||||
|
||||
Every request is emitted to some external location.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.request.logging.feed`|Feed name for requests.|none|
|
||||
|
||||
### Query Runner Factory Module
|
||||
|
||||
This module is required by nodes that can serve queries.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.query.chunkPeriod`|Long interval queries may be broken into shorter interval queries.|P1M|
|
||||
|
||||
#### GroupBy Query Config
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.query.groupBy.singleThreaded`|Run single threaded group By queries.|false|
|
||||
|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows.|50000|
|
||||
|`druid.query.groupBy.maxResults`|Maximum number of results.|500000|
|
||||
|
||||
|
||||
#### Search Query Config
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.query.search.maxSearchLimit`|Maximum number of search results to return.|1000|
|
||||
|
||||
### Discovery Module
|
||||
|
||||
The discovery module is used for service discovery.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.discovery.curator.path`|Services announce themselves under this Zookeeper path.|/druid/discovery|
|
||||
|
||||
### Server Inventory View Module
|
||||
|
||||
This module is used to read announcements of segments in Zookeeper. The configs are identical to the Announcer Module.
|
||||
|
||||
### Database Connector Module
|
||||
|
||||
These properties specify the jdbc connection and other configuration around the database. The only processes that connect to the DB with these properties are the [Coordinator](Coordinator.html) and [Indexing service](Indexing-service.html). This is tested on MySQL.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.db.connector.pollDuration`|The jdbc connection URI.|none|
|
||||
|`druid.db.connector.user`|The username to connect with.|none|
|
||||
|`druid.db.connector.password`|The password to connect with.|none|
|
||||
|`druid.db.connector.createTables`|If Druid requires a table and it doesn't exist, create it?|true|
|
||||
|`druid.db.connector.useValidationQuery`|Validate a table with a query.|false|
|
||||
|`druid.db.connector.validationQuery`|The query to validate with.|SELECT 1|
|
||||
|`druid.db.tables.base`|The base name for tables.|druid|
|
||||
|`druid.db.tables.segmentTable`|The table to use to look for segments.|druid_segments|
|
||||
|`druid.db.tables.ruleTable`|The table to use to look for segment load/drop rules.|druid_rules|
|
||||
|`druid.db.tables.configTable`|The table to use to look for configs.|druid_config|
|
||||
|`druid.db.tables.tasks`|Used by the indexing service to store tasks.|druid_tasks|
|
||||
|`druid.db.tables.taskLog`|Used by the indexing service to store task logs.|druid_taskLog|
|
||||
|`druid.db.tables.taskLock`|Used by the indexing service to store task locks.|druid_taskLock|
|
||||
|
||||
### Jackson Config Manager Module
|
||||
|
||||
The Jackson Config manager reads and writes config entries from the Druid config table using [Jackson](http://jackson.codehaus.org/).
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1m|
|
||||
|
||||
### Indexing Service Discovery Module
|
||||
|
||||
This module is used to find the [Indexing Service](Indexing-Service.html) using Curator service discovery.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.selectors.indexing.serviceName`|The druid.service name of the indexing service Overlord node.|none|
|
||||
|
||||
### DataSegment Pusher/Puller Module
|
||||
|
||||
This module is used to configure Druid deep storage. The configurations concern how to push and pull [Segments](Segments.html) from deep storage.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.storage.type`|Choices:local, noop, s3, hdfs, c*. The type of deep storage to use.|local|
|
||||
|
||||
#### Local Deep Storage
|
||||
|
||||
Local deep storage uses the local filesystem.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.storage.storageDirectory`|Directory on disk to use as deep storage.|/tmp/druid/localStorage|
|
||||
|
||||
#### Noop Deep Storage
|
||||
|
||||
This deep storage doesn't do anything. There are no configs.
|
||||
|
||||
#### S3 Deep Storage
|
||||
|
||||
This deep storage is used to interface with Amazon's S3.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.storage.bucket`|S3 bucket name.|none|
|
||||
|`druid.storage.basekey`|S3 base key.|none|
|
||||
|`druid.storage.disableAcl`|Boolean flag for ACL.|false|
|
||||
|
||||
#### HDFS Deep Storage
|
||||
|
||||
This deep storage is used to interface with HDFS.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.storage.storageDirectory`|HDFS directory to use as deep storage.|none|
|
||||
|
||||
#### Cassandra Deep Storage
|
||||
|
||||
This deep storage is used to interface with Cassandra.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.storage.host`|Cassandra host.|none|
|
||||
|`druid.storage.keyspace`|Cassandra key space.|none|
|
||||
|
||||
### Task Log Module
|
||||
|
||||
This module is used to configure the [Indexing Service](Indexing-Service.html) task logs.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.logs.type`|Choices:noop, S3. Where to store task logs|noop|
|
||||
|
||||
#### Noop Task Logs
|
||||
|
||||
No task logs are actually stored.
|
||||
|
||||
#### S3 Task Logs
|
||||
|
||||
Store Task Logs in S3.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.logs.s3Bucket`|S3 bucket name.|none|
|
||||
|`druid.indexer.logs.s3Prefix`|S3 key prefix.|none|
|
||||
|
||||
### Firehose Module
|
||||
|
||||
The Firehose module lists all available firehoses. There are no configurations.
|
||||
|
|
|
@ -2,7 +2,7 @@
|
|||
layout: doc_page
|
||||
---
|
||||
Coordinator
|
||||
======
|
||||
===========
|
||||
|
||||
The Druid coordinator node is primarily responsible for segment management and distribution. More specifically, the Druid coordinator node communicates to historical nodes to load or drop segments based on configurations. The Druid coordinator is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load.
|
||||
|
||||
|
@ -10,6 +10,96 @@ The Druid coordinator runs periodically and the time between each run is a confi
|
|||
|
||||
Before any unassigned segments are serviced by historical nodes, the available historical nodes for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always assigned to the nodes with least capacity to maintain a level of balance between nodes. The coordinator does not directly communicate with a historical node when assigning it a new segment; instead the coordinator creates some temporary information about the new segment under load queue path of the historical node. Once this request is seen, the historical node will load the segment and begin servicing it.
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server coordinator
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=coordinator
|
||||
druid.port=8082
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.coordinator.startDelay=PT60s
|
||||
```
|
||||
|
||||
JVM Configuration
|
||||
-----------------
|
||||
|
||||
The coordinator module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.coordinator.period`|The run period for the coordinator. The coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S|
|
||||
|`druid.coordinator.period.indexingPeriod`|How often to send indexing tasks to the indexing service. Only applies if merge or conversion is turned on.|PT1800S (30 mins)|
|
||||
|`druid.coordinator.removedSegmentLifetime`|When a node disappears, the coordinator can provide a grace period for how long it waits before deciding that the node really isn’t going to come back and it really should declare that all segments from that node are no longer available. This sets that grace period in number of runs of the coordinator.|1|
|
||||
|`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S|
|
||||
|`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|PT300S|
|
||||
|`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false|
|
||||
|`druid.coordinator.load.timeout`|The timeout duration for when the coordinator assigns a segment to a historical node.|15 minutes|
|
||||
|`druid.manager.segment.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the coordinator to notice new segments.|PT1M|
|
||||
|`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the coordinator to notice rules.|PT1M|
|
||||
|`druid.manager.rules.defaultTier`|The default tier from which default rules will be loaded from.|_default|
|
||||
|
||||
Dynamic Configuration
|
||||
---------------------
|
||||
|
||||
The coordinator has dynamic configuration to change certain behaviour on the fly. The coordinator a JSON spec object from the Druid [MySQL](MySQL.html) config table. This object is detailed below:
|
||||
|
||||
It is recommended that you use the Coordinator Console to configure these parameters. However, if you need to do it via HTTP, the JSON object can be submitted to the overlord via a POST request at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<PORT>/coordinator/config
|
||||
```
|
||||
|
||||
A sample worker setup spec is shown below:
|
||||
|
||||
```json
|
||||
{
|
||||
"millisToWaitBeforeDeleting": 900000,
|
||||
"mergeBytesLimit": 100000000L,
|
||||
"mergeSegmentsLimit" : 1000,
|
||||
"maxSegmentsToMove": 5,
|
||||
"replicantLifetime": 15,
|
||||
"replicationThrottleLimit": 10,
|
||||
"emitBalancingStats": false
|
||||
}
|
||||
```
|
||||
|
||||
Issuing a GET request at the same URL will return the spec that is currently in place. A description of the config setup spec is shown below.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`millisToWaitBeforeDeleting`|How long does the coordinator need to be active before it can start deleting segments.|90000 (15 mins)|
|
||||
|`mergeBytesLimit`|The maximum number of bytes to merge (for segments).|100000000L|
|
||||
|`mergeSegmentsLimit`|The maximum number of segments that can be in a single merge [task](Tasks.html).|Integer.MAX_VALUE|
|
||||
|`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|5|
|
||||
|`replicantLifetime`|The maximum number of coordinator runs for a segment to be replicated before we start alerting.|15|
|
||||
|`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10|
|
||||
|`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false|
|
||||
|
||||
### Running
|
||||
|
||||
```
|
||||
io.druid.cli.Main server coordinator
|
||||
```
|
||||
|
||||
Rules
|
||||
-----
|
||||
|
||||
|
@ -104,7 +194,13 @@ The coordinator node exposes several HTTP endpoints for interactions.
|
|||
The Coordinator Console
|
||||
------------------
|
||||
|
||||
The Druid coordinator exposes a web GUI for displaying cluster information and rule configuration. After the coordinator starts, the console can be accessed at http://<HOST>:<PORT>. There exists a full cluster view, as well as views for individual historical nodes, datasources and segments themselves. Segment information can be displayed in raw JSON form or as part of a sortable and filterable table.
|
||||
The Druid coordinator exposes a web GUI for displaying cluster information and rule configuration. After the coordinator starts, the console can be accessed at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<COORDINATOR_PORT>
|
||||
```
|
||||
|
||||
There exists a full cluster view, as well as views for individual historical nodes, datasources and segments themselves. Segment information can be displayed in raw JSON form or as part of a sortable and filterable table.
|
||||
|
||||
The coordinator console also exposes an interface to creating and editing rules. All valid datasources configured in the segment database, along with a default datasource, are available for configuration. Rules of different types can be added, deleted or edited.
|
||||
|
||||
|
@ -123,14 +219,4 @@ FAQ
|
|||
|
||||
No. If the Druid coordinator is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the coordinator node can be started up at any time, and after a configurable delay, will start running coordinator tasks.
|
||||
|
||||
This also means that if you have a working cluster and all of your coordinators die, the cluster will continue to function, it just won’t experience any changes to its data topology.
|
||||
|
||||
Running
|
||||
-------
|
||||
|
||||
Coordinator nodes can be run using the `io.druid.cli.Main` class with program parameters "server coordinator".
|
||||
|
||||
Configuration
|
||||
-------------
|
||||
|
||||
See [Configuration](Configuration.html).
|
||||
This also means that if you have a working cluster and all of your coordinators die, the cluster will continue to function, it just won’t experience any changes to its data topology.
|
|
@ -0,0 +1,37 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
|
||||
# Data Flow
|
||||
|
||||
The diagram below illustrates how different Druid nodes download data and respond to queries:
|
||||
|
||||
<img src="../img/druid-dataflow-2x.png" width="800"/>
|
||||
|
||||
### Real-time Nodes
|
||||
|
||||
Real-time nodes ingest streaming data and announce themselves and the segments they are serving in Zookeeper on start up. During the segment hand-off stage, real-time nodes create a segment metadata entry in MySQL for the segment to hand-off. This segment is uploaded to Deep Storage. Real-time nodes use Zookeeper to monitor when historical nodes complete downloading the segment (indicating hand-off completion) so that it can forget about it. Real-time nodes also respond to query requests from broker nodes and also return query results to the broker nodes.
|
||||
|
||||
### Deep Storage
|
||||
|
||||
Batch indexed segments and segments created by real-time nodes are uploaded to deep storage. Historical nodes download these segments to serve for queries.
|
||||
|
||||
### MySQL
|
||||
|
||||
Real-time nodes and batch indexing create new segment metadata entries for the new segments they've created. Coordinator nodes read this metadata table to determine what segments should be loaded in the cluster.
|
||||
|
||||
### Coordinator Nodes
|
||||
|
||||
Coordinator nodes read segment metadata information from MySQL to determine what segments should be loaded in the cluster. Coordinator nodes user Zookeeper to determine what historical nodes exist, and also create Zookeeper entries to tell historical nodes to load and drop new segments.
|
||||
|
||||
### Zookeeper
|
||||
|
||||
Real-time nodes announce themselves and the segments they are serving in Zookeeper and also use Zookeeper to monitor segment hand-off. Coordinator nodes use Zookeeper to determine what historical nodes exist in the cluster and create new entries to communicate to historical nodes to load or drop new data. Historical nodes announce themselves and the segments they serve in Zookeeper. Historical nodes also monitor Zookeeper for new load or drop requests. Broker nodes use Zookeeper to determine what historical and real-time nodes exist in the cluster.
|
||||
|
||||
### Historical Nodes
|
||||
|
||||
Historical nodes announce themselves and the segments they are serving in Zookeeper. Historical nodes also use Zookeeper to monitor for signals to load or drop new segments. Historical nodes download segments from deep storage, respond to the queries from broker nodes about these segments, and return results to the broker nodes.
|
||||
|
||||
### Broker Nodes
|
||||
|
||||
Broker nodes receive queries from external clients and forward those queries down to real-time and historical nodes. When the individual nodes return their results, broker nodes merge these results and returns them to the caller. Broker nodes use Zookeeper to determine what real-time and historical nodes exist.
|
|
@ -12,21 +12,21 @@ S3-compatible deep storage is basically either S3 or something like riak-cs whic
|
|||
S3 configuration parameters are
|
||||
|
||||
```
|
||||
com.metamx.aws.accessKey=<S3 access key>
|
||||
com.metamx.aws.secretKey=<S3 secret_key>
|
||||
druid.storage.s3.bucket=<bucket to store in>
|
||||
druid.storage.s3.baseKey=<base key prefix to use, i.e. what directory>
|
||||
druid.s3.accessKey=<S3 access key>
|
||||
druid.s3.secretKey=<S3 secret_key>
|
||||
druid.storage.bucket=<bucket to store in>
|
||||
druid.storage.baseKey=<base key prefix to use, i.e. what directory>
|
||||
```
|
||||
|
||||
## HDFS
|
||||
|
||||
As of 0.4.0, HDFS can be used for storage of segments as well.
|
||||
|
||||
In order to use hdfs for deep storage, you need to set the following configuration on your realtime nodes.
|
||||
In order to use hdfs for deep storage, you need to set the following configuration on your real-time nodes.
|
||||
|
||||
```
|
||||
druid.storage.hdfs=true
|
||||
druid.storage.hdfs.storageDirectory=<directory for storing segments>
|
||||
druid.storage.type=hdfs
|
||||
druid.storage.storageDirectory=<directory for storing segments>
|
||||
```
|
||||
|
||||
If you are using the Hadoop indexer, set your output directory to be a location on Hadoop and it will work
|
||||
|
@ -36,13 +36,13 @@ If you are using the Hadoop indexer, set your output directory to be a location
|
|||
|
||||
A local mount can be used for storage of segments as well. This allows you to use just your local file system or anything else that can be mount locally like NFS, Ceph, etc.
|
||||
|
||||
In order to use a local mount for deep storage, you need to set the following configuration on your realtime nodes.
|
||||
In order to use a local mount for deep storage, you need to set the following configuration on your real-time nodes.
|
||||
|
||||
```
|
||||
druid.storage.local=true
|
||||
druid.storage.local.storageDirectory=<directory for storing segments>
|
||||
druid.storage.type=local
|
||||
druid.storage.storageDirectory=<directory for storing segments>
|
||||
```
|
||||
|
||||
Note that you should generally set `druid.storage.local.storageDirectory` to something different from `druid.paths.indexCache`.
|
||||
Note that you should generally set `druid.storage.storageDirectory` to something different from `druid.segmentCache.locations` and `druid.segmentCache.infoDir`.
|
||||
|
||||
If you are using the Hadoop indexer in local mode, then just give it a local file as your output directory and it will work.
|
||||
|
|
|
@ -1,6 +1,14 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
|
||||
Hadoop has shown the world that it’s possible to house your data warehouse on commodity hardware for a fraction of the price of typical solutions. As people adopt Hadoop for their data warehousing needs, they find two things.
|
||||
|
||||
1. They can now query all of their data in a fairly flexible manner and answer any question they have
|
||||
2. The queries take a long time
|
||||
|
||||
The first one is the joy that everyone feels the first time they get Hadoop running. The latter is what they realize after they have used Hadoop interactively for a while because Hadoop is optimized for throughput, not latency.
|
||||
|
||||
Druid is a complementary addition to Hadoop. Hadoop is great at storing and making accessible large amounts of individually low-value data. Unfortunately, Hadoop is not great at providing query speed guarantees on top of that data, nor does it have very good operational characteristics for a customer-facing production system. Druid, on the other hand, excels at taking high-value summaries of the low-value data on Hadoop, making it available in a fast and always-on fashion, such that it could be exposed directly to a customer.
|
||||
|
||||
Druid also requires some infrastructure to exist for [deep storage](Deep-Storage.html). HDFS is one of the implemented options for this [deep storage](Deep-Storage.html).
|
||||
|
|
|
@ -25,11 +25,11 @@ git checkout druid-0.6.0
|
|||
|
||||
### Downloading the DSK (Druid Standalone Kit)
|
||||
|
||||
[Download](http://static.druid.io/data/examples/druid-services-0.4.6.tar.gz) a stand-alone tarball and run it:
|
||||
[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.0-bin.tar.gz) a stand-alone tarball and run it:
|
||||
|
||||
``` bash
|
||||
tar -xzf druid-services-0.X.X-SNAPSHOT-bin.tar.gz
|
||||
cd druid-services-0.X.X-SNAPSHOT
|
||||
tar -xzf druid-services-0.X.X-bin.tar.gz
|
||||
cd druid-services-0.X.X
|
||||
```
|
||||
|
||||
Twitter Example
|
||||
|
|
|
@ -6,6 +6,49 @@ Historical
|
|||
|
||||
Historical nodes load up historical segments and expose them for querying.
|
||||
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
druid.host=localhost
|
||||
druid.service=historical
|
||||
druid.port=8081
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.server.maxSize=100000000
|
||||
|
||||
druid.processing.buffer.sizeBytes=10000000
|
||||
|
||||
druid.segmentCache.infoPath=/tmp/druid/segmentInfoCache
|
||||
druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 100000000}]```
|
||||
```
|
||||
|
||||
Note: This will spin up a Historical node with the local filesystem as deep storage.
|
||||
|
||||
JVM Configuration
|
||||
-----------------
|
||||
The historical module uses several of the default modules in [Configuration](Configuration.html) and has no uniques configs of its own.
|
||||
|
||||
Running
|
||||
-------
|
||||
|
||||
```
|
||||
io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
Loading and Serving Segments
|
||||
----------------------------
|
||||
|
||||
|
@ -27,14 +70,4 @@ Querying Segments
|
|||
|
||||
Please see [Querying](Querying.html) for more information on querying historical nodes.
|
||||
|
||||
For every query that a historical node services, it will log the query and report metrics on the time taken to run the query.
|
||||
|
||||
Running
|
||||
-------
|
||||
p
|
||||
Historical nodes can be run using the `io.druid.cli.Main` class with program arguments "server historical".
|
||||
|
||||
Configuration
|
||||
-------------
|
||||
|
||||
See [Configuration](Configuration.html).
|
||||
For every query that a historical node services, it will log the query and report metrics on the time taken to run the query.
|
|
@ -2,53 +2,63 @@
|
|||
layout: doc_page
|
||||
---
|
||||
|
||||
Druid is an open-source analytics datastore designed for realtime, exploratory, queries on large-scale data sets (100’s of Billions entries, 100’s TB data). Druid provides for cost effective, always-on, realtime data ingestion and arbitrary data exploration.
|
||||
Druid is an open-source analytics data store designed for real-time exploratory queries on large-scale data sets (100’s of Billions entries, 100’s TB data). Druid provides for cost-effective and always-on realtime data ingestion and arbitrary data exploration.
|
||||
|
||||
- Check out some [Examples](Examples.html)
|
||||
- Try out Druid with our Getting Started [Tutorial](./Tutorial%3A-A-First-Look-at-Druid.html)
|
||||
- Learn more by reading the [White Paper](http://static.druid.io/docs/druid.pdf)
|
||||
|
||||
Key Features
|
||||
------------
|
||||
|
||||
- **Designed for Analytics** - Druid is built for exploratory analytics for OLAP workflows. It supports a variety of filters, aggregators and query types and provides a framework for plugging in new functionality. Users have leveraged Druid’s infrastructure to develop features such as top K queries and histograms.
|
||||
- **Interactive Queries** - Druid’s low-latency data ingestion architecture allows events to be queried milliseconds after they are created. Druid’s query latency is optimized by reading and scanning only exactly what is needed. Aggregate and filter on data without sitting around waiting for results.
|
||||
- **Highly Available** - Druid is used to back SaaS implementations that need to be up all the time. Your data is still available and queryable during system updates. Scale up or down without data loss.
|
||||
- **Scalable** - Existing Druid deployments handle billions of events and terabytes of data per day. Druid is designed to be petabyte scale.
|
||||
|
||||
|
||||
Why Druid?
|
||||
----------
|
||||
|
||||
Druid was originally created to resolve query latency issues seen with trying to use Hadoop to power an interactive service. Hadoop has shown the world that it’s possible to house your data warehouse on commodity hardware for a fraction of the price of typical solutions. As people adopt Hadoop for their data warehousing needs, they find two things.
|
||||
Druid was originally created to resolve query latency issues seen with trying to use Hadoop to power an interactive service. It's especially useful if you are summarizing your data sets and then querying the summarizations. Put your summarizations into Druid and get quick queryability out of a system that you can be confident will scale up as your data volumes increase. Deployments have scaled up to 2TB of data per hour at peak ingested and aggregated in real-time.
|
||||
|
||||
1. They can now query all of their data in a fairly flexible manner and answer any question they have
|
||||
2. The queries take a long time
|
||||
|
||||
The first one is the joy that everyone feels the first time they get Hadoop running. The latter is what they realize after they have used Hadoop interactively for a while because Hadoop is optimized for throughput, not latency. Druid is a system that you can set up in your organization next to Hadoop. It provides the ability to access your data in an interactive slice-and-dice fashion. It trades off some query flexibility and takes over the storage format in order to provide the speed.
|
||||
|
||||
Druid is especially useful if you are summarizing your data sets and then querying the summarizations. If you put your summarizations into Druid, you will get quick queryability out of a system that you can be confident will scale up as your data volumes increase. Deployments have scaled up to 2TB of data per hour at peak ingested and aggregated in real-time.
|
||||
Druid is a system that you can set up in your organization next to Hadoop. It provides the ability to access your data in an interactive slice-and-dice fashion. It trades off some query flexibility and takes over the storage format in order to provide the speed.
|
||||
|
||||
We have more details about the general design of the system and why you might want to use it in our [White Paper](http://static.druid.io/docs/druid.pdf) or in our [Design](Design.html) doc.
|
||||
|
||||
The data store world is vast, confusing and constantly in flux. This page is meant to help potential evaluators decide whether Druid is a good fit for the problem one needs to solve. If anything about it is incorrect please provide that feedback on the mailing list or via some other means, we will fix this page.
|
||||
|
||||
#### When Druid?
|
||||
When Druid?
|
||||
----------
|
||||
|
||||
* You need to do interactive, fast, exploration of large amounts of data
|
||||
* You need analytics (not key value store)
|
||||
* You have a lot of data (10s of Billions of events added per day, 10s of TB of data added per day)
|
||||
* You want to do your analysis on data as it’s happening (realtime)
|
||||
* Your store needs to be always-on, 24x7x365 and years into the future.
|
||||
|
||||
#### Not Druid?
|
||||
|
||||
Not Druid?
|
||||
----------
|
||||
|
||||
* The amount of data you have can easily be handled by MySql
|
||||
* Your querying for individual entries or doing lookups (Not Analytics)
|
||||
* Batch is good enough
|
||||
* Canned queries is good enough
|
||||
* Downtime is no big deal
|
||||
|
||||
#### Druid vs…
|
||||
|
||||
Druid vs…
|
||||
----------
|
||||
|
||||
* [Druid-vs-Impala-or-Shark](Druid-vs-Impala-or-Shark.html)
|
||||
* [Druid-vs-Redshift](Druid-vs-Redshift.html)
|
||||
* [Druid-vs-Vertica](Druid-vs-Vertica.html)
|
||||
* [Druid-vs-Cassandra](Druid-vs-Cassandra.html)
|
||||
* [Druid-vs-Hadoop](Druid-vs-Hadoop.html)
|
||||
|
||||
Key Features
|
||||
------------
|
||||
|
||||
- **Designed for Analytics** - Druid is built for exploratory analytics for OLAP workflows (streamalytics). It supports a variety of filters, aggregators and query types and provides a framework for plugging in new functionality. Users have leveraged Druid’s infrastructure to develop features such as top K queries and histograms.
|
||||
- **Interactive Queries** - Druid’s low latency data ingestion architecture allows events to be queried milliseconds after they are created. Druid’s query latency is optimized by only reading and scanning exactly what is needed. Aggregate and filter on data without sitting around waiting for results.
|
||||
- **Highly Available** - Druid is used to back SaaS implementations that need to be up all the time. Your data is still available and queryable during system updates. Scale up or down without data loss.
|
||||
- **Scalable** - Existing Druid deployments handle billions of events and terabytes of data per day. Druid is designed to be petabyte scale.
|
||||
About This Page
|
||||
----------
|
||||
The data store world is vast, confusing and constantly in flux. This page is meant to help potential evaluators decide whether Druid is a good fit for the problem one needs to solve. If anything about it is incorrect please provide that feedback on the mailing list or via some other means so we can fix it.
|
||||
|
||||
|
||||
|
||||
|
|
|
@ -1,69 +1,145 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Disclaimer: We are still in the process of finalizing the indexing service and these configs are prone to change at any time. We will announce when we feel the indexing service and the configurations described are stable.
|
||||
The indexing service is a highly-available, distributed service that runs indexing related tasks. Indexing service [tasks](Tasks.html) create (and sometimes destroy) Druid [segments](Segments.html). The indexing service has a master/slave like architecture.
|
||||
|
||||
The indexing service is a distributed task/job queue. It accepts requests in the form of [Tasks](Tasks.html) and executes those tasks across a set of worker nodes. Worker capacity can be automatically adjusted based on the number of tasks pending in the system. The indexing service is highly available, has built in retry logic, and can backup per task logs in deep storage.
|
||||
The indexing service is composed of three main components: a peon component that can run a single task, a middle manager component that manages peons, and an overlord component that manages task distribution to middle managers.
|
||||
Overlords and middle managers may run on the same node or across multiple nodes while middle managers and peons always run on the same node.
|
||||
|
||||
The indexing service is composed of two main components, a coordinator node that manages task distribution and worker capacity, and worker nodes that execute tasks in separate JVMs.
|
||||
Quick Start
|
||||
----------------------------------------
|
||||
Run:
|
||||
|
||||
```
|
||||
io.druid.cli.Main server overlord
|
||||
```
|
||||
|
||||
With the following JVM configuration:
|
||||
|
||||
```
|
||||
-server
|
||||
-Xmx2g
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
-Ddruid.host=localhost
|
||||
-Ddruid.port=8080
|
||||
-Ddruid.service=overlord
|
||||
|
||||
-Ddruid.zk.service.host=localhost
|
||||
|
||||
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
|
||||
-Ddruid.db.connector.user=druid
|
||||
-Ddruid.db.connector.password=diurd
|
||||
|
||||
-Ddruid.selectors.indexing.serviceName=overlord
|
||||
-Ddruid.indexer.runner.javaOpts="-server -Xmx1g"
|
||||
-Ddruid.indexer.runner.startPort=8081
|
||||
-Ddruid.indexer.fork.property.druid.computation.buffer.size=268435456
|
||||
```
|
||||
|
||||
You can now submit simple indexing tasks to the indexing service.
|
||||
|
||||
<!--
|
||||
Preamble
|
||||
--------
|
||||
|
||||
The truth is, the indexing service is an experience that is difficult to characterize with words. When they asked me to write this preamble, I was taken aback. I wasn’t quite sure what exactly to write or how to describe this… entity. I accepted the job, as much for the challenge and inner growth as the money, and took to the mountains for reflection. Six months later, I knew I had it, I was done and had achieved the next euphoric victory in the continuous struggle that plagues my life. But, enough about me. This is about the indexing service.
|
||||
|
||||
The indexing service is philosophical transcendence, an infallible truth that will shape your soul, mold your character, and define your reality. The indexing service is creating world peace, playing with puppies, unwrapping presents on Christmas morning, cradling a loved one, and beating Goro in Mortal Kombat for the first time. The indexing service is sustainable economic growth, global propensity, and a world of transparent financial transactions. The indexing service is a true belieber. The indexing service is panicking because you forgot you signed up for a course and the big exam is in a few minutes, only to wake up and realize it was all a dream. What is the indexing service? More like what isn’t the indexing service. The indexing service is here and it is ready, but are you?
|
||||
-->
|
||||
|
||||
Indexer Coordinator Node
|
||||
------------------------
|
||||
Indexing Service Overview
|
||||
-------------------------
|
||||
|
||||
The indexer coordinator node exposes HTTP endpoints where tasks can be submitted by posting a JSON blob to specific endpoints. It can be started by launching IndexerCoordinatorMain.java. The indexer coordinator node can operate in local mode or remote mode. In local mode, the coordinator and worker run on the same host and port. In remote mode, worker processes run on separate hosts and ports.
|
||||
![Indexing Service](../img/indexing_service.png "Indexing Service")
|
||||
|
||||
Tasks can be submitted via POST requests to:
|
||||
Overlord Node
|
||||
-------------
|
||||
|
||||
The overlord node is responsible for accepting tasks, coordinating task distribution, creating locks around tasks, and returning statuses to callers.
|
||||
|
||||
#### Usage
|
||||
|
||||
Tasks are submitted to the overlord node in the form of JSON objects. Tasks can be submitted via POST requests to:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<port>/druid/indexer/v1/task
|
||||
http://<OVERLORD_IP>:<port>/druid/indexer/v1/task
|
||||
```
|
||||
|
||||
Tasks can cancelled via POST requests to:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<port>/druid/indexer/v1/task/{taskId}/shutdown
|
||||
http://<OVERLORD_IP>:<port>/druid/indexer/v1/task/{taskId}/shutdown
|
||||
```
|
||||
|
||||
Issuing the cancel request once sends a graceful shutdown request. Graceful shutdowns may not stop a task right away, but instead issue a safe stop command at a point deemed least impactful to the system. Issuing the cancel request twice in succession will kill –9 the task.
|
||||
Issuing the cancel request will kill –9 the task.
|
||||
|
||||
Task statuses can be retrieved via GET requests to:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<port>/druid/indexer/v1/task/{taskId}/status
|
||||
http://<OVERLORD_IP>:<port>/druid/indexer/v1/task/{taskId}/status
|
||||
```
|
||||
|
||||
Task segments can be retrieved via GET requests to:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<port>/druid/indexer/v1/task/{taskId}/segments
|
||||
http://<OVERLORD_IP>:<port>/druid/indexer/v1/task/{taskId}/segments
|
||||
```
|
||||
|
||||
When a task is submitted, the coordinator creates a lock over the data source and interval of the task. The coordinator also stores the task in a MySQL database table. The database table is read at startup time to bootstrap any tasks that may have been submitted to the coordinator but may not yet have been executed.
|
||||
#### Console
|
||||
|
||||
The coordinator also exposes a simple UI to show what tasks are currently running on what nodes at
|
||||
The overlord console can be used to view pending tasks, running tasks, available workers, and recent worker creation and termination. The console can be accessed at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<port>/static/console.html
|
||||
http://<OVERLORD_IP>:8080/console.html
|
||||
```
|
||||
|
||||
#### Task Execution
|
||||
|
||||
The coordinator retrieves worker setup metadata from the Druid [MySQL](MySQL.html) config table. This metadata contains information about the version of workers to create, the maximum and minimum number of workers in the cluster at one time, and additional information required to automatically create workers.
|
||||
|
||||
Tasks are assigned to workers by creating entries under specific /tasks paths associated with a worker, similar to how the Druid coordinator node assigns segments to historical nodes. See [Worker Configuration](Indexing-Service#configuration-1). Once a worker picks up a task, it deletes the task entry and announces a task status under a /status path associated with the worker. Tasks are submitted to a worker until the worker hits capacity. If all workers in a cluster are at capacity, the indexer coordinator node automatically creates new worker resources.
|
||||
|
||||
#### Autoscaling
|
||||
|
||||
The Autoscaling mechanisms currently in place are tightly coupled with our deployment infrastructure but the framework should be in place for other implementations. We are highly open to new implementations or extensions of the existing mechanisms. In our own deployments, worker nodes are Amazon AWS EC2 nodes and they are provisioned to register themselves in a [galaxy](https://github.com/ning/galaxy) environment.
|
||||
The Autoscaling mechanisms currently in place are tightly coupled with our deployment infrastructure but the framework should be in place for other implementations. We are highly open to new implementations or extensions of the existing mechanisms. In our own deployments, middle manager nodes are Amazon AWS EC2 nodes and they are provisioned to register themselves in a [galaxy](https://github.com/ning/galaxy) environment.
|
||||
|
||||
The Coordinator node controls the number of workers in the cluster according to a worker setup spec that is submitted via a POST request to the indexer at:
|
||||
If autoscaling is enabled, new middle managers may be added when a task has been in pending state for too long. Middle managers may be terminated if they have not run any tasks for a period of time.
|
||||
|
||||
#### JVM Configuration
|
||||
|
||||
In addition to the configuration of some of the default modules in [Configuration](Configuration.html), the overlord module requires the following basic configs to run in remote mode:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.runner.type`|Choices "local" or "remote". Indicates whether tasks should be run locally or in a distributed environment.|local|
|
||||
|`druid.indexer.storage.type`|Choices are "local" or "db". Indicates whether incoming tasks should be stored locally (in heap) or in a database. Storing incoming tasks in a database allows for tasks to be bootstrapped if the overlord should fail.|local|
|
||||
|
||||
The following configs only apply if the overlord is running in remote mode:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.runner.taskAssignmentTimeout`|How long to wait after a task as been assigned to a middle manager before throwing an error.|PT5M|
|
||||
|`druid.indexer.runner.minWorkerVersion`|The minimum middle manager version to send tasks to. |none|
|
||||
|`druid.indexer.runner.compressZnodes`|Indicates whether or not the overlord should expect middle managers to compress Znodes.|false|
|
||||
|`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288|
|
||||
|
||||
There are additional configs for autoscaling (if it is enabled):
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.autoscale.doAutoscale`|If set to "true" autoscaling will be enabled.|false|
|
||||
|`druid.indexer.autoscale.provisionPeriod`|How often to check whether or not new middle managers should be added.|PT1M|
|
||||
|`druid.indexer.autoscale.terminatePeriod`|How often to check when middle managers should be removed.|PT1H|
|
||||
|`druid.indexer.autoscale.originTime`|The starting reference timestamp that the terminate period increments upon.|2012-01-01T00:55:00.000Z|
|
||||
|`druid.indexer.autoscale.strategy`|Choices are "noop" or "ec2". Sets the strategy to run when autoscaling is required.|noop|
|
||||
|`druid.indexer.autoscale.workerIdleTimeout`|How long can a worker be idle (not a run task) before it can be considered for termination.|PT10M|
|
||||
|`druid.indexer.autoscale.maxScalingDuration`|How long the overlord will wait around for a middle manager to show up before giving up.|PT15M|
|
||||
|`druid.indexer.autoscale.numEventsToTrack`|The number of autoscaling related events (node creation and termination) to track.|10|
|
||||
|`druid.indexer.autoscale.pendingTaskTimeout`|How long a task can be in "pending" state before the overlord tries to scale up.|PT30S|
|
||||
|`druid.indexer.autoscale.workerVersion`|If set, will only create nodes of set version during autoscaling. Overrides dynamic configuration. |null|
|
||||
|`druid.indexer.autoscale.workerPort`|The port that middle managers will run on.|8080|
|
||||
|
||||
#### Dynamic Configuration
|
||||
|
||||
Overlord dynamic configuration is mainly for autoscaling. The overlord reads a worker setup spec as a JSON object from the Druid [MySQL](MySQL.html) config table. This object contains information about the version of middle managers to create, the maximum and minimum number of middle managers in the cluster at one time, and additional information required to automatically create middle managers.
|
||||
|
||||
The JSON object can be submitted to the overlord via a POST request at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<port>/druid/indexer/v1/worker/setup
|
||||
|
@ -71,7 +147,7 @@ http://<COORDINATOR_IP>:<port>/druid/indexer/v1/worker/setup
|
|||
|
||||
A sample worker setup spec is shown below:
|
||||
|
||||
```
|
||||
```json
|
||||
{
|
||||
"minVersion":"some_version",
|
||||
"minNumWorkers":"0",
|
||||
|
@ -94,7 +170,7 @@ A sample worker setup spec is shown below:
|
|||
}
|
||||
```
|
||||
|
||||
Issuing a GET request at the same URL will return the current worker setup spec that is currently in place. The worker setup spec list above is just a sample and it is possible to write worker setup specs for other deployment environments. A description of the worker setup spec is shown below.
|
||||
Issuing a GET request at the same URL will return the current worker setup spec that is currently in place. The worker setup spec list above is just a sample and it is possible to extend the code base for other deployment environments. A description of the worker setup spec is shown below.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|
@ -104,109 +180,85 @@ Issuing a GET request at the same URL will return the current worker setup spec
|
|||
|`nodeData`|A JSON object that contains metadata about new nodes to create.|none|
|
||||
|`userData`|A JSON object that contains metadata about how the node should register itself on startup. This data is sent with node creation requests.|none|
|
||||
|
||||
For more information about configuring Auto-scaling, see [Auto-Scaling Configuration](https://github.com/metamx/druid/wiki/Indexing-Service#auto-scaling-configuration).
|
||||
#### Running
|
||||
|
||||
```
|
||||
io.druid.cli.Main server overlord
|
||||
```
|
||||
|
||||
Note: When running the overlord in local mode, all middle manager and peon configurations must be provided as well.
|
||||
|
||||
MiddleManager Node
|
||||
------------------
|
||||
|
||||
The middle manager node is a worker node that executes submitted tasks. Middle Managers forward tasks to peons that run in separate JVMs. Each peon is capable of running only one task at a time, however, a middle manager may have multiple peons.
|
||||
|
||||
#### JVM Configuration
|
||||
|
||||
Middle managers pass their configurations down to their child peons. The middle manager module requires the following configs:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.worker.ip`|The IP of the worker.|localhost|
|
||||
|`druid.worker.version`|Version identifier for the middle manager.|0|
|
||||
|`druid.worker.capacity`|Maximum number of tasks the middle manager can accept.|Number of available processors - 1|
|
||||
|`druid.indexer.runner.compressZnodes`|Indicates whether or not the middle managers should compress Znodes.|false|
|
||||
|`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288|
|
||||
|`druid.indexer.runner.taskDir`|Temporary intermediate directory used during task execution.|/tmp/persistent|
|
||||
|`druid.indexer.runner.javaCommand`|Command required to execute java.|java|
|
||||
|`druid.indexer.runner.javaOpts`|-X Java options to run the peon in its own JVM.|""|
|
||||
|`druid.indexer.runner.classpath`|Java classpath for the peon.|System.getProperty("java.class.path")|
|
||||
|`druid.indexer.runner.startPort`|The port that peons begin running on.|8080|
|
||||
|`druid.indexer.runner.allowedPrefixes`|Whitelist of prefixes for configs that can be passed down to child peons.|"com.metamx", "druid", "io.druid", "user.timezone","file.encoding"|
|
||||
|
||||
#### Running
|
||||
|
||||
Indexer Coordinator nodes can be run using the `com.metamx.druid.indexing.coordinator.http.IndexerCoordinatorMain` class.
|
||||
|
||||
#### Configuration
|
||||
|
||||
Indexer Coordinator nodes require [basic service configuration](https://github.com/metamx/druid/wiki/Configuration#basic-service-configuration). In addition, there are several extra configurations that are required.
|
||||
|
||||
```
|
||||
-Ddruid.zk.paths.indexer.announcementsPath=/druid/indexer/announcements
|
||||
-Ddruid.zk.paths.indexer.leaderLatchPath=/druid/indexer/leaderLatchPath
|
||||
-Ddruid.zk.paths.indexer.statusPath=/druid/indexer/status
|
||||
-Ddruid.zk.paths.indexer.tasksPath=/druid/demo/indexer/tasks
|
||||
|
||||
-Ddruid.indexer.runner=remote
|
||||
-Ddruid.indexer.taskDir=/mnt/persistent/task/
|
||||
-Ddruid.indexer.configTable=sample_config
|
||||
-Ddruid.indexer.workerSetupConfigName=worker_setup
|
||||
-Ddruid.indexer.strategy=ec2
|
||||
-Ddruid.indexer.hadoopWorkingPath=/tmp/druid-indexing
|
||||
-Ddruid.indexer.logs.s3bucket=some_bucket
|
||||
-Ddruid.indexer.logs.s3prefix=some_prefix
|
||||
io.druid.cli.Main server middleManager
|
||||
```
|
||||
|
||||
The indexing service requires some additional Zookeeper configs.
|
||||
Peons
|
||||
-----
|
||||
Peons run a single task in a single JVM. Peons are a part of middle managers and should rarely (if ever) be run on their own.
|
||||
|
||||
#### JVM Configuration
|
||||
Although peons inherit the configurations of their parent middle managers, explicit child peon configs can be set by prefixing them with:
|
||||
|
||||
```
|
||||
druid.indexer.fork.property
|
||||
```
|
||||
|
||||
Additional peon configs include:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.zk.paths.indexer.announcementsPath`|The base path where workers announce themselves.|none|
|
||||
|`druid.zk.paths.indexer.leaderLatchPath`|The base that coordinator nodes use to determine a leader.|none|
|
||||
|`druid.zk.paths.indexer.statusPath`|The base path where workers announce task statuses.|none|
|
||||
|`druid.zk.paths.indexer.tasksPath`|The base path where the coordinator assigns new tasks.|none|
|
||||
|`druid.peon.mode`|Choices are "local" and "remote". Setting this to local means you intend to run the peon as a standalone node (Not recommended).|remote|
|
||||
|`druid.indexer.baseDir`|Base temporary working directory.|/tmp|
|
||||
|`druid.indexer.baseTaskDir`|Base temporary working directory for tasks.|/tmp/persistent/tasks|
|
||||
|`druid.indexer.hadoopWorkingPath`|Temporary working directory for Hadoop tasks.|/tmp/druid-indexing|
|
||||
|`druid.indexer.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|50000|
|
||||
|`druid.indexer.task.chathandler.type`|Choices are "noop" and "announce". Certain tasks will use service discovery to announce an HTTP endpoint that events can be posted to.|noop|
|
||||
|
||||
There’s several additional configs that are required to run tasks.
|
||||
If the peon is running in remote mode, there must be an overlord up and running. Running peons in remote mode require the following configurations:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.runner`|Indicates whether tasks should be run locally or in a distributed environment. "local" or "remote".|local|
|
||||
|`druid.indexer.taskDir`|Intermediate temporary directory that tasks may use.|none|
|
||||
|`druid.indexer.configTable`|The MySQL config table where misc configs live.|none|
|
||||
|`druid.indexer.strategy`|The autoscaling strategy to use.|noop|
|
||||
|`druid.indexer.hadoopWorkingPath`|Intermediate temporary hadoop working directory that certain index tasks may use.|none|
|
||||
|`druid.indexer.logs.s3bucket`|S3 bucket to store logs.|none|
|
||||
|`druid.indexer.logs.s3prefix`|S3 key prefix to store logs.|none|
|
||||
|
||||
#### Console
|
||||
|
||||
The indexer console can be used to view pending tasks, running tasks, available workers, and recent worker creation and termination. The console can be accessed at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:8080/static/console.html
|
||||
```
|
||||
|
||||
Worker Node
|
||||
-----------
|
||||
|
||||
The worker node executes submitted tasks. Workers run tasks in separate JVMs.
|
||||
|`druid.peon.taskActionClient.retry.minWait`|The minimum retry time to communicate with overlord.|PT1M|
|
||||
|`druid.peon.taskActionClient.retry.maxWait`|The maximum retry time to communicate with overlord.|PT10M|
|
||||
|`druid.peon.taskActionClient.retry.maxRetryCount`|The maximum number of retries to communicate with overlord.|10|
|
||||
|
||||
#### Running
|
||||
|
||||
Worker nodes can be run using the `com.metamx.druid.indexing.worker.http.WorkerMain` class. Worker nodes can automatically be created by the Indexer Coordinator as part of autoscaling.
|
||||
|
||||
#### Configuration
|
||||
|
||||
Worker nodes require [basic service configuration](https://github.com/metamx/druid/wiki/Configuration#basic-service-configuration). In addition, there are several extra configurations that are required.
|
||||
The peon should very rarely ever be run independent of the middle manager.
|
||||
|
||||
```
|
||||
-Ddruid.worker.version=0
|
||||
-Ddruid.worker.capacity=3
|
||||
|
||||
-Ddruid.indexer.threads=3
|
||||
-Ddruid.indexer.taskDir=/mnt/persistent/task/
|
||||
-Ddruid.indexer.hadoopWorkingPath=/tmp/druid-indexing
|
||||
|
||||
-Ddruid.worker.coordinatorService=druid:sample_cluster:indexer
|
||||
|
||||
-Ddruid.indexer.fork.hostpattern=<IP>:%d
|
||||
-Ddruid.indexer.fork.startport=8080
|
||||
-Ddruid.indexer.fork.main=com.metamx.druid.indexing.worker.executor.ExecutorMain
|
||||
-Ddruid.indexer.fork.opts="-server -Xmx1g -Xms1g -XX:NewSize=256m -XX:MaxNewSize=256m"
|
||||
-Ddruid.indexer.fork.property.druid.service=druid/sample_cluster/executor
|
||||
|
||||
# These configs are the same configs you would set for basic service configuration, just with a different prefix
|
||||
-Ddruid.indexer.fork.property.druid.monitoring.monitorSystem=false
|
||||
-Ddruid.indexer.fork.property.druid.computation.buffer.size=268435456
|
||||
-Ddruid.indexer.fork.property.druid.indexer.taskDir=/mnt/persistent/task/
|
||||
-Ddruid.indexer.fork.property.druid.processing.formatString=processing-%s
|
||||
-Ddruid.indexer.fork.property.druid.processing.numThreads=1
|
||||
-Ddruid.indexer.fork.property.druid.server.maxSize=0
|
||||
-Ddruid.indexer.fork.property.druid.request.logging.dir=request_logs/
|
||||
io.druid.cli.Main internal peon <task_file> <status_file>
|
||||
```
|
||||
|
||||
Many of the configurations for workers are similar to those for basic service configuration":https://github.com/metamx/druid/wiki/Configuration\#basic-service-configuration, but with a different config prefix. Below we describe the unique worker configs.
|
||||
The task file contains the task JSON object.
|
||||
The status file indicates where the task status will be output.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.worker.version`|Version identifier for the worker.|0|
|
||||
|`druid.worker.capacity`|Maximum number of tasks the worker can accept.|1|
|
||||
|`druid.indexer.threads`|Number of processing threads per worker.|1|
|
||||
|`druid.worker.coordinatorService`|Name of the indexer coordinator used for service discovery.|none|
|
||||
|`druid.indexer.fork.hostpattern`|The format of the host name.|none|
|
||||
|`druid.indexer.fork.startport`|Port in which child JVM starts from.|none|
|
||||
|`druid.indexer.fork.opts`|JVM options for child JVMs.|none|
|
||||
Tasks
|
||||
-----
|
||||
|
||||
See [Tasks](Tasks.html).
|
||||
|
|
|
@ -1,393 +0,0 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Once you have a real-time node working, it is time to load your own data to see how Druid performs.
|
||||
|
||||
Druid can ingest data in three ways: via Kafka and a realtime node, via the indexing service, and via the Hadoop batch loader. Data is ingested in real-time using a [Firehose](Firehose.html).
|
||||
|
||||
## Create Config Directories ##
|
||||
Each type of node needs its own config file and directory, so create them as subdirectories under the druid directory if they not already exist.
|
||||
|
||||
```bash
|
||||
mkdir config
|
||||
mkdir config/realtime
|
||||
mkdir config/coordinator
|
||||
mkdir config/historical
|
||||
mkdir config/broker
|
||||
```
|
||||
|
||||
## Loading Data with Kafka ##
|
||||
|
||||
[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.0/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in realtime without writing any code. To load data to a realtime node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node.
|
||||
|
||||
### Booting Kafka ###
|
||||
|
||||
Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html).
|
||||
|
||||
1. Download Apache Kafka 0.7.2 from [http://kafka.apache.org/downloads.html](http://kafka.apache.org/downloads.html)
|
||||
|
||||
```bash
|
||||
wget http://apache.spinellicreations.com/incubator/kafka/kafka-0.7.2-incubating/kafka-0.7.2-incubating-src.tgz
|
||||
tar -xvzf kafka-0.7.2-incubating-src.tgz
|
||||
cd kafka-0.7.2-incubating-src
|
||||
```
|
||||
|
||||
2. Build Kafka
|
||||
|
||||
```bash
|
||||
./sbt update
|
||||
./sbt package
|
||||
```
|
||||
|
||||
3. Boot Kafka
|
||||
|
||||
```bash
|
||||
cat config/zookeeper.properties
|
||||
bin/zookeeper-server-start.sh config/zookeeper.properties
|
||||
# in a new console
|
||||
bin/kafka-server-start.sh config/server.properties
|
||||
```
|
||||
|
||||
4. Launch the console producer (so you can type in JSON kafka messages in a bit)
|
||||
|
||||
```bash
|
||||
bin/kafka-console-producer.sh --zookeeper localhost:2181 --topic druidtest
|
||||
```
|
||||
|
||||
### Launching a Realtime Node
|
||||
|
||||
1. Create a valid configuration file similar to this called config/realtime/runtime.properties:
|
||||
|
||||
```properties
|
||||
druid.host=localhost
|
||||
druid.service=example
|
||||
druid.port=8080
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.realtime.specFile=config/realtime/realtime.spec
|
||||
|
||||
druid.processing.buffer.sizeBytes=10000000
|
||||
|
||||
druid.processing.numThreads=3
|
||||
```
|
||||
|
||||
2. Create a valid realtime configuration file similar to this called realtime.spec:
|
||||
|
||||
```json
|
||||
[
|
||||
{
|
||||
"schema": {
|
||||
"dataSource": "druidtest",
|
||||
"aggregators": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "impressions"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "wp",
|
||||
"fieldName": "wp"
|
||||
}
|
||||
],
|
||||
"indexGranularity": "minute",
|
||||
"shardSpec": {
|
||||
"type": "none"
|
||||
}
|
||||
},
|
||||
"config": {
|
||||
"maxRowsInMemory": 500000,
|
||||
"intermediatePersistPeriod": "PT10m"
|
||||
},
|
||||
"firehose": {
|
||||
"type": "kafka-0.7.2",
|
||||
"consumerProps": {
|
||||
"zk.connect": "localhost:2181",
|
||||
"zk.connectiontimeout.ms": "15000",
|
||||
"zk.sessiontimeout.ms": "15000",
|
||||
"zk.synctime.ms": "5000",
|
||||
"groupid": "topic-pixel-local",
|
||||
"fetch.size": "1048586",
|
||||
"autooffset.reset": "largest",
|
||||
"autocommit.enable": "false"
|
||||
},
|
||||
"feed": "druidtest",
|
||||
"parser": {
|
||||
"timestampSpec": {
|
||||
"column": "utcdt",
|
||||
"format": "iso"
|
||||
},
|
||||
"data": {
|
||||
"format": "json"
|
||||
},
|
||||
"dimensionExclusions": [
|
||||
"wp"
|
||||
]
|
||||
}
|
||||
},
|
||||
"plumber": {
|
||||
"type": "realtime",
|
||||
"windowPeriod": "PT10m",
|
||||
"segmentGranularity": "hour",
|
||||
"basePersistDirectory": "\/tmp\/realtime\/basePersist",
|
||||
"rejectionPolicy": {
|
||||
"type": "messageTime"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
```
|
||||
|
||||
3. Launch the realtime node
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \
|
||||
-Ddruid.realtime.specFile=config/realtime/realtime.spec \
|
||||
-classpath lib/*:config/realtime io.druid.cli.Main server realtime
|
||||
```
|
||||
|
||||
4. Paste data into the Kafka console producer
|
||||
|
||||
```json
|
||||
{"utcdt": "2010-01-01T01:01:01", "wp": 1000, "gender": "male", "age": 100}
|
||||
{"utcdt": "2010-01-01T01:01:02", "wp": 2000, "gender": "female", "age": 50}
|
||||
{"utcdt": "2010-01-01T01:01:03", "wp": 3000, "gender": "male", "age": 20}
|
||||
{"utcdt": "2010-01-01T01:01:04", "wp": 4000, "gender": "female", "age": 30}
|
||||
{"utcdt": "2010-01-01T01:01:05", "wp": 5000, "gender": "male", "age": 40}
|
||||
```
|
||||
|
||||
5. Watch the events as they are ingested by Druid's realtime node
|
||||
|
||||
```bash
|
||||
...
|
||||
2013-06-17 21:41:55,569 INFO [Global--0] com.metamx.emitter.core.LoggingEmitter - Event [{"feed":"metrics","timestamp":"2013-06-17T21:41:55.569Z","service":"example","host":"127.0.0.1","metric":"events/processed","value":5,"user2":"druidtest"}]
|
||||
...
|
||||
```
|
||||
|
||||
6. In a new console, edit a file called query.body:
|
||||
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "druidtest",
|
||||
"granularity": "all",
|
||||
"dimensions": [],
|
||||
"aggregations": [
|
||||
{ "type": "count", "name": "rows" },
|
||||
{"type": "longSum", "name": "imps", "fieldName": "impressions"},
|
||||
{"type": "doubleSum", "name": "wp", "fieldName": "wp"}
|
||||
],
|
||||
"intervals": ["2010-01-01T00:00/2020-01-01T00"]
|
||||
}
|
||||
```
|
||||
|
||||
7. Submit the query via curl
|
||||
|
||||
```bash
|
||||
curl -X POST "http://localhost:8080/druid/v2/?pretty" \
|
||||
-H 'content-type: application/json' -d @query.body
|
||||
```
|
||||
|
||||
8. View Result!
|
||||
|
||||
```json
|
||||
[ {
|
||||
"timestamp" : "2010-01-01T01:01:00.000Z",
|
||||
"result" : {
|
||||
"imps" : 20,
|
||||
"wp" : 60000.0,
|
||||
"rows" : 5
|
||||
}
|
||||
} ]
|
||||
```
|
||||
|
||||
Now you're ready for [Querying Your Data](Querying-Your-Data.html)!
|
||||
|
||||
## Loading Data with the HadoopDruidIndexer ##
|
||||
|
||||
Historical data can be loaded via a Hadoop job.
|
||||
|
||||
The setup for a single node, 'standalone' Hadoop cluster is available at [http://hadoop.apache.org/docs/stable/single_node_setup.html](http://hadoop.apache.org/docs/stable/single_node_setup.html).
|
||||
|
||||
### Setup MySQL ###
|
||||
1. If you don't already have it, download MySQL Community Server here: [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/)
|
||||
2. Install MySQL
|
||||
3. Create a druid user and database
|
||||
|
||||
```bash
|
||||
mysql -u root
|
||||
```
|
||||
|
||||
```sql
|
||||
GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd';
|
||||
CREATE database druid;
|
||||
```
|
||||
|
||||
The [Coordinator](Coordinator.html) node will create the tables it needs based on its configuration.
|
||||
|
||||
### Make sure you have ZooKeeper Running ###
|
||||
|
||||
Make sure that you have a zookeeper instance running. If you followed the instructions for Kafka, it is probably running. If you are unsure if you have zookeeper running, try running
|
||||
|
||||
```bash
|
||||
ps auxww | grep zoo | grep -v grep
|
||||
```
|
||||
|
||||
If you get any result back, then zookeeper is most likely running. If you haven't setup Kafka or do not have zookeeper running, then you can download it and start it up with
|
||||
|
||||
```bash
|
||||
curl http://www.motorlogy.com/apache/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
|
||||
tar xzf zookeeper-3.4.5.tar.gz
|
||||
cd zookeeper-3.4.5
|
||||
cp conf/zoo_sample.cfg conf/zoo.cfg
|
||||
./bin/zkServer.sh start
|
||||
cd ..
|
||||
```
|
||||
|
||||
### Launch a Coordinator Node ###
|
||||
|
||||
If you've already setup a realtime node, be aware that although you can run multiple node types on one physical computer, you must assign them unique ports. Having used 8080 for the [Realtime](Realtime.html) node, we use 8081 for the [Coordinator](Coordinator.html).
|
||||
|
||||
1. Setup a configuration file called config/coordinator/runtime.properties similar to:
|
||||
|
||||
```properties
|
||||
druid.host=localhost
|
||||
druid.service=coordinator
|
||||
druid.port=8081
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.coordinator.startDelay=PT60s
|
||||
```
|
||||
|
||||
2. Launch the [Coordinator](Coordinator.html) node
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \
|
||||
-classpath lib/*:config/coordinator \
|
||||
io.druid.Cli.Main server coordinator
|
||||
```
|
||||
|
||||
### Launch a Historical Node ###
|
||||
|
||||
1. Create a configuration file in config/historical/runtime.properties similar to:
|
||||
|
||||
```properties
|
||||
druid.host=localhost
|
||||
druid.service=historical
|
||||
druid.port=8082
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
|
||||
druid.server.maxSize=100000000
|
||||
|
||||
druid.processing.buffer.sizeBytes=10000000
|
||||
|
||||
druid.segmentCache.infoPath=/tmp/druid/segmentInfoCache
|
||||
druid.segmentCache.locations=[{"path": "/tmp/druid/indexCache", "maxSize"\: 100000000}]
|
||||
```
|
||||
|
||||
2. Launch the historical node:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \
|
||||
-classpath lib/*:config/historical \
|
||||
io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
### Create a File of Records ###
|
||||
|
||||
We can use the same records we have been, in a file called records.json:
|
||||
|
||||
```json
|
||||
{"utcdt": "2010-01-01T01:01:01", "wp": 1000, "gender": "male", "age": 100}
|
||||
{"utcdt": "2010-01-01T01:01:02", "wp": 2000, "gender": "female", "age": 50}
|
||||
{"utcdt": "2010-01-01T01:01:03", "wp": 3000, "gender": "male", "age": 20}
|
||||
{"utcdt": "2010-01-01T01:01:04", "wp": 4000, "gender": "female", "age": 30}
|
||||
{"utcdt": "2010-01-01T01:01:05", "wp": 5000, "gender": "male", "age": 40}
|
||||
```
|
||||
|
||||
### Run the Hadoop Job ###
|
||||
|
||||
Now its time to run the Hadoop [Batch-ingestion](Batch-ingestion.html) job, HadoopDruidIndexer, which will fill a historical [Historical](Historical.html) node with data. First we'll need to configure the job.
|
||||
|
||||
1. Create a config called batchConfig.json similar to:
|
||||
|
||||
```json
|
||||
{
|
||||
"dataSource": "druidtest",
|
||||
"timestampColumn": "utcdt",
|
||||
"timestampFormat": "iso",
|
||||
"dataSpec": {
|
||||
"format": "json",
|
||||
"dimensions": [
|
||||
"gender",
|
||||
"age"
|
||||
]
|
||||
},
|
||||
"granularitySpec": {
|
||||
"type": "uniform",
|
||||
"intervals": [
|
||||
"2010-01-01T01\/PT1H"
|
||||
],
|
||||
"gran": "hour"
|
||||
},
|
||||
"pathSpec": {
|
||||
"type": "static",
|
||||
"paths": "\/druid\/records.json"
|
||||
},
|
||||
"rollupSpec": {
|
||||
"aggs": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "impressions"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "wp",
|
||||
"fieldName": "wp"
|
||||
}
|
||||
],
|
||||
"rollupGranularity": "minute"
|
||||
},
|
||||
"workingPath": "\/tmp\/working_path",
|
||||
"segmentOutputPath": "\/tmp\/segments",
|
||||
"partitionsSpec": {
|
||||
"targetPartitionSize": 5000000
|
||||
},
|
||||
"updaterJobSpec": {
|
||||
"type": "db",
|
||||
"connectURI": "jdbc:mysql:\/\/localhost:3306\/druid",
|
||||
"user": "druid",
|
||||
"password": "diurd",
|
||||
"segmentTable": "druid_segments"
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
2. Now run the job, with the config pointing at batchConfig.json:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 \
|
||||
-classpath `echo lib/* | tr ' ' ':'` \
|
||||
io.druid.cli.Main index hadoop batchConfig.json
|
||||
```
|
||||
|
||||
You can now move on to [Querying Your Data](Querying-Your-Data.html)!
|
|
@ -1,293 +0,0 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
# Setup #
|
||||
|
||||
Before we start querying druid, we're going to finish setting up a complete cluster on localhost. In [Loading Your Data](Loading-Your-Data.html) we setup a [Realtime](Realtime.html), [Historical](Historical.html) and [Coordinator](Coordinator.html) node. If you've already completed that tutorial, you need only follow the directions for 'Booting a Broker Node'.
|
||||
|
||||
## Booting a Broker Node ##
|
||||
|
||||
1. Setup a config file at config/broker/runtime.properties that looks like this:
|
||||
|
||||
```
|
||||
druid.host=localhost
|
||||
druid.service=broker
|
||||
druid.port=8080
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
```
|
||||
|
||||
2. Run the broker node:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/broker io.druid.cli.Main server broker
|
||||
```
|
||||
|
||||
With the Broker node and the other Druid nodes types up and running, you have a fully functional Druid Cluster and are ready to query your data!
|
||||
|
||||
# Querying Your Data #
|
||||
|
||||
Now that we have a complete cluster setup on localhost, we need to load data. To do so, refer to [Loading Your Data](Loading-Your-Data.html). Having done that, its time to query our data! For a complete specification of queries, see [Querying](Querying.html).
|
||||
|
||||
## Querying Different Nodes ##
|
||||
|
||||
As a shared-nothing system, there are three ways to query druid, against the [Realtime](Realtime.html), [Historical](Historical.html) or [Broker](Broker.html) node. Querying a Realtime node returns only realtime data, querying a historical node returns only historical segments. Querying the broker may query both realtime and historical segments and compose an overall result for the query. This is the normal mode of operation for queries in Druid.
|
||||
|
||||
### Construct a Query ###
|
||||
|
||||
For constructing this query, see: Querying against the realtime.spec
|
||||
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "druidtest",
|
||||
"granularity": "all",
|
||||
"dimensions": [],
|
||||
"aggregations": [
|
||||
{"type": "count", "name": "rows"},
|
||||
{"type": "longSum", "name": "imps", "fieldName": "impressions"},
|
||||
{"type": "doubleSum", "name": "wp", "fieldName": "wp"}
|
||||
],
|
||||
"intervals": ["2010-01-01T00:00/2020-01-01T00"]
|
||||
}
|
||||
```
|
||||
|
||||
### Querying the Realtime Node ###
|
||||
|
||||
Run our query against port 8080:
|
||||
|
||||
```bash
|
||||
curl -X POST "http://localhost:8080/druid/v2/?pretty" -H 'content-type: application/json' -d @query.body
|
||||
```
|
||||
|
||||
See our result:
|
||||
|
||||
```json
|
||||
[ {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : { "imps" : 5, "wp" : 15000.0, "rows" : 5 }
|
||||
} ]
|
||||
```
|
||||
|
||||
### Querying the Historical node ###
|
||||
Run the query against port 8082:
|
||||
|
||||
```bash
|
||||
curl -X POST "http://localhost:8082/druid/v2/?pretty" -H 'content-type: application/json' -d @query.body
|
||||
```
|
||||
|
||||
And get (similar to):
|
||||
|
||||
```json
|
||||
[ {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : { "imps" : 27, "wp" : 77000.0, "rows" : 9 }
|
||||
} ]
|
||||
```
|
||||
|
||||
### Querying the Broker ###
|
||||
Run the query against port 8083:
|
||||
|
||||
```bash
|
||||
curl -X POST "http://localhost:8083/druid/v2/?pretty" -H 'content-type: application/json' -d @query.body
|
||||
```
|
||||
|
||||
And get:
|
||||
|
||||
```json
|
||||
[ {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : { "imps" : 5, "wp" : 15000.0, "rows" : 5 }
|
||||
} ]
|
||||
```
|
||||
|
||||
Now that we know what nodes can be queried (although you should usually use the broker node), lets learn how to know what queries are available.
|
||||
|
||||
## Examining the realtime.spec ##
|
||||
|
||||
How are we to know what queries we can run? Although [Querying](Querying.html) is a helpful index, to get a handle on querying our data we need to look at our [Realtime](Realtime.html) node's realtime.spec file:
|
||||
|
||||
```json
|
||||
[
|
||||
{
|
||||
"schema": {
|
||||
"dataSource": "druidtest",
|
||||
"aggregators": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "impressions"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "wp",
|
||||
"fieldName": "wp"
|
||||
}
|
||||
],
|
||||
"indexGranularity": "minute",
|
||||
"shardSpec": {
|
||||
"type": "none"
|
||||
}
|
||||
},
|
||||
"config": {
|
||||
"maxRowsInMemory": 500000,
|
||||
"intermediatePersistPeriod": "PT10m"
|
||||
},
|
||||
"firehose": {
|
||||
"type": "kafka-0.7.2",
|
||||
"consumerProps": {
|
||||
"zk.connect": "localhost:2181",
|
||||
"zk.connectiontimeout.ms": "15000",
|
||||
"zk.sessiontimeout.ms": "15000",
|
||||
"zk.synctime.ms": "5000",
|
||||
"groupid": "topic-pixel-local",
|
||||
"fetch.size": "1048586",
|
||||
"autooffset.reset": "largest",
|
||||
"autocommit.enable": "false"
|
||||
},
|
||||
"feed": "druidtest",
|
||||
"parser": {
|
||||
"timestampSpec": {
|
||||
"column": "utcdt",
|
||||
"format": "iso"
|
||||
},
|
||||
"data": {
|
||||
"format": "json"
|
||||
},
|
||||
"dimensionExclusions": [
|
||||
"wp"
|
||||
]
|
||||
}
|
||||
},
|
||||
"plumber": {
|
||||
"type": "realtime",
|
||||
"windowPeriod": "PT10m",
|
||||
"segmentGranularity": "hour",
|
||||
"basePersistDirectory": "\/tmp\/realtime\/basePersist",
|
||||
"rejectionPolicy": {
|
||||
"type": "messageTime"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
```
|
||||
|
||||
### dataSource ###
|
||||
|
||||
```json
|
||||
"dataSource":"druidtest"
|
||||
```
|
||||
|
||||
Our dataSource tells us the name of the relation/table, or 'source of data', to query in both our realtime.spec and query.body!
|
||||
|
||||
### aggregations ###
|
||||
|
||||
Note the [Aggregations](Aggregations.html) in our query:
|
||||
|
||||
```json
|
||||
"aggregations": [
|
||||
{"type": "count", "name": "rows"},
|
||||
{"type": "longSum", "name": "imps", "fieldName": "impressions"},
|
||||
{"type": "doubleSum", "name": "wp", "fieldName": "wp"}
|
||||
],
|
||||
```
|
||||
|
||||
this matches up to the aggregators in the schema of our realtime.spec!
|
||||
|
||||
```json
|
||||
"aggregators":[ {"type":"count", "name":"impressions"},
|
||||
{"type":"doubleSum","name":"wp","fieldName":"wp"}],
|
||||
```
|
||||
|
||||
### dimensions ###
|
||||
|
||||
Lets look back at our actual records (from [Loading Your Data](Loading Your Data.html)):
|
||||
|
||||
```json
|
||||
{"utcdt": "2010-01-01T01:01:01", "wp": 1000, "gender": "male", "age": 100}
|
||||
{"utcdt": "2010-01-01T01:01:02", "wp": 2000, "gender": "female", "age": 50}
|
||||
{"utcdt": "2010-01-01T01:01:03", "wp": 3000, "gender": "male", "age": 20}
|
||||
{"utcdt": "2010-01-01T01:01:04", "wp": 4000, "gender": "female", "age": 30}
|
||||
{"utcdt": "2010-01-01T01:01:05", "wp": 5000, "gender": "male", "age": 40}
|
||||
```
|
||||
|
||||
Note that we have two dimensions to our data, other than our primary metric, wp. They are 'gender' and 'age'. We can specify these in our query! Note that we have added a dimension: age, below.
|
||||
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "druidtest",
|
||||
"granularity": "all",
|
||||
"dimensions": ["age"],
|
||||
"aggregations": [
|
||||
{"type": "count", "name": "rows"},
|
||||
{"type": "longSum", "name": "imps", "fieldName": "impressions"},
|
||||
{"type": "doubleSum", "name": "wp", "fieldName": "wp"}
|
||||
],
|
||||
"intervals": ["2010-01-01T00:00/2020-01-01T00"]
|
||||
}
|
||||
```
|
||||
|
||||
Which gets us grouped data in return!
|
||||
|
||||
```json
|
||||
[ {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : { "imps" : 1, "age" : "100", "wp" : 1000.0, "rows" : 1 }
|
||||
}, {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : { "imps" : 1, "age" : "20", "wp" : 3000.0, "rows" : 1 }
|
||||
}, {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : { "imps" : 1, "age" : "30", "wp" : 4000.0, "rows" : 1 }
|
||||
}, {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : { "imps" : 1, "age" : "40", "wp" : 5000.0, "rows" : 1 }
|
||||
}, {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : { "imps" : 1, "age" : "50", "wp" : 2000.0, "rows" : 1 }
|
||||
} ]
|
||||
```
|
||||
|
||||
### filtering ###
|
||||
|
||||
Now that we've observed our dimensions, we can also filter:
|
||||
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "druidtest",
|
||||
"granularity": "all",
|
||||
"filter": { "type": "selector", "dimension": "gender", "value": "male" },
|
||||
"aggregations": [
|
||||
{"type": "count", "name": "rows"},
|
||||
{"type": "longSum", "name": "imps", "fieldName": "impressions"},
|
||||
{"type": "doubleSum", "name": "wp", "fieldName": "wp"}
|
||||
],
|
||||
"intervals": ["2010-01-01T00:00/2020-01-01T00"]
|
||||
}
|
||||
```
|
||||
|
||||
Which gets us just people aged 40:
|
||||
|
||||
```json
|
||||
[ {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : { "imps" : 3, "wp" : 9000.0, "rows" : 3 }
|
||||
} ]
|
||||
```
|
||||
|
||||
Check out [Filters](Filters.html) for more information.
|
||||
|
||||
## Learn More ##
|
||||
|
||||
You can learn more about querying at [Querying](Querying.html)! Now check out [Booting a production cluster](Booting-a-production-cluster.html)!
|
|
@ -6,22 +6,46 @@ Realtime
|
|||
|
||||
Realtime nodes provide a realtime index. Data indexed via these nodes is immediately available for querying. Realtime nodes will periodically build segments representing the data they’ve collected over some span of time and hand these segments off to [Historical](Historical.html) nodes.
|
||||
|
||||
Running
|
||||
-------
|
||||
Quick Start
|
||||
-----------
|
||||
Run:
|
||||
|
||||
Realtime nodes can be run using the `com.metamx.druid.realtime.RealtimeMain` class.
|
||||
```
|
||||
io.druid.cli.Main server realtime
|
||||
```
|
||||
|
||||
Segment Propagation
|
||||
-------------------
|
||||
With the following JVM configuration:
|
||||
|
||||
The segment propagation diagram for real-time data ingestion can be seen below:
|
||||
```
|
||||
-server
|
||||
-Xmx256m
|
||||
-Duser.timezone=UTC
|
||||
-Dfile.encoding=UTF-8
|
||||
|
||||
![Segment Propagation](https://raw.github.com/metamx/druid/druid-0.5.4/doc/segment_propagation.png "Segment Propagation")
|
||||
druid.host=localhost
|
||||
druid.service=realtime
|
||||
druid.port=8083
|
||||
|
||||
Configuration
|
||||
-------------
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
Realtime nodes take a mix of base server configuration and spec files that describe how to connect, process and expose the realtime feed. See [Configuration](Configuration.html) for information about general server configuration.
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.processing.buffer.sizeBytes=10000000
|
||||
```
|
||||
|
||||
Note: This setup will not hand off segments to the rest of the cluster.
|
||||
|
||||
JVM Configuration
|
||||
-----------------
|
||||
|
||||
The realtime module uses several of the default modules in [Configuration](Configuration.html) and has the following set of configurations as well:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.realtime.specFile`|The file with realtime specifications in it.|none|
|
||||
|`druid.publish.type`|Choices:noop, db. After a real-time node completes building a segment after the window period, what does it do with it? For true handoff to occur, this should be set to "db".|noop|
|
||||
|
||||
### Realtime "specFile"
|
||||
|
||||
|
@ -138,6 +162,20 @@ The normal, expected use cases have the following overall constraints: `indexGra
|
|||
|
||||
If the RealtimeNode process runs out of heap, try adjusting druid.computation.buffer.size property which specifies a size in bytes that must fit into the heap.
|
||||
|
||||
Running
|
||||
-------
|
||||
|
||||
```
|
||||
io.druid.cli.Main server realtime
|
||||
```
|
||||
|
||||
Segment Propagation
|
||||
-------------------
|
||||
|
||||
The segment propagation diagram for real-time data ingestion can be seen below:
|
||||
|
||||
![Segment Propagation](https://raw.github.com/metamx/druid/druid-0.5.4/doc/segment_propagation.png "Segment Propagation")
|
||||
|
||||
Requirements
|
||||
------------
|
||||
|
||||
|
|
|
@ -1,71 +1,264 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Tasks are run on workers and always operate on a single datasource. Once an indexer coordinator node accepts a task, a lock is created for the datasource and interval specified in the task. Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release locks early if they desire. Tasks ids are unique by naming them using UUIDs or the timestamp in which the task was created. Tasks are also part of a "task group", which is a set of tasks that can share interval locks.
|
||||
Tasks are run on middle managers and always operate on a single data source.
|
||||
|
||||
There are several different types of tasks.
|
||||
|
||||
Append Task
|
||||
-----------
|
||||
Segment Creation Tasks
|
||||
----------------------
|
||||
|
||||
#### Index Task
|
||||
|
||||
The Index Task is a simpler variation of the Index Hadoop task that is designed to be used for smaller data sets. The task executes within the indexing service and does not require an external Hadoop setup to use. The grammar of the index task is as follows:
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "index",
|
||||
"dataSource" : "example",
|
||||
"granularitySpec" : {
|
||||
"type" : "uniform",
|
||||
"gran" : "DAY",
|
||||
"intervals" : [ "2010/2020" ]
|
||||
},
|
||||
"aggregators" : [ {
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "value",
|
||||
"fieldName" : "value"
|
||||
} ],
|
||||
"firehose" : {
|
||||
"type" : "local",
|
||||
"baseDir" : "/tmp/data/json",
|
||||
"filter" : "sample_data.json",
|
||||
"parser" : {
|
||||
"timestampSpec" : {
|
||||
"column" : "timestamp"
|
||||
},
|
||||
"data" : {
|
||||
"format" : "json",
|
||||
"dimensions" : [ "dim1", "dim2", "dim3" ]
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|The task type, this should always be "index".|yes|
|
||||
|id|The task ID.|no|
|
||||
|granularitySpec|See [granularitySpec](Tasks.html)|yes|
|
||||
|spatialDimensions|Dimensions to build spatial indexes over. See [Spatial-Indexing](Spatial-Indexing.html)|no|
|
||||
|aggregators|The metrics to aggregate in the data set. For more info, see [Aggregations](Aggregations.html)|yes|
|
||||
|indexGranularity|The rollup granularity for timestamps.|no|
|
||||
|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|no|
|
||||
|firehose|The input source of data. For more info, see [Firehose](Firehose.html)|yes|
|
||||
|rowFlushBoundary|Used in determining when intermediate persist should occur to disk.|no|
|
||||
|
||||
#### Index Hadoop Task
|
||||
|
||||
The Hadoop Index Task is used to index larger data sets that require the parallelization and processing power of a Hadoop cluster.
|
||||
|
||||
```
|
||||
{
|
||||
"type" : "index_hadoop",
|
||||
"config": <Hadoop index config>
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|The task type, this should always be "index_hadoop".|yes|
|
||||
|config|A Hadoop Index Config. See [Batch Ingestion](Batch-ingestion.html)|yes|
|
||||
|hadoopCoordinates|The Maven <groupId>:<artifactId>:<version> of Hadoop to use. The default is "org.apache.hadoop:hadoop-core:1.0.3".|no|
|
||||
|
||||
|
||||
The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopBatchIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `updaterJobSpec`. The Indexing Service takes care of setting these fields internally.
|
||||
|
||||
#### Realtime Index Task
|
||||
|
||||
The indexing service can also run real-time tasks. These tasks effectively transform a middle manager into a real-time node. We introduced real-time tasks as a way to programmatically add new real-time data sources without needing to manually add nodes. The grammar for the real-time task is as follows:
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "index_realtime",
|
||||
"id": "example",
|
||||
"resource": {
|
||||
"availabilityGroup" : "someGroup",
|
||||
"requiredCapacity" : 1
|
||||
},
|
||||
"schema": {
|
||||
"dataSource": "dataSourceName",
|
||||
"aggregators": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "events"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "outColumn",
|
||||
"fieldName": "inColumn"
|
||||
}
|
||||
],
|
||||
"indexGranularity": "minute",
|
||||
"shardSpec": {
|
||||
"type": "none"
|
||||
}
|
||||
},
|
||||
"firehose": {
|
||||
"type": "kafka-0.7.2",
|
||||
"consumerProps": {
|
||||
"zk.connect": "zk_connect_string",
|
||||
"zk.connectiontimeout.ms": "15000",
|
||||
"zk.sessiontimeout.ms": "15000",
|
||||
"zk.synctime.ms": "5000",
|
||||
"groupid": "consumer-group",
|
||||
"fetch.size": "1048586",
|
||||
"autooffset.reset": "largest",
|
||||
"autocommit.enable": "false"
|
||||
},
|
||||
"feed": "your_kafka_topic",
|
||||
"parser": {
|
||||
"timestampSpec": {
|
||||
"column": "timestamp",
|
||||
"format": "iso"
|
||||
},
|
||||
"data": {
|
||||
"format": "json"
|
||||
},
|
||||
"dimensionExclusions": [
|
||||
"value"
|
||||
]
|
||||
}
|
||||
},
|
||||
"fireDepartmentConfig": {
|
||||
"maxRowsInMemory": 500000,
|
||||
"intermediatePersistPeriod": "PT10m"
|
||||
},
|
||||
"windowPeriod": "PT10m",
|
||||
"segmentGranularity": "hour",
|
||||
"rejectionPolicy": {
|
||||
"type": "messageTime"
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
Id:
|
||||
The ID of the task. Not required.
|
||||
|
||||
Resource:
|
||||
A JSON object used for high availability purposes. Not required.
|
||||
|
||||
|Field|Type|Description|Required|
|
||||
|-----|----|-----------|--------|
|
||||
|availabilityGroup|String|An uniqueness identifier for the task. Tasks with the same availability group will always run on different middle managers. Used mainly for replication. |yes|
|
||||
|requiredCapacity|Integer|How much middle manager capacity this task will take.|yes|
|
||||
|
||||
Schema:
|
||||
See [Schema](Realtime.html).
|
||||
|
||||
Fire Department Config:
|
||||
See [Config](Realtime.html).
|
||||
|
||||
Firehose:
|
||||
See [Firehose](Firehose.html).
|
||||
|
||||
Window Period:
|
||||
See [Realtime](Realtime.html).
|
||||
|
||||
Segment Granularity:
|
||||
See [Realtime](Realtime.html).
|
||||
|
||||
Rejection Policy:
|
||||
See [Realtime](Realtime.html).
|
||||
|
||||
Segment Merging Tasks
|
||||
---------------------
|
||||
|
||||
#### Append Task
|
||||
|
||||
Append tasks append a list of segments together into a single segment (one after the other). The grammar is:
|
||||
|
||||
{
|
||||
"id": <task_id>,
|
||||
"dataSource": <task_datasource>,
|
||||
"segments": <JSON list of DataSegment objects to append>
|
||||
}
|
||||
```json
|
||||
{
|
||||
"id": <task_id>,
|
||||
"dataSource": <task_datasource>,
|
||||
"segments": <JSON list of DataSegment objects to append>
|
||||
}
|
||||
```
|
||||
|
||||
Merge Task
|
||||
----------
|
||||
#### Merge Task
|
||||
|
||||
Merge tasks merge a list of segments together. Any common timestamps are merged. The grammar is:
|
||||
|
||||
{
|
||||
"id": <task_id>,
|
||||
"dataSource": <task_datasource>,
|
||||
"segments": <JSON list of DataSegment objects to append>
|
||||
}
|
||||
```json
|
||||
{
|
||||
"id": <task_id>,
|
||||
"dataSource": <task_datasource>,
|
||||
"segments": <JSON list of DataSegment objects to append>
|
||||
}
|
||||
```
|
||||
|
||||
Delete Task
|
||||
-----------
|
||||
Segment Destroying Tasks
|
||||
------------------------
|
||||
|
||||
#### Delete Task
|
||||
|
||||
Delete tasks create empty segments with no data. The grammar is:
|
||||
|
||||
{
|
||||
"id": <task_id>,
|
||||
"dataSource": <task_datasource>,
|
||||
"segments": <JSON list of DataSegment objects to append>
|
||||
}
|
||||
```json
|
||||
{
|
||||
"id": <task_id>,
|
||||
"dataSource": <task_datasource>,
|
||||
"segments": <JSON list of DataSegment objects to append>
|
||||
}
|
||||
```
|
||||
|
||||
Kill Task
|
||||
---------
|
||||
#### Kill Task
|
||||
|
||||
Kill tasks delete all information about a segment and removes it from deep storage. Killable segments must be disabled (used==0) in the Druid segment table. The available grammar is:
|
||||
|
||||
{
|
||||
"id": <task_id>,
|
||||
"dataSource": <task_datasource>,
|
||||
"segments": <JSON list of DataSegment objects to append>
|
||||
}
|
||||
```json
|
||||
{
|
||||
"id": <task_id>,
|
||||
"dataSource": <task_datasource>,
|
||||
"segments": <JSON list of DataSegment objects to append>
|
||||
}
|
||||
```
|
||||
|
||||
Index Task
|
||||
----------
|
||||
Misc. Tasks
|
||||
-----------
|
||||
|
||||
Index Partitions Task
|
||||
---------------------
|
||||
#### Version Converter Task
|
||||
|
||||
Index Generator Task
|
||||
--------------------
|
||||
These tasks convert segments from an existing older index version to the latest index version. The available grammar is:
|
||||
|
||||
Index Hadoop Task
|
||||
-----------------
|
||||
```json
|
||||
{
|
||||
"id": <task_id>,
|
||||
"groupId" : <task_group_id>,
|
||||
"dataSource": <task_datasource>,
|
||||
"interval" : <segment_interval>,
|
||||
"segment": <JSON DataSegment object to convert>
|
||||
}
|
||||
```
|
||||
|
||||
Index Realtime Task
|
||||
-------------------
|
||||
#### Noop Task
|
||||
|
||||
Version Converter Task
|
||||
----------------------
|
||||
These tasks start, sleep for a time and are used only for testing. The available grammar is:
|
||||
|
||||
Version Converter SubTask
|
||||
-------------------------
|
||||
```json
|
||||
{
|
||||
"id": <optional_task_id>,
|
||||
"interval" : <optional_segment_interval>,
|
||||
"runTime" : <optional_millis_to_sleep>,
|
||||
"firehose": <optional_firehose_to_test_connect>
|
||||
}
|
||||
```
|
||||
|
||||
Locking
|
||||
-------
|
||||
Once an overlord node accepts a task, a lock is created for the data source and interval specified in the task. Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release locks early if they desire. Tasks ids are unique by naming them using UUIDs or the timestamp in which the task was created. Tasks are also part of a "task group", which is a set of tasks that can share interval locks.
|
||||
|
|
|
@ -43,12 +43,11 @@ These metrics track the number of characters added, deleted, and changed.
|
|||
Setting Up
|
||||
----------
|
||||
|
||||
There are two ways to setup Druid: download a tarball, or [Build From Source](Build From Source.html). You only need to do one of these.
|
||||
There are two ways to setup Druid: download a tarball, or [Build From Source](Build-from-source.html). You only need to do one of these.
|
||||
|
||||
### Download a Tarball
|
||||
|
||||
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.0-bin.tar.gz)
|
||||
Download this file to a directory of your choosing.
|
||||
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.0-bin.tar.gz). Download this file to a directory of your choosing.
|
||||
|
||||
You can extract the awesomeness within by issuing:
|
||||
|
||||
|
@ -98,7 +97,7 @@ Okay, things are about to get real-time. To query the real-time node you've spun
|
|||
./run_example_client.sh
|
||||
```
|
||||
|
||||
Select "wikipedia" once again. This script issues [GroupByQuery](GroupByQuery.html)s to the data we've been ingesting. The query looks like this:
|
||||
Select "wikipedia" once again. This script issues [GroupByQueries](GroupByQuery.html) to the data we've been ingesting. The query looks like this:
|
||||
|
||||
```json
|
||||
{
|
||||
|
@ -108,7 +107,7 @@ Select "wikipedia" once again. This script issues [GroupByQuery](GroupByQuery.ht
|
|||
"dimensions":[ "page" ],
|
||||
"aggregations":[
|
||||
{"type":"count", "name":"rows"},
|
||||
{"type":"longSum", "fieldName":"edit_count", "name":"count"}
|
||||
{"type":"longSum", "fieldName":"count", "name":"edit_count"}
|
||||
],
|
||||
"filter":{ "type":"selector", "dimension":"namespace", "value":"article" },
|
||||
"intervals":[ "2013-06-01T00:00/2020-01-01T00" ]
|
||||
|
@ -151,7 +150,7 @@ time_boundary_query.body
|
|||
|
||||
Druid queries are JSON blobs which are relatively painless to create programmatically, but an absolute pain to write by hand. So anyway, we are going to create a Druid query by hand. Add the following to the file you just created:
|
||||
|
||||
```
|
||||
```json
|
||||
{
|
||||
"queryType": "timeBoundary",
|
||||
"dataSource": "wikipedia"
|
||||
|
@ -186,7 +185,7 @@ timeseries_query.body
|
|||
|
||||
We are going to make a slightly more complicated query, the [TimeseriesQuery](TimeseriesQuery.html). Copy and paste the following into the file:
|
||||
|
||||
```
|
||||
```json
|
||||
{
|
||||
"queryType": "timeseries",
|
||||
"dataSource": "wikipedia",
|
||||
|
@ -221,7 +220,7 @@ Right now all the results you are getting back are being aggregated into a singl
|
|||
|
||||
If you loudly exclaimed "we can change granularity to minute", you are absolutely correct! We can specify different granularities to bucket our results, like so:
|
||||
|
||||
```
|
||||
```json
|
||||
{
|
||||
"queryType": "timeseries",
|
||||
"dataSource": "wikipedia",
|
||||
|
@ -267,7 +266,7 @@ group_by_query.body
|
|||
|
||||
and put the following in there:
|
||||
|
||||
```
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "wikipedia",
|
||||
|
@ -321,13 +320,13 @@ Feel free to tweak other query parameters to answer other questions you may have
|
|||
Next Steps
|
||||
----------
|
||||
|
||||
What to know even more information about the Druid Cluster? Check out [Tutorial: The Druid Cluster](Tutorial:-The-Druid-Cluster.html)
|
||||
What to know even more information about the Druid Cluster? Check out [The Druid Cluster](Tutorial%3A-The-Druid-Cluster.html)
|
||||
|
||||
Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Loading-Your-Data.html).
|
||||
Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-1.html).
|
||||
|
||||
Additional Information
|
||||
----------------------
|
||||
|
||||
This tutorial is merely showcasing a small fraction of what Druid can do. If you are interested in more information about Druid, including setting up a more sophisticated Druid cluster, please read the other links in our wiki.
|
||||
|
||||
And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](http://www.groups.google.com/forum/#!forum/druid-development).
|
||||
And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](https://groups.google.com/forum/#!forum/druid-development).
|
||||
|
|
|
@ -0,0 +1,197 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Hello! This tutorial is meant to provide a more in-depth look into Druid queries. The tutorial is somewhat incomplete right now but we hope to add more content to it in the near future.
|
||||
|
||||
Setup
|
||||
-----
|
||||
|
||||
Before we start digging into how to query Druid, make sure you've gone through the other tutorials and are comfortable with spinning up a local cluster and loading data into Druid.
|
||||
|
||||
#### Booting a Druid Cluster
|
||||
|
||||
Let's start up a simple Druid cluster so we can query all the things.
|
||||
|
||||
To start a Coordinator node:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/coordinator io.druid.cli.Main server coordinator
|
||||
```
|
||||
|
||||
To start a Historical node:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/historical io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
To start a Broker node:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/broker io.druid.cli.Main server broker
|
||||
```
|
||||
|
||||
Querying Your Data
|
||||
------------------
|
||||
|
||||
Make sure you've completed [Loading Your Data](Loading-Your-Data-Part-1.html) so we have some data to query. Having done that, it's time to query our data! For a complete specification of queries, see [Querying](Querying.html).
|
||||
|
||||
#### Construct a Query
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "wikipedia",
|
||||
"granularity": "all",
|
||||
"dimensions": [],
|
||||
"aggregations": [
|
||||
{"type": "count", "name": "rows"},
|
||||
{"type": "longSum", "name": "edit_count", "fieldName": "count"},
|
||||
{"type": "doubleSum", "name": "chars_added", "fieldName": "added"}
|
||||
],
|
||||
"intervals": ["2010-01-01T00:00/2020-01-01T00"]
|
||||
}
|
||||
```
|
||||
|
||||
#### Query That Data
|
||||
Run the query against your broker:
|
||||
|
||||
```bash
|
||||
curl -X POST "http://localhost:8080/druid/v2/?pretty" -H 'Content-type: application/json' -d @query.body
|
||||
```
|
||||
|
||||
And get:
|
||||
|
||||
```json
|
||||
[ {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : {
|
||||
"chars_added" : 1545.0,
|
||||
"edit_count" : 5,
|
||||
"rows" : 5
|
||||
}
|
||||
} ]
|
||||
```
|
||||
|
||||
This result tells us that our query has 5 edits, and we have 5 rows of data as well. In those 5 edits, we have 1545 characters added.
|
||||
|
||||
#### What can I query for?
|
||||
|
||||
How are we to know what queries we can run? Although [Querying](Querying.html) is a helpful index, to get a handle on querying our data we need to look at our ingestion schema. There are a few particular fields we care about in the ingestion schema. All of these fields should in present in the real-time ingestion schema and the batch ingestion schema.
|
||||
|
||||
Datasource:
|
||||
|
||||
```json
|
||||
"dataSource":"wikipedia"
|
||||
```
|
||||
|
||||
Our dataSource tells us the name of the relation/table, or 'source of data'. What we decide to name our data source must match the data source we are going to be querying.
|
||||
|
||||
Granularity:
|
||||
|
||||
```json
|
||||
"indexGranularity": "none",
|
||||
```
|
||||
|
||||
Druid will roll up data at ingestion time unless the index/rollup granularity is specified as "none". Your query granularity cannot be lower than your index granularity.
|
||||
|
||||
Aggregators:
|
||||
|
||||
```json
|
||||
"aggregators" : [{
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "added",
|
||||
"fieldName" : "added"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "deleted",
|
||||
"fieldName" : "deleted"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "delta",
|
||||
"fieldName" : "delta"
|
||||
}]
|
||||
```
|
||||
|
||||
The [Aggregations](Aggregations.html) specified at ingestion time correlated directly to the metrics that can be queried.
|
||||
|
||||
Dimensions:
|
||||
|
||||
```json
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
```
|
||||
|
||||
These specify the dimensions that we can filter our data on. If we added a dimension to our groupBy query, we get:
|
||||
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "wikipedia",
|
||||
"granularity": "all",
|
||||
"dimensions": ["namespace"],
|
||||
"aggregations": [
|
||||
{"type": "longSum", "name": "edit_count", "fieldName": "count"},
|
||||
{"type": "doubleSum", "name": "chars_added", "fieldName": "added"}
|
||||
],
|
||||
"intervals": ["2010-01-01T00:00/2020-01-01T00"]
|
||||
}
|
||||
```
|
||||
|
||||
Which gets us data grouped over the namespace dimension in return!
|
||||
|
||||
```json
|
||||
[ {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : {
|
||||
"chars_added" : 180.0,
|
||||
"edit_count" : 2,
|
||||
"namespace" : "article"
|
||||
}
|
||||
}, {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : {
|
||||
"chars_added" : 1365.0,
|
||||
"edit_count" : 3,
|
||||
"namespace" : "wikipedia"
|
||||
}
|
||||
} ]
|
||||
```
|
||||
|
||||
Additionally,, we can also filter our query to narrow down our metric values:
|
||||
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "wikipedia",
|
||||
"granularity": "all",
|
||||
"filter": { "type": "selector", "dimension": "namespace", "value": "article" },
|
||||
"aggregations": [
|
||||
{"type": "longSum", "name": "edit_count", "fieldName": "count"},
|
||||
{"type": "doubleSum", "name": "chars_added", "fieldName": "added"}
|
||||
],
|
||||
"intervals": ["2010-01-01T00:00/2020-01-01T00"]
|
||||
}
|
||||
```
|
||||
|
||||
Which gets us metrics about only those edits where the namespace is 'article':
|
||||
|
||||
```json
|
||||
[ {
|
||||
"version" : "v1",
|
||||
"timestamp" : "2010-01-01T00:00:00.000Z",
|
||||
"event" : {
|
||||
"chars_added" : 180.0,
|
||||
"edit_count" : 2
|
||||
}
|
||||
} ]
|
||||
```
|
||||
|
||||
Check out [Filters](Filters.html) for more information.
|
||||
|
||||
## Learn More ##
|
||||
|
||||
You can learn more about querying at [Querying](Querying.html)! If you are ready to evaluate Druid more in depth, check out [Booting a production cluster](Booting-a-production-cluster.html)!
|
|
@ -0,0 +1,255 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
In our last [tutorial](Tutorial%3A-The-Druid-Cluster.html), we set up a complete Druid cluster. We created all the Druid dependencies and loaded some batched data. Druid shards data into self-contained chunks known as [segments](Segments.html). Segments are the fundamental unit of storage in Druid and all Druid nodes only understand segments.
|
||||
|
||||
In this tutorial, we will learn about batch ingestion (as opposed to real-time ingestion) and how to create segments using the final piece of the Druid Cluster, the [indexing service](Indexing-Service.html). The indexing service is a standalone service that accepts [tasks](Tasks.html) in the form of POST requests. The output of most tasks are segments.
|
||||
|
||||
If you are interested more about ingesting your own data into Druid, skip to the next [tutorial](Tutorial%3A-Loading-Your-Data-Part-2.html).
|
||||
|
||||
About the data
|
||||
--------------
|
||||
|
||||
The data source we'll be working with is Wikipedia edits once again. The data schema is the same as the previous tutorials:
|
||||
|
||||
Dimensions (things to filter on):
|
||||
|
||||
```json
|
||||
"page"
|
||||
"language"
|
||||
"user"
|
||||
"unpatrolled"
|
||||
"newPage"
|
||||
"robot"
|
||||
"anonymous"
|
||||
"namespace"
|
||||
"continent"
|
||||
"country"
|
||||
"region"
|
||||
"city"
|
||||
```
|
||||
|
||||
Metrics (things to aggregate over):
|
||||
|
||||
```json
|
||||
"count"
|
||||
"added"
|
||||
"delta"
|
||||
"deleted"
|
||||
```
|
||||
Setting Up
|
||||
----------
|
||||
|
||||
At this point, you should already have Druid downloaded and are comfortable with running a Druid cluster locally. If you are not, see [here](Tutiroal%3A-The-Druid-Cluster.html).
|
||||
|
||||
Let's start from our usual starting point in the tarball directory.
|
||||
|
||||
Segments require data, so before we can build a Druid segment, we are going to need some raw data. Make sure that the following file exists:
|
||||
|
||||
```
|
||||
examples/indexing/wikipedia_data.json
|
||||
```
|
||||
|
||||
Open the file and make sure the following events exist:
|
||||
|
||||
```json
|
||||
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
|
||||
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
||||
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
||||
```
|
||||
|
||||
There are five data points spread across the day of 2013-08-31. Talk about big data right? Thankfully, we don't need a ton of data to introduce how batch ingestion works.
|
||||
|
||||
In order to ingest and query this data, we are going to need to run a historical node, a coordinator node, and an indexing service to run the batch ingestion.
|
||||
|
||||
#### Starting a Local Indexing Service
|
||||
|
||||
The simplest indexing service we can start up is to run an [overlord](Indexing-Service.html) node in local mode. You can do so by issuing:
|
||||
|
||||
```bash
|
||||
java -Xmx2g -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/overlord io.druid.cli.Main server overlord
|
||||
```
|
||||
|
||||
The overlord configurations should already exist in:
|
||||
|
||||
```
|
||||
config/overlord/runtime.properties
|
||||
```
|
||||
|
||||
The configurations for the overlord node are as follows:
|
||||
|
||||
```bash
|
||||
druid.host=localhost
|
||||
druid.port=8087
|
||||
druid.service=overlord
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.selectors.indexing.serviceName=overlord
|
||||
druid.indexer.runner.javaOpts="-server -Xmx1g"
|
||||
druid.indexer.runner.startPort=8088
|
||||
druid.indexer.fork.property.druid.computation.buffer.size=268435456
|
||||
```
|
||||
|
||||
If you are interested in reading more about these configurations, see [here](Indexing-Service.html).
|
||||
|
||||
When the overlord node is ready for tasks, you should see a message like the following:
|
||||
|
||||
```bash
|
||||
2013-10-09 21:30:32,817 INFO [Thread-14] io.druid.indexing.overlord.TaskQueue - Waiting for work...
|
||||
```
|
||||
|
||||
#### Starting Other Nodes
|
||||
|
||||
Just in case you forgot how, let's start up the other nodes we require:
|
||||
|
||||
Coordinator node:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/coordinator io.druid.cli.Main server coordinator
|
||||
```
|
||||
|
||||
Historical node:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/historical io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
Note: Historical, real-time and broker nodes share the same query interface. Hence, we do not explicitly need a broker node for this tutorial. All queries can go against the historical node directly.
|
||||
|
||||
Once all the nodes are up and running, we are ready to index some data.
|
||||
|
||||
Indexing the Data
|
||||
-----------------
|
||||
|
||||
To index the data and build a Druid segment, we are going to need to submit a task to the indexing service. This task should already exist:
|
||||
|
||||
```
|
||||
examples/indexing/index_task.json
|
||||
```
|
||||
|
||||
Open up the file to see the following:
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "index",
|
||||
"dataSource" : "wikipedia",
|
||||
"granularitySpec" : {
|
||||
"type" : "uniform",
|
||||
"gran" : "DAY",
|
||||
"intervals" : [ "2013-08-31/2013-09-01" ]
|
||||
},
|
||||
"aggregators" : [{
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "added",
|
||||
"fieldName" : "added"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "deleted",
|
||||
"fieldName" : "deleted"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "delta",
|
||||
"fieldName" : "delta"
|
||||
}],
|
||||
"firehose" : {
|
||||
"type" : "local",
|
||||
"baseDir" : "examples/indexing",
|
||||
"filter" : "wikipedia_data.json",
|
||||
"parser" : {
|
||||
"timestampSpec" : {
|
||||
"column" : "timestamp"
|
||||
},
|
||||
"data" : {
|
||||
"format" : "json",
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
Okay, so what is happening here? The "type" field indicates the type of task we plan to run. In this case, it is a simple "index" task. The "granularitySpec" indicates that we are building a daily segment for 2013-08-31 to 2013-09-01. Next, the "aggregators" indicate which fields in our data set we plan to build metric columns for. The "fieldName" corresponds to the metric name in the raw data. The "name" corresponds to what our metric column is actually going to be called in the segment. Finally, we have a local "firehose" that is going to read data from disk. We tell the firehose where our data is located and the types of files we are looking to ingest. In our case, we only have a single data file.
|
||||
|
||||
Let's send our task to the indexing service now:
|
||||
|
||||
```bash
|
||||
curl -X 'POST' -H 'Content-Type:application/json' -d @examples/indexing/wikipedia_index_task.json localhost:8087/druid/indexer/v1/task
|
||||
```
|
||||
|
||||
Issuing the request should return a task ID like so:
|
||||
|
||||
```bash
|
||||
$ curl -X 'POST' -H 'Content-Type:application/json' -d @examples/indexing/wikipedia_index_task.json localhost:8087/druid/indexer/v1/task
|
||||
{"task":"index_wikipedia_2013-10-09T21:30:32.802Z"}
|
||||
$
|
||||
```
|
||||
|
||||
In your indexing service logs, you should see the following:
|
||||
|
||||
```bash
|
||||
2013-10-09 21:41:41,150 INFO [qtp300448720-21] io.druid.indexing.overlord.HeapMemoryTaskStorage - Inserting task index_wikipedia_2013-10-09T21:41:41.147Z with status: TaskStatus{id=index_wikipedia_2013-10-09T21:41:41.147Z, status=RUNNING, duration=-1}
|
||||
2013-10-09 21:41:41,151 INFO [qtp300448720-21] io.druid.indexing.overlord.TaskLockbox - Created new TaskLockPosse: TaskLockPosse{taskLock=TaskLock{groupId=index_wikipedia_2013-10-09T21:41:41.147Z, dataSource=wikipedia, interval=2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z, version=2013-10-09T21:41:41.151Z}, taskIds=[]}
|
||||
...
|
||||
013-10-09 21:41:41,215 INFO [pool-6-thread-1] io.druid.indexing.overlord.ForkingTaskRunner - Logging task index_wikipedia_2013-10-09T21:41:41.147Z_generator_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_0 output to: /tmp/persistent/index_wikipedia_2013-10-09T21:41:41.147Z_generator_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_0/b5099fdb-d6b0-4b81-9053-b2af70336a7e/log
|
||||
2013-10-09 21:41:45,017 INFO [qtp300448720-22] io.druid.indexing.common.actions.LocalTaskActionClient - Performing action for task[index_wikipedia_2013-10-09T21:41:41.147Z_generator_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_0]: LockListAction{}
|
||||
|
||||
````
|
||||
|
||||
After a few seconds, the task should complete and you should see in the indexing service logs:
|
||||
|
||||
```bash
|
||||
2013-10-09 21:41:45,765 INFO [pool-6-thread-1] io.druid.indexing.overlord.exec.TaskConsumer - Received SUCCESS status for task: IndexGeneratorTask{id=index_wikipedia_2013-10-09T21:41:41.147Z_generator_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_0, type=index_generator, dataSource=wikipedia, interval=Optional.of(2013-08-31T00:00:00.000Z/2013-09-01T00:00:00.000Z)}
|
||||
```
|
||||
|
||||
Congratulations! The segment has completed building. Once a segment is built, a segment metadata entry is created in your MySQL table. The coordinator compares what is in the segment metadata table with what is in the cluster. A new entry in the metadata table will cause the coordinator to load the new segment in a minute or so.
|
||||
|
||||
You should see the following logs on the coordinator:
|
||||
|
||||
```bash
|
||||
2013-10-09 21:41:54,368 INFO [Coordinator-Exec--0] io.druid.server.coordinator.DruidCoordinatorLogger - [_default_tier] : Assigned 1 segments among 1 servers
|
||||
2013-10-09 21:41:54,369 INFO [Coordinator-Exec--0] io.druid.server.coordinator.DruidCoordinatorLogger - Load Queues:
|
||||
2013-10-09 21:41:54,369 INFO [Coordinator-Exec--0] io.druid.server.coordinator.DruidCoordinatorLogger - Server[localhost:8081, historical, _default_tier] has 1 left to load, 0 left to drop, 4,477 bytes queued, 4,477 bytes served.
|
||||
```
|
||||
|
||||
These logs indicate that the coordinator has assigned our new segment to the historical node to download and serve. If you look at the historical node logs, you should see:
|
||||
|
||||
```bash
|
||||
2013-10-09 21:41:54,369 INFO [ZkCoordinator-0] io.druid.server.coordination.ZkCoordinator - Loading segment wikipedia_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2013-10-09T21:41:41.151Z
|
||||
2013-10-09 21:41:54,369 INFO [ZkCoordinator-0] io.druid.segment.loading.LocalDataSegmentPuller - Unzipping local file[/tmp/druid/localStorage/wikipedia/2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z/2013-10-09T21:41:41.151Z/0/index.zip] to [/tmp/druid/indexCache/wikipedia/2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z/2013-10-09T21:41:41.151Z/0]
|
||||
2013-10-09 21:41:54,370 INFO [ZkCoordinator-0] io.druid.utils.CompressionUtils - Unzipping file[/tmp/druid/localStorage/wikipedia/2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z/2013-10-09T21:41:41.151Z/0/index.zip] to [/tmp/druid/indexCache/wikipedia/2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z/2013-10-09T21:41:41.151Z/0]
|
||||
2013-10-09 21:41:54,380 INFO [ZkCoordinator-0] io.druid.server.coordination.SingleDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2013-10-09T21:41:41.151Z] to path[/druid/servedSegments/localhost:8081/wikipedia_2013-08-31T00:00:00.000Z_2013-09-01T00:00:00.000Z_2013-10-09T21:41:41.151Z]
|
||||
```
|
||||
|
||||
Once the segment is announced the segment is queryable. Now you should be able to query the data.
|
||||
|
||||
Issuing a [TimeBoundaryQuery](TimeBoundaryQuery.html) should yield:
|
||||
|
||||
```json
|
||||
[ {
|
||||
"timestamp" : "2013-08-31T01:02:33.000Z",
|
||||
"result" : {
|
||||
"minTime" : "2013-08-31T01:02:33.000Z",
|
||||
"maxTime" : "2013-08-31T12:41:27.000Z"
|
||||
}
|
||||
} ]
|
||||
```
|
||||
|
||||
Next Steps
|
||||
----------
|
||||
|
||||
This tutorial covered ingesting a small batch data set and loading it into Druid. In [Loading Your Data Part 2](Tutorial%3A-Loading-Your-Data-Part-2.html), we will cover how to ingest data using Hadoop for larger data sets.
|
||||
|
||||
Additional Information
|
||||
----------------------
|
||||
|
||||
Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-development).
|
|
@ -0,0 +1,321 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
In this tutorial we will cover more advanced/real-world ingestion topics.
|
||||
|
||||
Druid can ingest streaming or batch data. Streaming data is ingested via the real-time node, and batch data is ingested via the Hadoop batch indexer. Druid also has a standalone ingestion service called the [indexing service](Indexing-Service.html).
|
||||
|
||||
The Data
|
||||
--------
|
||||
The data source we'll be using is (surprise!) Wikipedia edits. The data schema is still:
|
||||
|
||||
Dimensions (things to filter on):
|
||||
|
||||
```json
|
||||
"page"
|
||||
"language"
|
||||
"user"
|
||||
"unpatrolled"
|
||||
"newPage"
|
||||
"robot"
|
||||
"anonymous"
|
||||
"namespace"
|
||||
"continent"
|
||||
"country"
|
||||
"region"
|
||||
"city"
|
||||
```
|
||||
|
||||
Metrics (things to aggregate over):
|
||||
|
||||
```json
|
||||
"count"
|
||||
"added"
|
||||
"delta"
|
||||
"deleted"
|
||||
```
|
||||
|
||||
Streaming Event Ingestion
|
||||
-------------------------
|
||||
|
||||
With real-world data, we recommend having a message bus such as [Apache Kafka](http://kafka.apache.org/) sit between the data stream and the real-time node. The message bus provides higher availability for production environments. [Firehoses](Firehose.html) are the key abstraction for real-time ingestion.
|
||||
|
||||
#### Setting up Kafka
|
||||
|
||||
[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.0/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node.
|
||||
|
||||
Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html).
|
||||
|
||||
1. Download Apache Kafka 0.7.2 from [http://kafka.apache.org/downloads.html](http://kafka.apache.org/downloads.html)
|
||||
|
||||
```bash
|
||||
wget http://apache.spinellicreations.com/incubator/kafka/kafka-0.7.2-incubating/kafka-0.7.2-incubating-src.tgz
|
||||
tar -xvzf kafka-0.7.2-incubating-src.tgz
|
||||
cd kafka-0.7.2-incubating-src
|
||||
```
|
||||
|
||||
2. Build Kafka
|
||||
|
||||
```bash
|
||||
./sbt update
|
||||
./sbt package
|
||||
```
|
||||
|
||||
3. Boot Kafka
|
||||
|
||||
```bash
|
||||
cat config/zookeeper.properties
|
||||
bin/zookeeper-server-start.sh config/zookeeper.properties
|
||||
|
||||
# in a new console
|
||||
bin/kafka-server-start.sh config/server.properties
|
||||
```
|
||||
|
||||
4. Launch the console producer (so you can type in JSON kafka messages in a bit)
|
||||
|
||||
```bash
|
||||
bin/kafka-console-producer.sh --zookeeper localhost:2181 --topic wikipedia
|
||||
```
|
||||
|
||||
When things are ready, you should see log messages such as:
|
||||
|
||||
```
|
||||
[2013-10-09 22:03:07,802] INFO zookeeper state changed (SyncConnected) (org.I0Itec.zkclient.ZkClient)
|
||||
```
|
||||
|
||||
#### Launch a Realtime Node
|
||||
|
||||
You should be comfortable starting Druid nodes at this point. If not, it may be worthwhile to revisit the first few tutorials.
|
||||
|
||||
1. Real-time nodes can be started with:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=examples/indexing/wikipedia.spec -classpath lib/*:config/realtime io.druid.cli.Main server realtime
|
||||
```
|
||||
|
||||
2. A realtime.spec should already exist for the data source in the Druid tarball. You should be able to find it at:
|
||||
|
||||
```bash
|
||||
examples/indexing/wikipedia.spec
|
||||
```
|
||||
|
||||
The contents of the file should match:
|
||||
|
||||
```json
|
||||
[
|
||||
{
|
||||
"schema": {
|
||||
"dataSource": "wikipedia",
|
||||
"aggregators" : [{
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "added",
|
||||
"fieldName" : "added"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "deleted",
|
||||
"fieldName" : "deleted"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "delta",
|
||||
"fieldName" : "delta"
|
||||
}],
|
||||
"indexGranularity": "none"
|
||||
},
|
||||
"config": {
|
||||
"maxRowsInMemory": 500000,
|
||||
"intermediatePersistPeriod": "PT10m"
|
||||
},
|
||||
"firehose": {
|
||||
"type": "kafka-0.7.2",
|
||||
"consumerProps": {
|
||||
"zk.connect": "localhost:2181",
|
||||
"zk.connectiontimeout.ms": "15000",
|
||||
"zk.sessiontimeout.ms": "15000",
|
||||
"zk.synctime.ms": "5000",
|
||||
"groupid": "druid-example",
|
||||
"fetch.size": "1048586",
|
||||
"autooffset.reset": "largest",
|
||||
"autocommit.enable": "false"
|
||||
},
|
||||
"feed": "wikipedia",
|
||||
"parser": {
|
||||
"timestampSpec": {
|
||||
"column": "timestamp"
|
||||
},
|
||||
"data": {
|
||||
"format": "json",
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
}
|
||||
}
|
||||
},
|
||||
"plumber": {
|
||||
"type": "realtime",
|
||||
"windowPeriod": "PT10m",
|
||||
"segmentGranularity": "hour",
|
||||
"basePersistDirectory": "\/tmp\/realtime\/basePersist",
|
||||
"rejectionPolicy": {
|
||||
"type": "none"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
```
|
||||
|
||||
3. Let's copy and paste some data into the Kafka console producer
|
||||
|
||||
```json
|
||||
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
|
||||
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
||||
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
||||
```
|
||||
|
||||
Disclaimer: We recognize the timestamps of these events aren't actually recent.
|
||||
|
||||
5. Watch the events as they are ingested by Druid's real-time node:
|
||||
|
||||
```bash
|
||||
...
|
||||
2013-10-10 05:13:18,976 INFO [chief-wikipedia] io.druid.server.coordination.BatchDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-31T01:00:00.000Z_2013-08-31T02:00:00.000Z_2013-08-31T01:00:00.000Z] at path[/druid/segments/localhost:8083/2013-10-10T05:13:18.972Z0]
|
||||
2013-10-10 05:13:18,992 INFO [chief-wikipedia] io.druid.server.coordination.BatchDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-31T03:00:00.000Z_2013-08-31T04:00:00.000Z_2013-08-31T03:00:00.000Z] at path[/druid/segments/localhost:8083/2013-10-10T05:13:18.972Z0]
|
||||
2013-10-10 05:13:18,997 INFO [chief-wikipedia] io.druid.server.coordination.BatchDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-31T07:00:00.000Z_2013-08-31T08:00:00.000Z_2013-08-31T07:00:00.000Z] at path[/druid/segments/localhost:8083/2013-10-10T05:13:18.972Z0]
|
||||
2013-10-10 05:13:19,003 INFO [chief-wikipedia] io.druid.server.coordination.BatchDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-31T11:00:00.000Z_2013-08-31T12:00:00.000Z_2013-08-31T11:00:00.000Z] at path[/druid/segments/localhost:8083/2013-10-10T05:13:18.972Z0]
|
||||
2013-10-10 05:13:19,008 INFO [chief-wikipedia] io.druid.server.coordination.BatchDataSegmentAnnouncer - Announcing segment[wikipedia_2013-08-31T12:00:00.000Z_2013-08-31T13:00:00.000Z_2013-08-31T12:00:00.000Z] at path[/druid/segments/localhost:8083/2013-10-10T05:13:18.972Z0]
|
||||
...
|
||||
```
|
||||
|
||||
Issuing a [TimeBoundaryQuery](TimeBoundaryQuery.html) to the real-time node should yield valid results:
|
||||
|
||||
```json
|
||||
[ {
|
||||
"timestamp" : "2013-08-31T01:02:33.000Z",
|
||||
"result" : {
|
||||
"minTime" : "2013-08-31T01:02:33.000Z",
|
||||
"maxTime" : "2013-08-31T12:41:27.000Z"
|
||||
}
|
||||
} ]
|
||||
```
|
||||
|
||||
Batch Ingestion
|
||||
---------------
|
||||
Druid is designed for large data volumes, and most real-world data sets require batch indexing be done through a Hadoop job.
|
||||
|
||||
The setup for a single node, 'standalone' Hadoop cluster is available [here](http://hadoop.apache.org/docs/stable/single_node_setup.html).
|
||||
|
||||
For the purposes of this tutorial, we are going to use our very small and simple Wikipedia data set. This data can directly be ingested via other means as shown in the previous [tutorial](Tutorial%3A-Loading-Your-Data-Part-1), but we are going to use Hadoop here for demonstration purposes.
|
||||
|
||||
Our data is located at:
|
||||
|
||||
```
|
||||
examples/indexing/wikipedia_data.json
|
||||
```
|
||||
|
||||
The following events should exist in the file:
|
||||
|
||||
```json
|
||||
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
|
||||
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
||||
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
||||
```
|
||||
|
||||
#### Setup a Druid Cluster
|
||||
|
||||
To index the data, we are going to need an indexing service, a historical node, and a coordinator node.
|
||||
|
||||
To start the Indexing Service:
|
||||
|
||||
```bash
|
||||
java -Xmx2g -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:<hadoop_config_path>:config/overlord io.druid.cli.Main server overlord
|
||||
```
|
||||
|
||||
To start the Coordinator Node:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/coordinator io.druid.cli.Main server coordinator
|
||||
```
|
||||
|
||||
To start the Historical Node:
|
||||
|
||||
```bash
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/historical io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
#### Index the Data
|
||||
|
||||
Before indexing the data, make sure you have a valid Hadoop cluster running. To build our Druid segment, we are going to submit a [Hadoop index task](Tasks.html) to the indexing service. The grammar for the Hadoop index task is very similar to the index task of the last tutorial. The tutorial Hadoop index task should be located at:
|
||||
|
||||
```
|
||||
examples/indexing/wikipedia_index_hadoop_task.json
|
||||
```
|
||||
|
||||
Examining the contents of the file, you should find:
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "index_hadoop",
|
||||
"config": {
|
||||
"dataSource" : "wikipedia",
|
||||
"timestampColumn" : "timestamp",
|
||||
"timestampFormat" : "auto",
|
||||
"dataSpec" : {
|
||||
"format" : "json",
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
},
|
||||
"granularitySpec" : {
|
||||
"type" : "uniform",
|
||||
"gran" : "DAY",
|
||||
"intervals" : [ "2013-08-31/2013-09-01" ]
|
||||
},
|
||||
"pathSpec" : {
|
||||
"type" : "static",
|
||||
"paths" : "examples/indexing/wikipedia_data.json"
|
||||
},
|
||||
"targetPartitionSize" : 5000000,
|
||||
"rollupSpec" : {
|
||||
"aggs": [{
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "added",
|
||||
"fieldName" : "added"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "deleted",
|
||||
"fieldName" : "deleted"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "delta",
|
||||
"fieldName" : "delta"
|
||||
}],
|
||||
"rollupGranularity" : "none"
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
If you are curious about what all this configuration means, see [here](Task.html)
|
||||
To submit the task:
|
||||
|
||||
```bash
|
||||
curl -X 'POST' -H 'Content-Type:application/json' -d @examples/indexing/wikipedia_index_hadoop_task.json localhost:8087/druid/indexer/v1/task
|
||||
```
|
||||
|
||||
After the task is completed, the segment should be assigned to your historical node. You should be able to query the segment.
|
||||
|
||||
Next Steps
|
||||
----------
|
||||
We demonstrated using the indexing service as a way to ingest data into Druid. Previous versions of Druid used the [HadoopDruidIndexer](Batch-ingestion.html) to ingest batch data. The `HadoopDruidIndexer` still remains a valid option for batch ingestion, however, we recommend using the indexing service as the preferred method of getting batch data into Druid.
|
||||
|
||||
For more information on querying, check out this [tutorial](Tutorial%3A-All-About-Queries.html).
|
||||
|
||||
Additional Information
|
||||
----------------------
|
||||
|
||||
Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-development).
|
|
@ -1,13 +1,13 @@
|
|||
---
|
||||
layout: doc_page
|
||||
---
|
||||
Welcome back! In our first [tutorial](Tutorial:-A-First-Look-at-Druid), we introduced you to the most basic Druid setup: a single realtime node. We streamed in some data and queried it. Realtime nodes collect very recent data and periodically hand that data off to the rest of the Druid cluster. Some questions about the architecture must naturally come to mind. What does the rest of Druid cluster look like? How does Druid load available static data?
|
||||
Welcome back! In our first [tutorial](Tutorial%3A-A-First-Look-at-Druid.html), we introduced you to the most basic Druid setup: a single realtime node. We streamed in some data and queried it. Realtime nodes collect very recent data and periodically hand that data off to the rest of the Druid cluster. Some questions about the architecture must naturally come to mind. What does the rest of Druid cluster look like? How does Druid load available static data?
|
||||
|
||||
This tutorial will hopefully answer these questions!
|
||||
|
||||
In this tutorial, we will set up other types of Druid nodes as well as and external dependencies for a fully functional Druid cluster. The architecture of Druid is very much like the [Megazord](http://www.youtube.com/watch?v=7mQuHh1X4H4) from the popular 90s show Mighty Morphin' Power Rangers. Each Druid node has a specific purpose and the nodes come together to form a fully functional system.
|
||||
|
||||
## Downloading Druid ##
|
||||
## Downloading Druid
|
||||
|
||||
If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first.
|
||||
|
||||
|
@ -20,15 +20,15 @@ tar -zxvf druid-services-*-bin.tar.gz
|
|||
cd druid-services-*
|
||||
```
|
||||
|
||||
You can also [Build From Source](Build-From-Source.html).
|
||||
You can also [Build From Source](Build-from-source.html).
|
||||
|
||||
## External Dependencies ##
|
||||
## External Dependencies
|
||||
|
||||
Druid requires 3 external dependencies. A "deep" storage that acts as a backup data repository, a relational database such as MySQL to hold configuration and metadata information, and [Apache Zookeeper](http://zookeeper.apache.org/) for coordination among different pieces of the cluster.
|
||||
|
||||
For deep storage, we have made a public S3 bucket (static.druid.io) available where data for this particular tutorial can be downloaded. More on the data [later](Tutorial-Part-2.html#the-data).
|
||||
For deep storage, we have made a public S3 bucket (static.druid.io) available where data for this particular tutorial can be downloaded. More on the data later.
|
||||
|
||||
### Setting up MySQL ###
|
||||
#### Setting up MySQL
|
||||
|
||||
1. If you don't already have it, download MySQL Community Server here: [http://dev.mysql.com/downloads/mysql/](http://dev.mysql.com/downloads/mysql/)
|
||||
2. Install MySQL
|
||||
|
@ -43,7 +43,7 @@ GRANT ALL ON druid.* TO 'druid'@'localhost' IDENTIFIED BY 'diurd';
|
|||
CREATE database druid;
|
||||
```
|
||||
|
||||
### Setting up Zookeeper ###
|
||||
#### Setting up Zookeeper
|
||||
|
||||
```bash
|
||||
curl http://www.motorlogy.com/apache/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz
|
||||
|
@ -54,9 +54,9 @@ cp conf/zoo_sample.cfg conf/zoo.cfg
|
|||
cd ..
|
||||
```
|
||||
|
||||
## The Data ##
|
||||
## The Data
|
||||
|
||||
Similar to the first tutorial, the data we will be loading is based on edits that have occurred on Wikipedia. Every time someone edits a page in Wikipedia, metadata is generated about the editor and edited page. Druid collects each individual event and packages them together in a container known as a [segment](https://github.com/metamx/druid/wiki/Segments). Segments contain data over some span of time. We've prebuilt a segment for this tutorial and will cover making your own segments in other [pages](Loading-Your-Data.html).The segment we are going to work with has the following format:
|
||||
Similar to the first tutorial, the data we will be loading is based on edits that have occurred on Wikipedia. Every time someone edits a page in Wikipedia, metadata is generated about the editor and edited page. Druid collects each individual event and packages them together in a container known as a [segment](Segments.html). Segments contain data over some span of time. We've prebuilt a segment for this tutorial and will cover making your own segments in other [pages](Tutorial%3A-Loading-Your-Data-Part-1.html).The segment we are going to work with has the following format:
|
||||
|
||||
Dimensions (things to filter on):
|
||||
|
||||
|
@ -84,28 +84,28 @@ Metrics (things to aggregate over):
|
|||
"deleted"
|
||||
```
|
||||
|
||||
## The Cluster ##
|
||||
## The Cluster
|
||||
|
||||
Let's start up a few nodes and download our data. First things though, let's create a config directory where we will store configs for our various nodes:
|
||||
Let's start up a few nodes and download our data. First things though, let's make sure we have config directory where we will store configs for our various nodes:
|
||||
|
||||
```
|
||||
mkdir config
|
||||
ls config
|
||||
```
|
||||
|
||||
If you are interested in learning more about Druid configuration files, check out this [link](Configuration.html). Many aspects of Druid are customizable. For the purposes of this tutorial, we are going to use default values for most things.
|
||||
|
||||
### Start a Coordinator Node ###
|
||||
#### Start a Coordinator Node
|
||||
|
||||
Coordinator nodes are in charge of load assignment and distribution. Coordinator nodes monitor the status of the cluster and command historical nodes to assign and drop segments.
|
||||
For more information about coordinator nodes, see [here](Coordinator.html).
|
||||
|
||||
To create the coordinator config file:
|
||||
The coordinator config file should already exist at:
|
||||
|
||||
```
|
||||
mkdir config/coordinator
|
||||
config/coordinator
|
||||
```
|
||||
|
||||
Under the directory we just created, create the file `runtime.properties` with the following contents if it does not exist:
|
||||
In the directory, there should be a `runtime.properties` file with the following contents:
|
||||
|
||||
```
|
||||
druid.host=localhost
|
||||
|
@ -114,9 +114,6 @@ druid.port=8082
|
|||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
@ -130,18 +127,18 @@ To start the coordinator node:
|
|||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/coordinator io.druid.cli.Main server coordinator
|
||||
```
|
||||
|
||||
### Start a historical node ###
|
||||
#### Start a Historical Node
|
||||
|
||||
Historical nodes are the workhorses of a cluster and are in charge of loading historical segments and making them available for queries. Our Wikipedia segment will be downloaded by a historical node.
|
||||
For more information about Historical nodes, see [here](Historical.html).
|
||||
|
||||
To create the historical config file:
|
||||
The historical config file should exist at:
|
||||
|
||||
```
|
||||
mkdir config/historical
|
||||
config/historical
|
||||
```
|
||||
|
||||
Under the directory we just created, create the file `runtime.properties` with the following contents:
|
||||
In the directory we just created, we should have the file `runtime.properties` with the following contents:
|
||||
|
||||
```
|
||||
druid.host=localhost
|
||||
|
@ -150,6 +147,7 @@ druid.port=8081
|
|||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
# Dummy read only AWS account (used to download example data)
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
|
||||
|
@ -167,18 +165,18 @@ To start the historical node:
|
|||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/historical io.druid.cli.Main server historical
|
||||
```
|
||||
|
||||
### Start a Broker Node ###
|
||||
#### Start a Broker Node
|
||||
|
||||
Broker nodes are responsible for figuring out which historical and/or realtime nodes correspond to which queries. They also merge partial results from these nodes in a scatter/gather fashion.
|
||||
For more information about Broker nodes, see [here](Broker.html).
|
||||
|
||||
To create the broker config file:
|
||||
The broker config file should exist at:
|
||||
|
||||
```
|
||||
mkdir config/broker
|
||||
config/broker
|
||||
```
|
||||
|
||||
Under the directory we just created, create the file ```runtime.properties``` with the following contents:
|
||||
In the directory, there should be a `runtime.properties` file with the following contents:
|
||||
|
||||
```
|
||||
druid.host=localhost
|
||||
|
@ -194,7 +192,7 @@ To start the broker node:
|
|||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -classpath lib/*:config/broker io.druid.cli.Main server broker
|
||||
```
|
||||
|
||||
## Loading the Data ##
|
||||
## Loading the Data
|
||||
|
||||
The MySQL dependency we introduced earlier on contains a 'segments' table that contains entries for segments that should be loaded into our cluster. The Druid coordinator compares this table with segments that already exist in the cluster to determine what should be loaded and dropped. To load our wikipedia segment, we need to create an entry in our MySQL segment table.
|
||||
|
||||
|
@ -220,7 +218,31 @@ When the segment completes downloading and ready for queries, you should see the
|
|||
|
||||
At this point, we can query the segment. For more information on querying, see this [link](Querying.html).
|
||||
|
||||
## Next Steps ##
|
||||
### Bonus Round: Start a Realtime Node
|
||||
|
||||
Now that you have an understanding of what the Druid clsuter looks like, why not load some of your own data?
|
||||
Check out the [Loading Your Own Data](Loading-Your-Data.html) section for more info!
|
||||
To start the realtime node that was used in our first tutorial, you simply have to issue:
|
||||
|
||||
```
|
||||
java -Xmx256m -Duser.timezone=UTC -Dfile.encoding=UTF-8 -Ddruid.realtime.specFile=examples/wikipedia/wikipedia_realtime.spec -classpath lib/*:config/realtime io.druid.cli.Main server realtime
|
||||
```
|
||||
|
||||
The configurations are located in `config/realtime/runtime.properties` and should contain the following:
|
||||
|
||||
```
|
||||
druid.host=localhost
|
||||
druid.service=realtime
|
||||
druid.port=8083
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.processing.buffer.sizeBytes=10000000
|
||||
```
|
||||
|
||||
Next Steps
|
||||
----------
|
||||
If you are intested in how data flows through the different Druid components, check out the Druid [Data Flow](Data-Flow.html). Now that you have an understanding of what the Druid cluster looks like, why not load some of your own data?
|
||||
Check out the next [tutorial](Tutorial%3A-Loading-Your-Data-Part-1.html) section for more info!
|
|
@ -80,7 +80,7 @@ Okay, things are about to get real. To query the real-time node you've spun up,
|
|||
./run_example_client.sh
|
||||
```
|
||||
|
||||
Select "webstream" once again. This script issues [GroupByQuery](GroupByQuery.html)s to the data we've been ingesting. The query looks like this:
|
||||
Select "webstream" once again. This script issues [GroupByQueries](GroupByQuery.html) to the data we've been ingesting. The query looks like this:
|
||||
|
||||
```json
|
||||
{
|
||||
|
@ -304,15 +304,9 @@ You should see an answer to our question. For my stream, it looks like this:
|
|||
|
||||
Feel free to tweak other query parameters to answer other questions you may have about the data.
|
||||
|
||||
Next Steps
|
||||
----------
|
||||
|
||||
What to know even more information about the Druid Cluster? Check out [Tutorial: The Druid Cluster](Tutorial:-The-Druid-Cluster.html)
|
||||
Druid is even more fun if you load your own data into it! To learn how to load your data, see [Loading Your Data](Loading-Your-Data.html).
|
||||
|
||||
Additional Information
|
||||
----------------------
|
||||
|
||||
This tutorial is merely showcasing a small fraction of what Druid can do. If you are interested in more information about Druid, including setting up a more sophisticated Druid cluster, please read the other links in our wiki.
|
||||
|
||||
And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](http://www.groups.google.com/forum/#!forum/druid-development).
|
||||
And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our [google groups page](https://groups.google.com/forum/#!forum/druid-development).
|
||||
|
|
|
@ -9,16 +9,16 @@ There are two ways to setup Druid: download a tarball, or build it from source.
|
|||
|
||||
h3. Download a Tarball
|
||||
|
||||
We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/data/examples/druid-services-0.6.0.tar.gz.
|
||||
We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.0-bin.tar.gz.
|
||||
Download this bad boy to a directory of your choosing.
|
||||
|
||||
You can extract the awesomeness within by issuing:
|
||||
|
||||
pre. tar -zxvf druid-services-0.6.0.tar.gz
|
||||
pre. tar -zxvf druid-services-0.X.X.tar.gz
|
||||
|
||||
Not too lost so far right? That's great! If you cd into the directory:
|
||||
|
||||
pre. cd druid-services-0.6.0-SNAPSHOT
|
||||
pre. cd druid-services-0.X.X
|
||||
|
||||
You should see a bunch of files:
|
||||
* run_example_server.sh
|
||||
|
@ -31,7 +31,7 @@ The other way to setup Druid is from source via git. To do so, run these command
|
|||
|
||||
<pre><code>git clone git@github.com:metamx/druid.git
|
||||
cd druid
|
||||
git checkout druid-0.6.0
|
||||
git checkout druid-0.X.X
|
||||
./build.sh
|
||||
</code></pre>
|
||||
|
||||
|
@ -322,6 +322,6 @@ Feel free to tweak other query parameters to answer other questions you may have
|
|||
|
||||
h2. Additional Information
|
||||
|
||||
This tutorial is merely showcasing a small fraction of what Druid can do. Next, continue on to "Loading Your Data":./Loading-Your-Data.html.
|
||||
This tutorial is merely showcasing a small fraction of what Druid can do. Next, continue on to "The Druid Cluster":./Tutorial:-The-Druid-Cluster.html.
|
||||
|
||||
And thus concludes our journey! Hopefully you learned a thing or two about Druid real-time ingestion, querying Druid, and how Druid can be used to solve problems. If you have additional questions, feel free to post in our "google groups page":http://www.groups.google.com/forum/#!forum/druid-development.
|
||||
|
|
|
@ -6,6 +6,8 @@ Druid uses ZooKeeper (ZK) for management of current cluster state. The operation
|
|||
1. [Coordinator](Coordinator.html) leader election
|
||||
2. Segment "publishing" protocol from [Historical](Historical.html) and [Realtime](Realtime.html)
|
||||
3. Segment load/drop protocol between [Coordinator](Coordinator.html) and [Historical](Historical.html)
|
||||
4. [Overlord](Indexing-Service.html) leader election
|
||||
5. [Indexing Service](Indexing-Service.html) task management
|
||||
|
||||
### Property Configuration
|
||||
|
||||
|
@ -31,6 +33,21 @@ druid.zk.paths.indexer.statusPath=${druid.zk.paths.base}/indexer/status
|
|||
druid.zk.paths.indexer.leaderLatchPath=${druid.zk.paths.base}/indexer/leaderLatchPath
|
||||
```
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.zk.paths.base`|Base Zookeeper path.|druid|
|
||||
|`druid.zk.paths.propertiesPath`|Zookeeper properties path.|druid/properties|
|
||||
|`druid.zk.paths.announcementsPath`|Druid node announcement path.|druid/announcements|
|
||||
|`druid.zk.paths.servedSegmentsPath`|Legacy path for where Druid nodes announce their segments.|druid/servedSegments|
|
||||
|`druid.zk.paths.liveSegmentsPath`|Current path for where Druid nodes announce their segments.|druid/segments|
|
||||
|`druid.zk.paths.loadQueuePath`|Entries here cause historical nodes to load and drop segments.|druid/loadQueue|
|
||||
|`druid.zk.paths.coordinatorPath`|Used by the coordinator for leader election.|druid/coordinator|
|
||||
|`druid.zk.paths.indexer.announcementsPath`|Middle managers announce themselves here.|druid/indexer/announcements|
|
||||
|`druid.zk.paths.indexer.tasksPath`|Used to assign tasks to middle managers.|druid/indexer/tasks|
|
||||
|`druid.zk.paths.indexer.statusPath`|Parent path for announcement of task statuses.|druid/indexer/status|
|
||||
|`druid.zk.paths.indexer.leaderLatchPath`|Used for Overlord leader election.|druid/indexer/leaderLatchPath|
|
||||
|
||||
|
||||
NOTE: We also use Curator’s service discovery module to expose some services via zookeeper. This also uses a zookeeper path, but this path is **not** affected by `druid.zk.paths.base` and **must** be specified separately. This property is
|
||||
|
||||
```
|
||||
|
|
|
@ -10,18 +10,26 @@
|
|||
Getting Started
|
||||
* [Tutorial: A First Look at Druid](Tutorial:-A-First-Look-at-Druid.html)
|
||||
* [Tutorial: The Druid Cluster](Tutorial:-The-Druid-Cluster.html)
|
||||
* [Loading Your Data](Loading-Your-Data.html)
|
||||
* [Querying Your Data](Querying-Your-Data.html)
|
||||
* [Booting a Production Cluster](Booting-a-Production-Cluster.html)
|
||||
* [Examples](Examples.html)
|
||||
* [Cluster Setup](Cluster-Setup.html)
|
||||
* [Configuration](Configuration.html)
|
||||
* [Tutorial: Loading Your Data Part 1](Tutorial:-Loading-Your-Data-Part-1.html)
|
||||
* [Tutorial: Loading Your Data Part 2](Tutorial:-Loading-Your-Data-Part-2.html)
|
||||
* [Tutorial: All About Queries](Tutorial:-All-About-Queries.html)
|
||||
--------------------------------------
|
||||
|
||||
Evaluate Druid
|
||||
* [Cluster Setup](Cluster-setup.html)
|
||||
* [Booting a Production Cluster](Booting-a-production-cluster.html)
|
||||
--------------------------------------
|
||||
|
||||
Configuration
|
||||
* [Configuration](Configuration.html)
|
||||
-------------------------------------
|
||||
|
||||
Data Ingestion
|
||||
* [Realtime](Realtime.html)
|
||||
* [Batch|Batch Ingestion](Batch|Batch-Ingestion.html)
|
||||
* [Batch Ingestion](Batch-ingestion.html)
|
||||
* [Indexing Service](Indexing-Service.html)
|
||||
*** ]
|
||||
*** [Tasks](Tasks.html)
|
||||
----------------------------
|
||||
|
||||
Querying
|
||||
|
|
|
@ -12,16 +12,22 @@ h1. Contents
|
|||
h2. Getting Started
|
||||
* "Tutorial: A First Look at Druid":./Tutorial:-A-First-Look-at-Druid.html
|
||||
* "Tutorial: The Druid Cluster":./Tutorial:-The-Druid-Cluster.html
|
||||
* "Loading Your Data":./Loading-Your-Data.html
|
||||
* "Querying Your Data":./Querying-your-data.html
|
||||
* "Tutorial: Loading Your Data Part 1":./Tutorial:-Loading-Your-Data-Part-1.html
|
||||
* "Tutorial: Loading Your Data Part 2":./Tutorial:-Loading-Your-Data-Part-2.html
|
||||
* "Tutorial: All About Queries":./Tutorial:-All-About-Queries.html
|
||||
|
||||
h2. Evaluate Druid
|
||||
* "Cluster Setup":./Cluster-setup.html
|
||||
* "Booting a Production Cluster":./Booting-a-production-cluster.html
|
||||
* "Examples":./Examples.html
|
||||
|
||||
h2. Configuration
|
||||
* "Configuration":Configuration.html
|
||||
|
||||
h2. Data Ingestion
|
||||
* "Realtime":./Realtime.html
|
||||
* "Batch":./Batch-ingestion.html
|
||||
* "Indexing Service":./Indexing-Service.html
|
||||
** "Tasks":./Tasks.html
|
||||
|
||||
h2. Querying
|
||||
* "Querying":./Querying.html
|
||||
|
@ -41,6 +47,7 @@ h2. Querying
|
|||
|
||||
h2. Architecture
|
||||
* "Design":./Design.html
|
||||
** "Data Flow":./Data-Flow.html
|
||||
* "Segments":./Segments.html
|
||||
* Node Types
|
||||
** "Historical":./Historical.html
|
||||
|
@ -50,6 +57,7 @@ h2. Architecture
|
|||
** "Realtime":./Realtime.html
|
||||
*** "Firehose":./Firehose.html
|
||||
*** "Plumber":./Plumber.html
|
||||
** "Indexing Service":./Indexing-Service.html
|
||||
* External Dependencies
|
||||
** "Deep Storage":./Deep-Storage.html
|
||||
** "MySQL":./MySQL.html
|
||||
|
|
Binary file not shown.
After Width: | Height: | Size: 138 KiB |
Binary file not shown.
After Width: | Height: | Size: 47 KiB |
|
@ -1,22 +1,24 @@
|
|||
[
|
||||
{
|
||||
"schema": {
|
||||
"dataSource": "druidtest",
|
||||
"aggregators": [
|
||||
{
|
||||
"type": "count",
|
||||
"name": "impressions"
|
||||
},
|
||||
{
|
||||
"type": "doubleSum",
|
||||
"name": "wp",
|
||||
"fieldName": "wp"
|
||||
}
|
||||
],
|
||||
"indexGranularity": "minute",
|
||||
"shardSpec": {
|
||||
"type": "none"
|
||||
}
|
||||
"dataSource": "wikipedia",
|
||||
"aggregators" : [{
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "added",
|
||||
"fieldName" : "added"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "deleted",
|
||||
"fieldName" : "deleted"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "delta",
|
||||
"fieldName" : "delta"
|
||||
}],
|
||||
"indexGranularity": "none"
|
||||
},
|
||||
"config": {
|
||||
"maxRowsInMemory": 500000,
|
||||
|
@ -29,23 +31,20 @@
|
|||
"zk.connectiontimeout.ms": "15000",
|
||||
"zk.sessiontimeout.ms": "15000",
|
||||
"zk.synctime.ms": "5000",
|
||||
"groupid": "topic-pixel-local",
|
||||
"groupid": "druid-example",
|
||||
"fetch.size": "1048586",
|
||||
"autooffset.reset": "largest",
|
||||
"autocommit.enable": "false"
|
||||
},
|
||||
"feed": "druidtest",
|
||||
"feed": "wikipedia",
|
||||
"parser": {
|
||||
"timestampSpec": {
|
||||
"column": "utcdt",
|
||||
"format": "iso"
|
||||
"column": "timestamp"
|
||||
},
|
||||
"data": {
|
||||
"format": "json"
|
||||
},
|
||||
"dimensionExclusions": [
|
||||
"wp"
|
||||
]
|
||||
"format": "json",
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
}
|
||||
}
|
||||
},
|
||||
"plumber": {
|
|
@ -0,0 +1,5 @@
|
|||
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
|
||||
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
|
||||
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
|
||||
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
|
||||
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "cancer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
|
|
@ -0,0 +1,49 @@
|
|||
{
|
||||
"dataSource": "wikipedia",
|
||||
"timestampColumn": "timestamp",
|
||||
"timestampFormat": "iso",
|
||||
"dataSpec": {
|
||||
"format": "json",
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
},
|
||||
"granularitySpec" : {
|
||||
"type" : "uniform",
|
||||
"gran" : "DAY",
|
||||
"intervals" : [ "2013-08-31/2013-09-01" ]
|
||||
},
|
||||
"pathSpec": {
|
||||
"type": "static",
|
||||
"paths": "examples/indexing/wikipedia_data.json"
|
||||
},
|
||||
"rollupSpec": {
|
||||
"aggs": [{
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "added",
|
||||
"fieldName" : "added"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "deleted",
|
||||
"fieldName" : "deleted"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "delta",
|
||||
"fieldName" : "delta"
|
||||
}],
|
||||
"rollupGranularity": "none"
|
||||
},
|
||||
"workingPath": "\/tmp\/working_path",
|
||||
"segmentOutputPath": "\/tmp\/segments",
|
||||
"partitionsSpec": {
|
||||
"targetPartitionSize": 5000000
|
||||
},
|
||||
"updaterJobSpec": {
|
||||
"type": "db",
|
||||
"connectURI": "jdbc:mysql:\/\/localhost:3306\/druid",
|
||||
"user": "druid",
|
||||
"password": "diurd",
|
||||
"segmentTable": "druid_segments"
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
{
|
||||
"type" : "index_hadoop",
|
||||
"config": {
|
||||
"dataSource" : "wikipedia",
|
||||
"timestampColumn" : "timestamp",
|
||||
"timestampFormat" : "auto",
|
||||
"dataSpec" : {
|
||||
"format" : "json",
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
},
|
||||
"granularitySpec" : {
|
||||
"type" : "uniform",
|
||||
"gran" : "DAY",
|
||||
"intervals" : [ "2013-08-31/2013-09-01" ]
|
||||
},
|
||||
"pathSpec" : {
|
||||
"type" : "static",
|
||||
"paths" : "examples/indexing/wikipedia_data.json"
|
||||
},
|
||||
"targetPartitionSize" : 5000000,
|
||||
"rollupSpec" : {
|
||||
"aggs": [{
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "added",
|
||||
"fieldName" : "added"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "deleted",
|
||||
"fieldName" : "deleted"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "delta",
|
||||
"fieldName" : "delta"
|
||||
}],
|
||||
"rollupGranularity" : "none"
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,39 @@
|
|||
{
|
||||
"type" : "index",
|
||||
"dataSource" : "wikipedia",
|
||||
"granularitySpec" : {
|
||||
"type" : "uniform",
|
||||
"gran" : "DAY",
|
||||
"intervals" : [ "2013-08-31/2013-09-01" ]
|
||||
},
|
||||
"aggregators" : [{
|
||||
"type" : "count",
|
||||
"name" : "count"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "added",
|
||||
"fieldName" : "added"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "deleted",
|
||||
"fieldName" : "deleted"
|
||||
}, {
|
||||
"type" : "doubleSum",
|
||||
"name" : "delta",
|
||||
"fieldName" : "delta"
|
||||
}],
|
||||
"firehose" : {
|
||||
"type" : "local",
|
||||
"baseDir" : "examples/indexing/",
|
||||
"filter" : "wikipedia_data.json",
|
||||
"parser" : {
|
||||
"timestampSpec" : {
|
||||
"column" : "timestamp"
|
||||
},
|
||||
"data" : {
|
||||
"format" : "json",
|
||||
"dimensions" : ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"]
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -12,8 +12,8 @@
|
|||
},
|
||||
{
|
||||
"type":"longSum",
|
||||
"fieldName":"edit_count",
|
||||
"name":"count"
|
||||
"fieldName":"count",
|
||||
"name":"edit_count"
|
||||
}
|
||||
],
|
||||
"filter":{
|
||||
|
|
|
@ -60,7 +60,7 @@ ssh -q -f -i ~/.ssh/druid-keypair -o StrictHostKeyChecking=no ubuntu@${INSTANCE_
|
|||
echo "Prepared $INSTANCE_ADDRESS for druid."
|
||||
|
||||
# Now to scp a tarball up that can run druid!
|
||||
if [ -f ../../services/target/druid-services-*-SNAPSHOT-bin.tar.gz ];
|
||||
if [ -f ../../services/target/druid-services-*-bin.tar.gz ];
|
||||
then
|
||||
echo "Uploading druid tarball to server..."
|
||||
scp -i ~/.ssh/druid-keypair -o StrictHostKeyChecking=no ../../services/target/druid-services-*-bin.tar.gz ubuntu@${INSTANCE_ADDRESS}:
|
||||
|
|
|
@ -4,9 +4,6 @@ druid.port=8082
|
|||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
|
|
@ -4,6 +4,7 @@ druid.port=8081
|
|||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
# Dummy read only AWS account (used to download example data)
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
|
||||
|
|
|
@ -0,0 +1,14 @@
|
|||
druid.host=localhost
|
||||
druid.port=8087
|
||||
druid.service=overlord
|
||||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.selectors.indexing.serviceName=overlord
|
||||
druid.indexer.runner.javaOpts="-server -Xmx1g"
|
||||
druid.indexer.runner.startPort=8088
|
||||
druid.indexer.fork.property.druid.computation.buffer.size=268435456
|
|
@ -4,13 +4,8 @@ druid.port=8083
|
|||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
|
||||
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
|
||||
druid.db.connector.user=druid
|
||||
druid.db.connector.password=diurd
|
||||
|
||||
druid.realtime.specFile=config/realtime/realtime.spec
|
||||
|
||||
druid.processing.buffer.sizeBytes=10000000
|
|
@ -20,30 +20,25 @@
|
|||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-examples</artifactId>
|
||||
<name>druid-examples</name>
|
||||
<description>druid-examples</description>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<artifactId>druid-realtime</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-common</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
|
|
|
@ -20,15 +20,15 @@
|
|||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid.extensions</groupId>
|
||||
<artifactId>druid-hdfs-storage</artifactId>
|
||||
<name>druid-hdfs-storage</name>
|
||||
<description>druid-hdfs-storage</description>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -20,20 +20,20 @@
|
|||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-indexing-hadoop</artifactId>
|
||||
<name>druid-indexing-hadoop</name>
|
||||
<description>Druid Indexing Hadoop</description>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
|
|
|
@ -81,13 +81,13 @@ import java.util.Set;
|
|||
/**
|
||||
* Determines appropriate ShardSpecs for a job by determining whether or not partitioning is necessary, and if so,
|
||||
* choosing the best dimension that satisfies the criteria:
|
||||
*
|
||||
* <p/>
|
||||
* <ul>
|
||||
* <li>Must have exactly one value per row.</li>
|
||||
* <li>Must not generate oversized partitions. A dimension with N rows having the same value will necessarily
|
||||
* put all those rows in the same partition, and that partition may be much larger than the target size.</li>
|
||||
* <li>Must have exactly one value per row.</li>
|
||||
* <li>Must not generate oversized partitions. A dimension with N rows having the same value will necessarily
|
||||
* put all those rows in the same partition, and that partition may be much larger than the target size.</li>
|
||||
* </ul>
|
||||
*
|
||||
* <p/>
|
||||
* "Best" means a very high cardinality dimension, or, if none exist, the dimension that minimizes variation of
|
||||
* segment size relative to the target.
|
||||
*/
|
||||
|
@ -125,7 +125,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
* in the final segment.
|
||||
*/
|
||||
|
||||
if(!config.getPartitionsSpec().isAssumeGrouped()) {
|
||||
if (!config.getPartitionsSpec().isAssumeGrouped()) {
|
||||
final Job groupByJob = new Job(
|
||||
new Configuration(),
|
||||
String.format("%s-determine_partitions_groupby-%s", config.getDataSource(), config.getIntervals())
|
||||
|
@ -150,7 +150,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
groupByJob.submit();
|
||||
log.info("Job %s submitted, status available at: %s", groupByJob.getJobName(), groupByJob.getTrackingURL());
|
||||
|
||||
if(!groupByJob.waitForCompletion(true)) {
|
||||
if (!groupByJob.waitForCompletion(true)) {
|
||||
log.error("Job failed: %s", groupByJob.getJobID());
|
||||
return false;
|
||||
}
|
||||
|
@ -170,7 +170,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
|
||||
injectSystemProperties(dimSelectionJob);
|
||||
|
||||
if(!config.getPartitionsSpec().isAssumeGrouped()) {
|
||||
if (!config.getPartitionsSpec().isAssumeGrouped()) {
|
||||
// Read grouped data from the groupByJob.
|
||||
dimSelectionJob.setMapperClass(DeterminePartitionsDimSelectionPostGroupByMapper.class);
|
||||
dimSelectionJob.setInputFormatClass(SequenceFileInputFormat.class);
|
||||
|
@ -203,7 +203,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
dimSelectionJob.getTrackingURL()
|
||||
);
|
||||
|
||||
if(!dimSelectionJob.waitForCompletion(true)) {
|
||||
if (!dimSelectionJob.waitForCompletion(true)) {
|
||||
log.error("Job failed: %s", dimSelectionJob.getJobID().toString());
|
||||
return false;
|
||||
}
|
||||
|
@ -237,15 +237,15 @@ public class DeterminePartitionsJob implements Jobby
|
|||
}
|
||||
|
||||
shardSpecs.put(bucket, actualSpecs);
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
log.info("Path[%s] didn't exist!?", partitionInfoPath);
|
||||
}
|
||||
}
|
||||
config.setShardSpecs(shardSpecs);
|
||||
|
||||
return true;
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
@ -271,9 +271,9 @@ public class DeterminePartitionsJob implements Jobby
|
|||
{
|
||||
// Create group key, there are probably more efficient ways of doing this
|
||||
final Map<String, Set<String>> dims = Maps.newTreeMap();
|
||||
for(final String dim : inputRow.getDimensions()) {
|
||||
for (final String dim : inputRow.getDimensions()) {
|
||||
final Set<String> dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim));
|
||||
if(dimValues.size() > 0) {
|
||||
if (dimValues.size() > 0) {
|
||||
dims.put(dim, dimValues);
|
||||
}
|
||||
}
|
||||
|
@ -314,7 +314,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
protected void setup(Context context)
|
||||
throws IOException, InterruptedException
|
||||
{
|
||||
final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration());
|
||||
final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration());
|
||||
final String partitionDimension = config.getPartitionDimension();
|
||||
helper = new DeterminePartitionsDimSelectionMapperHelper(config, partitionDimension);
|
||||
}
|
||||
|
@ -346,7 +346,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
throws IOException, InterruptedException
|
||||
{
|
||||
super.setup(context);
|
||||
final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration());
|
||||
final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration());
|
||||
final String partitionDimension = config.getPartitionDimension();
|
||||
helper = new DeterminePartitionsDimSelectionMapperHelper(config, partitionDimension);
|
||||
}
|
||||
|
@ -359,7 +359,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
) throws IOException, InterruptedException
|
||||
{
|
||||
final Map<String, Iterable<String>> dims = Maps.newHashMap();
|
||||
for(final String dim : inputRow.getDimensions()) {
|
||||
for (final String dim : inputRow.getDimensions()) {
|
||||
dims.put(dim, inputRow.getDimension(dim));
|
||||
}
|
||||
helper.emitDimValueCounts(context, new DateTime(inputRow.getTimestampFromEpoch()), dims);
|
||||
|
@ -383,9 +383,9 @@ public class DeterminePartitionsJob implements Jobby
|
|||
|
||||
final ImmutableMap.Builder<DateTime, Integer> timeIndexBuilder = ImmutableMap.builder();
|
||||
int idx = 0;
|
||||
for(final Interval bucketInterval: config.getGranularitySpec().bucketIntervals()) {
|
||||
for (final Interval bucketInterval : config.getGranularitySpec().bucketIntervals()) {
|
||||
timeIndexBuilder.put(bucketInterval.getStart(), idx);
|
||||
idx ++;
|
||||
idx++;
|
||||
}
|
||||
|
||||
this.intervalIndexes = timeIndexBuilder.build();
|
||||
|
@ -399,7 +399,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
{
|
||||
final Optional<Interval> maybeInterval = config.getGranularitySpec().bucketInterval(timestamp);
|
||||
|
||||
if(!maybeInterval.isPresent()) {
|
||||
if (!maybeInterval.isPresent()) {
|
||||
throw new ISE("WTF?! No bucket found for timestamp: %s", timestamp);
|
||||
}
|
||||
|
||||
|
@ -414,13 +414,13 @@ public class DeterminePartitionsJob implements Jobby
|
|||
// Emit row-counter value.
|
||||
write(context, groupKey, new DimValueCount("", "", 1));
|
||||
|
||||
for(final Map.Entry<String, Iterable<String>> dimAndValues : dims.entrySet()) {
|
||||
for (final Map.Entry<String, Iterable<String>> dimAndValues : dims.entrySet()) {
|
||||
final String dim = dimAndValues.getKey();
|
||||
|
||||
if(partitionDimension == null || partitionDimension.equals(dim)) {
|
||||
if (partitionDimension == null || partitionDimension.equals(dim)) {
|
||||
final Iterable<String> dimValues = dimAndValues.getValue();
|
||||
|
||||
if(Iterables.size(dimValues) == 1) {
|
||||
if (Iterables.size(dimValues) == 1) {
|
||||
// Emit this value.
|
||||
write(context, groupKey, new DimValueCount(dim, Iterables.getOnlyElement(dimValues), 1));
|
||||
} else {
|
||||
|
@ -433,7 +433,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
}
|
||||
|
||||
public static class DeterminePartitionsDimSelectionPartitioner
|
||||
extends Partitioner<BytesWritable, Text>
|
||||
extends Partitioner<BytesWritable, Text>
|
||||
{
|
||||
@Override
|
||||
public int getPartition(BytesWritable bytesWritable, Text text, int numPartitions)
|
||||
|
@ -463,7 +463,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
if (config == null) {
|
||||
synchronized (DeterminePartitionsDimSelectionBaseReducer.class) {
|
||||
if (config == null) {
|
||||
config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration());
|
||||
config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -561,7 +561,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
final DimValueCount firstDvc = iterator.next();
|
||||
final int totalRows = firstDvc.numRows;
|
||||
|
||||
if(!firstDvc.dim.equals("") || !firstDvc.value.equals("")) {
|
||||
if (!firstDvc.dim.equals("") || !firstDvc.value.equals("")) {
|
||||
throw new IllegalStateException("WTF?! Expected total row indicator on first k/v pair!");
|
||||
}
|
||||
|
||||
|
@ -574,10 +574,10 @@ public class DeterminePartitionsJob implements Jobby
|
|||
// We'll store possible partitions in here
|
||||
final Map<String, DimPartitions> dimPartitionss = Maps.newHashMap();
|
||||
|
||||
while(iterator.hasNext()) {
|
||||
while (iterator.hasNext()) {
|
||||
final DimValueCount dvc = iterator.next();
|
||||
|
||||
if(currentDimPartitions == null || !currentDimPartitions.dim.equals(dvc.dim)) {
|
||||
if (currentDimPartitions == null || !currentDimPartitions.dim.equals(dvc.dim)) {
|
||||
// Starting a new dimension! Exciting!
|
||||
currentDimPartitions = new DimPartitions(dvc.dim);
|
||||
currentDimPartition = new DimPartition();
|
||||
|
@ -586,17 +586,17 @@ public class DeterminePartitionsJob implements Jobby
|
|||
}
|
||||
|
||||
// Respect poisoning
|
||||
if(!currentDimSkip && dvc.numRows < 0) {
|
||||
if (!currentDimSkip && dvc.numRows < 0) {
|
||||
log.info("Cannot partition on multi-valued dimension: %s", dvc.dim);
|
||||
currentDimSkip = true;
|
||||
}
|
||||
|
||||
if(currentDimSkip) {
|
||||
if (currentDimSkip) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// See if we need to cut a new partition ending immediately before this dimension value
|
||||
if(currentDimPartition.rows > 0 && currentDimPartition.rows + dvc.numRows >= config.getTargetPartitionSize()) {
|
||||
if (currentDimPartition.rows > 0 && currentDimPartition.rows + dvc.numRows >= config.getTargetPartitionSize()) {
|
||||
final ShardSpec shardSpec = new SingleDimensionShardSpec(
|
||||
currentDimPartitions.dim,
|
||||
currentDimPartitionStart,
|
||||
|
@ -618,20 +618,20 @@ public class DeterminePartitionsJob implements Jobby
|
|||
}
|
||||
|
||||
// Update counters
|
||||
currentDimPartition.cardinality ++;
|
||||
currentDimPartition.cardinality++;
|
||||
currentDimPartition.rows += dvc.numRows;
|
||||
|
||||
if(!iterator.hasNext() || !currentDimPartitions.dim.equals(iterator.peek().dim)) {
|
||||
if (!iterator.hasNext() || !currentDimPartitions.dim.equals(iterator.peek().dim)) {
|
||||
// Finalize the current dimension
|
||||
|
||||
if(currentDimPartition.rows > 0) {
|
||||
if (currentDimPartition.rows > 0) {
|
||||
// One more shard to go
|
||||
final ShardSpec shardSpec;
|
||||
|
||||
if (currentDimPartitions.partitions.isEmpty()) {
|
||||
shardSpec = new NoneShardSpec();
|
||||
} else {
|
||||
if(currentDimPartition.rows < config.getTargetPartitionSize() * SHARD_COMBINE_THRESHOLD) {
|
||||
if (currentDimPartition.rows < config.getTargetPartitionSize() * SHARD_COMBINE_THRESHOLD) {
|
||||
// Combine with previous shard
|
||||
final DimPartition previousDimPartition = currentDimPartitions.partitions.remove(
|
||||
currentDimPartitions.partitions.size() - 1
|
||||
|
@ -685,7 +685,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
}
|
||||
|
||||
// Choose best dimension
|
||||
if(dimPartitionss.isEmpty()) {
|
||||
if (dimPartitionss.isEmpty()) {
|
||||
throw new ISE("No suitable partitioning dimension found!");
|
||||
}
|
||||
|
||||
|
@ -694,8 +694,8 @@ public class DeterminePartitionsJob implements Jobby
|
|||
DimPartitions minDistancePartitions = null;
|
||||
DimPartitions maxCardinalityPartitions = null;
|
||||
|
||||
for(final DimPartitions dimPartitions : dimPartitionss.values()) {
|
||||
if(dimPartitions.getRows() != totalRows) {
|
||||
for (final DimPartitions dimPartitions : dimPartitionss.values()) {
|
||||
if (dimPartitions.getRows() != totalRows) {
|
||||
log.info(
|
||||
"Dimension[%s] is not present in all rows (row count %,d != expected row count %,d)",
|
||||
dimPartitions.dim,
|
||||
|
@ -708,32 +708,32 @@ public class DeterminePartitionsJob implements Jobby
|
|||
|
||||
// Make sure none of these shards are oversized
|
||||
boolean oversized = false;
|
||||
for(final DimPartition partition : dimPartitions.partitions) {
|
||||
if(partition.rows > config.getMaxPartitionSize()) {
|
||||
for (final DimPartition partition : dimPartitions.partitions) {
|
||||
if (partition.rows > config.getMaxPartitionSize()) {
|
||||
log.info("Dimension[%s] has an oversized shard: %s", dimPartitions.dim, partition.shardSpec);
|
||||
oversized = true;
|
||||
}
|
||||
}
|
||||
|
||||
if(oversized) {
|
||||
if (oversized) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final int cardinality = dimPartitions.getCardinality();
|
||||
final long distance = dimPartitions.getDistanceSquaredFromTarget(config.getTargetPartitionSize());
|
||||
|
||||
if(cardinality > maxCardinality) {
|
||||
if (cardinality > maxCardinality) {
|
||||
maxCardinality = cardinality;
|
||||
maxCardinalityPartitions = dimPartitions;
|
||||
}
|
||||
|
||||
if(distance < minDistance) {
|
||||
if (distance < minDistance) {
|
||||
minDistance = distance;
|
||||
minDistancePartitions = dimPartitions;
|
||||
}
|
||||
}
|
||||
|
||||
if(maxCardinalityPartitions == null) {
|
||||
if (maxCardinalityPartitions == null) {
|
||||
throw new ISE("No suitable partitioning dimension found!");
|
||||
}
|
||||
|
||||
|
@ -821,7 +821,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
public int getCardinality()
|
||||
{
|
||||
int sum = 0;
|
||||
for(final DimPartition dimPartition : partitions) {
|
||||
for (final DimPartition dimPartition : partitions) {
|
||||
sum += dimPartition.cardinality;
|
||||
}
|
||||
return sum;
|
||||
|
@ -830,7 +830,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
public long getDistanceSquaredFromTarget(long target)
|
||||
{
|
||||
long distance = 0;
|
||||
for(final DimPartition dimPartition : partitions) {
|
||||
for (final DimPartition dimPartition : partitions) {
|
||||
distance += (dimPartition.rows - target) * (dimPartition.rows - target);
|
||||
}
|
||||
|
||||
|
@ -841,7 +841,7 @@ public class DeterminePartitionsJob implements Jobby
|
|||
public int getRows()
|
||||
{
|
||||
int sum = 0;
|
||||
for(final DimPartition dimPartition : partitions) {
|
||||
for (final DimPartition dimPartition : partitions) {
|
||||
sum += dimPartition.rows;
|
||||
}
|
||||
return sum;
|
||||
|
@ -892,7 +892,11 @@ public class DeterminePartitionsJob implements Jobby
|
|||
throws IOException, InterruptedException
|
||||
{
|
||||
context.write(
|
||||
new SortableBytes(groupKey, tabJoiner.join(dimValueCount.dim, dimValueCount.value).getBytes(HadoopDruidIndexerConfig.javaNativeCharset)).toBytesWritable(),
|
||||
new SortableBytes(
|
||||
groupKey, tabJoiner.join(dimValueCount.dim, dimValueCount.value).getBytes(
|
||||
HadoopDruidIndexerConfig.javaNativeCharset
|
||||
)
|
||||
).toBytesWritable(),
|
||||
dimValueCount.toText()
|
||||
);
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.indexer;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
|
@ -68,7 +67,6 @@ import org.joda.time.DateTime;
|
|||
import org.joda.time.Interval;
|
||||
import org.joda.time.format.ISODateTimeFormat;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Collections;
|
||||
|
@ -80,13 +78,13 @@ import java.util.Set;
|
|||
*/
|
||||
public class HadoopDruidIndexerConfig
|
||||
{
|
||||
public static final Charset javaNativeCharset = Charset.forName("Unicode");
|
||||
|
||||
public static final Splitter tabSplitter = Splitter.on("\t");
|
||||
public static final Joiner tabJoiner = Joiner.on("\t");
|
||||
|
||||
private static final Logger log = new Logger(HadoopDruidIndexerConfig.class);
|
||||
private static final Injector injector;
|
||||
|
||||
public static final String CONFIG_PROPERTY = "druid.indexer.config";
|
||||
public static final Charset javaNativeCharset = Charset.forName("Unicode");
|
||||
public static final Splitter tabSplitter = Splitter.on("\t");
|
||||
public static final Joiner tabJoiner = Joiner.on("\t");
|
||||
public static final ObjectMapper jsonMapper;
|
||||
|
||||
static {
|
||||
|
@ -113,85 +111,33 @@ public class HadoopDruidIndexerConfig
|
|||
INVALID_ROW_COUNTER
|
||||
}
|
||||
|
||||
public static HadoopDruidIndexerConfig fromMap(Map<String, Object> argSpec)
|
||||
{
|
||||
return jsonMapper.convertValue(argSpec, HadoopDruidIndexerConfig.class);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static HadoopDruidIndexerConfig fromFile(File file)
|
||||
{
|
||||
try {
|
||||
return fromMap((Map<String, Object>) jsonMapper.readValue(file, new TypeReference<Map<String, Object>>(){}));
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static HadoopDruidIndexerConfig fromString(String str)
|
||||
{
|
||||
try {
|
||||
return fromMap(
|
||||
(Map<String, Object>) jsonMapper.readValue(
|
||||
str, new TypeReference<Map<String, Object>>()
|
||||
{
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static HadoopDruidIndexerConfig fromConfiguration(Configuration conf)
|
||||
{
|
||||
final HadoopDruidIndexerConfig retVal = fromString(conf.get(CONFIG_PROPERTY));
|
||||
retVal.verify();
|
||||
return retVal;
|
||||
}
|
||||
|
||||
private static final Logger log = new Logger(HadoopDruidIndexerConfig.class);
|
||||
|
||||
private static final String CONFIG_PROPERTY = "druid.indexer.config";
|
||||
|
||||
private volatile String dataSource;
|
||||
private volatile String timestampColumnName;
|
||||
private volatile String timestampFormat;
|
||||
private volatile TimestampSpec timestampSpec;
|
||||
private volatile DataSpec dataSpec;
|
||||
@Deprecated
|
||||
private volatile Granularity segmentGranularity;
|
||||
private volatile GranularitySpec granularitySpec;
|
||||
private volatile PathSpec pathSpec;
|
||||
private volatile String jobOutputDir;
|
||||
private volatile String segmentOutputDir;
|
||||
private volatile String version = new DateTime().toString();
|
||||
private volatile String workingPath;
|
||||
private volatile String segmentOutputPath;
|
||||
private volatile String version;
|
||||
private volatile PartitionsSpec partitionsSpec;
|
||||
private volatile boolean leaveIntermediate = false;
|
||||
private volatile boolean cleanupOnFailure = true;
|
||||
private volatile Map<DateTime, List<HadoopyShardSpec>> shardSpecs = ImmutableMap.of();
|
||||
private volatile boolean overwriteFiles = false;
|
||||
private volatile boolean leaveIntermediate;
|
||||
private volatile boolean cleanupOnFailure;
|
||||
private volatile Map<DateTime, List<HadoopyShardSpec>> shardSpecs;
|
||||
private volatile boolean overwriteFiles;
|
||||
private volatile DataRollupSpec rollupSpec;
|
||||
private volatile DbUpdaterJobSpec updaterJobSpec;
|
||||
private volatile boolean ignoreInvalidRows = false;
|
||||
private volatile boolean ignoreInvalidRows;
|
||||
|
||||
@JsonCreator
|
||||
public HadoopDruidIndexerConfig(
|
||||
final @JsonProperty("intervals") List<Interval> intervals,
|
||||
final @JsonProperty("dataSource") String dataSource,
|
||||
final @JsonProperty("timestampColumn") String timestampColumnName,
|
||||
final @JsonProperty("timestampFormat") String timestampFormat,
|
||||
final @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
|
||||
final @JsonProperty("dataSpec") DataSpec dataSpec,
|
||||
final @JsonProperty("segmentGranularity") Granularity segmentGranularity,
|
||||
final @JsonProperty("granularitySpec") GranularitySpec granularitySpec,
|
||||
final @JsonProperty("pathSpec") PathSpec pathSpec,
|
||||
final @JsonProperty("workingPath") String jobOutputDir,
|
||||
final @JsonProperty("segmentOutputPath") String segmentOutputDir,
|
||||
final @JsonProperty("workingPath") String workingPath,
|
||||
final @JsonProperty("segmentOutputPath") String segmentOutputPath,
|
||||
final @JsonProperty("version") String version,
|
||||
final @JsonProperty("partitionDimension") String partitionDimension,
|
||||
final @JsonProperty("targetPartitionSize") Long targetPartitionSize,
|
||||
final @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec,
|
||||
final @JsonProperty("leaveIntermediate") boolean leaveIntermediate,
|
||||
final @JsonProperty("cleanupOnFailure") Boolean cleanupOnFailure,
|
||||
|
@ -199,48 +145,51 @@ public class HadoopDruidIndexerConfig
|
|||
final @JsonProperty("overwriteFiles") boolean overwriteFiles,
|
||||
final @JsonProperty("rollupSpec") DataRollupSpec rollupSpec,
|
||||
final @JsonProperty("updaterJobSpec") DbUpdaterJobSpec updaterJobSpec,
|
||||
final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows
|
||||
final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows,
|
||||
// These fields are deprecated and will be removed in the future
|
||||
final @JsonProperty("timestampColumn") String timestampColumn,
|
||||
final @JsonProperty("timestampFormat") String timestampFormat,
|
||||
final @JsonProperty("intervals") List<Interval> intervals,
|
||||
final @JsonProperty("segmentGranularity") Granularity segmentGranularity,
|
||||
final @JsonProperty("partitionDimension") String partitionDimension,
|
||||
final @JsonProperty("targetPartitionSize") Long targetPartitionSize
|
||||
)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
this.timestampColumnName = (timestampColumnName == null) ? null : timestampColumnName.toLowerCase();
|
||||
this.timestampFormat = timestampFormat;
|
||||
this.timestampSpec = (timestampSpec == null) ? new TimestampSpec(timestampColumn, timestampFormat) : timestampSpec;
|
||||
this.dataSpec = dataSpec;
|
||||
this.granularitySpec = granularitySpec;
|
||||
this.pathSpec = pathSpec;
|
||||
this.jobOutputDir = jobOutputDir;
|
||||
this.segmentOutputDir = segmentOutputDir;
|
||||
this.workingPath = workingPath;
|
||||
this.segmentOutputPath = segmentOutputPath;
|
||||
this.version = version == null ? new DateTime().toString() : version;
|
||||
this.partitionsSpec = partitionsSpec;
|
||||
this.leaveIntermediate = leaveIntermediate;
|
||||
this.cleanupOnFailure = (cleanupOnFailure == null ? true : cleanupOnFailure);
|
||||
this.shardSpecs = shardSpecs;
|
||||
this.shardSpecs = (shardSpecs == null ? ImmutableMap.<DateTime, List<HadoopyShardSpec>>of() : shardSpecs);
|
||||
this.overwriteFiles = overwriteFiles;
|
||||
this.rollupSpec = rollupSpec;
|
||||
this.updaterJobSpec = updaterJobSpec;
|
||||
this.ignoreInvalidRows = ignoreInvalidRows;
|
||||
|
||||
if(partitionsSpec != null) {
|
||||
if (partitionsSpec != null) {
|
||||
Preconditions.checkArgument(
|
||||
partitionDimension == null && targetPartitionSize == null,
|
||||
"Cannot mix partitionsSpec with partitionDimension/targetPartitionSize"
|
||||
);
|
||||
|
||||
this.partitionsSpec = partitionsSpec;
|
||||
} else {
|
||||
// Backwards compatibility
|
||||
this.partitionsSpec = new PartitionsSpec(partitionDimension, targetPartitionSize, null, false);
|
||||
}
|
||||
|
||||
if(granularitySpec != null) {
|
||||
if (granularitySpec != null) {
|
||||
Preconditions.checkArgument(
|
||||
segmentGranularity == null && intervals == null,
|
||||
"Cannot mix granularitySpec with segmentGranularity/intervals"
|
||||
);
|
||||
this.granularitySpec = granularitySpec;
|
||||
} else {
|
||||
// Backwards compatibility
|
||||
this.segmentGranularity = segmentGranularity;
|
||||
if(segmentGranularity != null && intervals != null) {
|
||||
if (segmentGranularity != null && intervals != null) {
|
||||
this.granularitySpec = new UniformGranularitySpec(segmentGranularity, intervals);
|
||||
}
|
||||
}
|
||||
|
@ -253,21 +202,6 @@ public class HadoopDruidIndexerConfig
|
|||
{
|
||||
}
|
||||
|
||||
public List<Interval> getIntervals()
|
||||
{
|
||||
return JodaUtils.condenseIntervals(getGranularitySpec().bucketIntervals());
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
public void setIntervals(List<Interval> intervals)
|
||||
{
|
||||
Preconditions.checkState(this.granularitySpec == null, "Cannot mix setIntervals with granularitySpec");
|
||||
Preconditions.checkState(this.segmentGranularity != null, "Cannot use setIntervals without segmentGranularity");
|
||||
|
||||
// For backwards compatibility
|
||||
this.granularitySpec = new UniformGranularitySpec(this.segmentGranularity, intervals);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
|
@ -279,31 +213,15 @@ public class HadoopDruidIndexerConfig
|
|||
this.dataSource = dataSource.toLowerCase();
|
||||
}
|
||||
|
||||
@JsonProperty("timestampColumn")
|
||||
public String getTimestampColumnName()
|
||||
{
|
||||
return timestampColumnName;
|
||||
}
|
||||
|
||||
public void setTimestampColumnName(String timestampColumnName)
|
||||
{
|
||||
this.timestampColumnName = timestampColumnName.toLowerCase();
|
||||
}
|
||||
|
||||
@JsonProperty()
|
||||
public String getTimestampFormat()
|
||||
{
|
||||
return timestampFormat;
|
||||
}
|
||||
|
||||
public void setTimestampFormat(String timestampFormat)
|
||||
{
|
||||
this.timestampFormat = timestampFormat;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public TimestampSpec getTimestampSpec()
|
||||
{
|
||||
return new TimestampSpec(timestampColumnName, timestampFormat);
|
||||
return timestampSpec;
|
||||
}
|
||||
|
||||
public void setTimestampSpec(TimestampSpec timestampSpec)
|
||||
{
|
||||
this.timestampSpec = timestampSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -317,32 +235,6 @@ public class HadoopDruidIndexerConfig
|
|||
this.dataSpec = new ToLowercaseDataSpec(dataSpec);
|
||||
}
|
||||
|
||||
public StringInputRowParser getParser()
|
||||
{
|
||||
final List<String> dimensionExclusions;
|
||||
|
||||
if(getDataSpec().hasCustomDimensions()) {
|
||||
dimensionExclusions = null;
|
||||
} else {
|
||||
dimensionExclusions = Lists.newArrayList();
|
||||
dimensionExclusions.add(getTimestampColumnName());
|
||||
dimensionExclusions.addAll(
|
||||
Lists.transform(
|
||||
getRollupSpec().getAggs(), new Function<AggregatorFactory, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(AggregatorFactory aggregatorFactory)
|
||||
{
|
||||
return aggregatorFactory.getName();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
return new StringInputRowParser(getTimestampSpec(), getDataSpec(), dimensionExclusions);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public GranularitySpec getGranularitySpec()
|
||||
{
|
||||
|
@ -354,17 +246,6 @@ public class HadoopDruidIndexerConfig
|
|||
this.granularitySpec = granularitySpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public PartitionsSpec getPartitionsSpec()
|
||||
{
|
||||
return partitionsSpec;
|
||||
}
|
||||
|
||||
public void setPartitionsSpec(PartitionsSpec partitionsSpec)
|
||||
{
|
||||
this.partitionsSpec = partitionsSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public PathSpec getPathSpec()
|
||||
{
|
||||
|
@ -376,26 +257,26 @@ public class HadoopDruidIndexerConfig
|
|||
this.pathSpec = pathSpec;
|
||||
}
|
||||
|
||||
@JsonProperty("workingPath")
|
||||
public String getJobOutputDir()
|
||||
@JsonProperty
|
||||
public String getWorkingPath()
|
||||
{
|
||||
return jobOutputDir;
|
||||
return workingPath;
|
||||
}
|
||||
|
||||
public void setJobOutputDir(String jobOutputDir)
|
||||
public void setWorkingPath(String workingPath)
|
||||
{
|
||||
this.jobOutputDir = jobOutputDir;
|
||||
this.workingPath = workingPath;
|
||||
}
|
||||
|
||||
@JsonProperty("segmentOutputPath")
|
||||
public String getSegmentOutputDir()
|
||||
@JsonProperty
|
||||
public String getSegmentOutputPath()
|
||||
{
|
||||
return segmentOutputDir;
|
||||
return segmentOutputPath;
|
||||
}
|
||||
|
||||
public void setSegmentOutputDir(String segmentOutputDir)
|
||||
public void setSegmentOutputPath(String segmentOutputPath)
|
||||
{
|
||||
this.segmentOutputDir = segmentOutputDir;
|
||||
this.segmentOutputPath = segmentOutputPath;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -409,29 +290,15 @@ public class HadoopDruidIndexerConfig
|
|||
this.version = version;
|
||||
}
|
||||
|
||||
public String getPartitionDimension()
|
||||
@JsonProperty
|
||||
public PartitionsSpec getPartitionsSpec()
|
||||
{
|
||||
return partitionsSpec.getPartitionDimension();
|
||||
return partitionsSpec;
|
||||
}
|
||||
|
||||
public boolean partitionByDimension()
|
||||
public void setPartitionsSpec(PartitionsSpec partitionsSpec)
|
||||
{
|
||||
return partitionsSpec.isDeterminingPartitions();
|
||||
}
|
||||
|
||||
public Long getTargetPartitionSize()
|
||||
{
|
||||
return partitionsSpec.getTargetPartitionSize();
|
||||
}
|
||||
|
||||
public long getMaxPartitionSize()
|
||||
{
|
||||
return partitionsSpec.getMaxPartitionSize();
|
||||
}
|
||||
|
||||
public boolean isUpdaterJobSpecSet()
|
||||
{
|
||||
return (updaterJobSpec != null);
|
||||
this.partitionsSpec = partitionsSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -511,6 +378,72 @@ public class HadoopDruidIndexerConfig
|
|||
this.ignoreInvalidRows = ignoreInvalidRows;
|
||||
}
|
||||
|
||||
public List<Interval> getIntervals()
|
||||
{
|
||||
return JodaUtils.condenseIntervals(getGranularitySpec().bucketIntervals());
|
||||
}
|
||||
|
||||
public String getPartitionDimension()
|
||||
{
|
||||
return partitionsSpec.getPartitionDimension();
|
||||
}
|
||||
|
||||
public boolean partitionByDimension()
|
||||
{
|
||||
return partitionsSpec.isDeterminingPartitions();
|
||||
}
|
||||
|
||||
public Long getTargetPartitionSize()
|
||||
{
|
||||
return partitionsSpec.getTargetPartitionSize();
|
||||
}
|
||||
|
||||
public long getMaxPartitionSize()
|
||||
{
|
||||
return partitionsSpec.getMaxPartitionSize();
|
||||
}
|
||||
|
||||
public boolean isUpdaterJobSpecSet()
|
||||
{
|
||||
return (updaterJobSpec != null);
|
||||
}
|
||||
|
||||
public StringInputRowParser getParser()
|
||||
{
|
||||
final List<String> dimensionExclusions;
|
||||
|
||||
if (getDataSpec().hasCustomDimensions()) {
|
||||
dimensionExclusions = null;
|
||||
} else {
|
||||
dimensionExclusions = Lists.newArrayList();
|
||||
dimensionExclusions.add(timestampSpec.getTimestampColumn());
|
||||
dimensionExclusions.addAll(
|
||||
Lists.transform(
|
||||
getRollupSpec().getAggs(), new Function<AggregatorFactory, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(AggregatorFactory aggregatorFactory)
|
||||
{
|
||||
return aggregatorFactory.getName();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
return new StringInputRowParser(getTimestampSpec(), getDataSpec(), dimensionExclusions);
|
||||
}
|
||||
|
||||
public HadoopyShardSpec getShardSpec(Bucket bucket)
|
||||
{
|
||||
return shardSpecs.get(bucket.time).get(bucket.partitionNum);
|
||||
}
|
||||
|
||||
public Job addInputPaths(Job job) throws IOException
|
||||
{
|
||||
return getPathSpec().addInputPaths(this, job);
|
||||
}
|
||||
|
||||
/********************************************
|
||||
Granularity/Bucket Helper Methods
|
||||
********************************************/
|
||||
|
@ -590,11 +523,6 @@ public class HadoopDruidIndexerConfig
|
|||
);
|
||||
}
|
||||
|
||||
public HadoopyShardSpec getShardSpec(Bucket bucket)
|
||||
{
|
||||
return shardSpecs.get(bucket.time).get(bucket.partitionNum);
|
||||
}
|
||||
|
||||
/******************************************
|
||||
Path helper logic
|
||||
******************************************/
|
||||
|
@ -606,7 +534,7 @@ public class HadoopDruidIndexerConfig
|
|||
*/
|
||||
public Path makeIntermediatePath()
|
||||
{
|
||||
return new Path(String.format("%s/%s/%s", getJobOutputDir(), dataSource, getVersion().replace(":", "")));
|
||||
return new Path(String.format("%s/%s/%s", getWorkingPath(), getDataSource(), getVersion().replace(":", "")));
|
||||
}
|
||||
|
||||
public Path makeSegmentPartitionInfoPath(Bucket bucket)
|
||||
|
@ -638,38 +566,33 @@ public class HadoopDruidIndexerConfig
|
|||
return new Path(makeDescriptorInfoDir(), String.format("%s.json", segment.getIdentifier().replace(":", "")));
|
||||
}
|
||||
|
||||
public Path makeSegmentOutputPath(FileSystem fileSystem, Bucket bucket)
|
||||
{
|
||||
final Interval bucketInterval = getGranularitySpec().bucketInterval(bucket.time).get();
|
||||
if (fileSystem instanceof DistributedFileSystem)
|
||||
{
|
||||
return new Path(
|
||||
String.format(
|
||||
"%s/%s/%s_%s/%s/%s",
|
||||
getSegmentOutputDir(),
|
||||
dataSource,
|
||||
bucketInterval.getStart().toString(ISODateTimeFormat.basicDateTime()),
|
||||
bucketInterval.getEnd().toString(ISODateTimeFormat.basicDateTime()),
|
||||
getVersion().replace(":", "_"),
|
||||
bucket.partitionNum
|
||||
)
|
||||
);
|
||||
}
|
||||
return new Path(
|
||||
String.format(
|
||||
"%s/%s/%s_%s/%s/%s",
|
||||
getSegmentOutputDir(),
|
||||
dataSource,
|
||||
bucketInterval.getStart().toString(),
|
||||
bucketInterval.getEnd().toString(),
|
||||
getVersion(),
|
||||
bucket.partitionNum
|
||||
));
|
||||
}
|
||||
|
||||
public Job addInputPaths(Job job) throws IOException
|
||||
public Path makeSegmentOutputPath(FileSystem fileSystem, Bucket bucket)
|
||||
{
|
||||
return pathSpec.addInputPaths(this, job);
|
||||
final Interval bucketInterval = getGranularitySpec().bucketInterval(bucket.time).get();
|
||||
if (fileSystem instanceof DistributedFileSystem) {
|
||||
return new Path(
|
||||
String.format(
|
||||
"%s/%s/%s_%s/%s/%s",
|
||||
getSegmentOutputPath(),
|
||||
getDataSource(),
|
||||
bucketInterval.getStart().toString(ISODateTimeFormat.basicDateTime()),
|
||||
bucketInterval.getEnd().toString(ISODateTimeFormat.basicDateTime()),
|
||||
getVersion().replace(":", "_"),
|
||||
bucket.partitionNum
|
||||
)
|
||||
);
|
||||
}
|
||||
return new Path(
|
||||
String.format(
|
||||
"%s/%s/%s_%s/%s/%s",
|
||||
getSegmentOutputPath(),
|
||||
getDataSource(),
|
||||
bucketInterval.getStart().toString(),
|
||||
bucketInterval.getEnd().toString(),
|
||||
getVersion(),
|
||||
bucket.partitionNum
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public void intoConfiguration(Job job)
|
||||
|
@ -677,7 +600,7 @@ public class HadoopDruidIndexerConfig
|
|||
Configuration conf = job.getConfiguration();
|
||||
|
||||
try {
|
||||
conf.set(CONFIG_PROPERTY, jsonMapper.writeValueAsString(this));
|
||||
conf.set(HadoopDruidIndexerConfig.CONFIG_PROPERTY, HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(this));
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
|
@ -695,12 +618,11 @@ public class HadoopDruidIndexerConfig
|
|||
|
||||
Preconditions.checkNotNull(dataSource, "dataSource");
|
||||
Preconditions.checkNotNull(dataSpec, "dataSpec");
|
||||
Preconditions.checkNotNull(timestampColumnName, "timestampColumn");
|
||||
Preconditions.checkNotNull(timestampFormat, "timestampFormat");
|
||||
Preconditions.checkNotNull(timestampSpec, "timestampSpec");
|
||||
Preconditions.checkNotNull(granularitySpec, "granularitySpec");
|
||||
Preconditions.checkNotNull(pathSpec, "pathSpec");
|
||||
Preconditions.checkNotNull(jobOutputDir, "workingPath");
|
||||
Preconditions.checkNotNull(segmentOutputDir, "segmentOutputPath");
|
||||
Preconditions.checkNotNull(workingPath, "workingPath");
|
||||
Preconditions.checkNotNull(segmentOutputPath, "segmentOutputPath");
|
||||
Preconditions.checkNotNull(version, "version");
|
||||
Preconditions.checkNotNull(rollupSpec, "rollupSpec");
|
||||
|
||||
|
|
|
@ -0,0 +1,278 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexer;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.druid.data.input.impl.DataSpec;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.indexer.granularity.GranularitySpec;
|
||||
import io.druid.indexer.partitions.PartitionsSpec;
|
||||
import io.druid.indexer.path.PathSpec;
|
||||
import io.druid.indexer.rollup.DataRollupSpec;
|
||||
import io.druid.indexer.updater.DbUpdaterJobSpec;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class HadoopDruidIndexerConfigBuilder
|
||||
{
|
||||
public static HadoopDruidIndexerConfig fromSchema(HadoopDruidIndexerSchema schema)
|
||||
{
|
||||
return HadoopDruidIndexerConfig.jsonMapper.convertValue(schema, HadoopDruidIndexerConfig.class);
|
||||
}
|
||||
|
||||
public static HadoopDruidIndexerConfig fromMap(Map<String, Object> argSpec)
|
||||
{
|
||||
return HadoopDruidIndexerConfig.jsonMapper.convertValue(argSpec, HadoopDruidIndexerConfig.class);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static HadoopDruidIndexerConfig fromFile(File file)
|
||||
{
|
||||
try {
|
||||
return fromMap(
|
||||
(Map<String, Object>) HadoopDruidIndexerConfig.jsonMapper.readValue(
|
||||
file, new TypeReference<Map<String, Object>>()
|
||||
{
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static HadoopDruidIndexerConfig fromString(String str)
|
||||
{
|
||||
try {
|
||||
return fromMap(
|
||||
(Map<String, Object>) HadoopDruidIndexerConfig.jsonMapper.readValue(
|
||||
str, new TypeReference<Map<String, Object>>()
|
||||
{
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public static HadoopDruidIndexerConfig fromConfiguration(Configuration conf)
|
||||
{
|
||||
final HadoopDruidIndexerConfig retVal = fromString(conf.get(HadoopDruidIndexerConfig.CONFIG_PROPERTY));
|
||||
retVal.verify();
|
||||
return retVal;
|
||||
}
|
||||
|
||||
private volatile String dataSource;
|
||||
private volatile TimestampSpec timestampSpec;
|
||||
private volatile DataSpec dataSpec;
|
||||
private volatile GranularitySpec granularitySpec;
|
||||
private volatile PathSpec pathSpec;
|
||||
private volatile String workingPath;
|
||||
private volatile String segmentOutputPath;
|
||||
private volatile String version;
|
||||
private volatile PartitionsSpec partitionsSpec;
|
||||
private volatile boolean leaveIntermediate;
|
||||
private volatile boolean cleanupOnFailure;
|
||||
private volatile Map<DateTime, List<HadoopyShardSpec>> shardSpecs;
|
||||
private volatile boolean overwriteFiles;
|
||||
private volatile DataRollupSpec rollupSpec;
|
||||
private volatile DbUpdaterJobSpec updaterJobSpec;
|
||||
private volatile boolean ignoreInvalidRows;
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder()
|
||||
{
|
||||
this.dataSource = null;
|
||||
this.timestampSpec = null;
|
||||
this.dataSpec = null;
|
||||
this.granularitySpec = null;
|
||||
this.pathSpec = null;
|
||||
this.workingPath = null;
|
||||
this.segmentOutputPath = null;
|
||||
this.version = new DateTime().toString();
|
||||
this.partitionsSpec = null;
|
||||
this.leaveIntermediate = false;
|
||||
this.cleanupOnFailure = true;
|
||||
this.shardSpecs = ImmutableMap.of();
|
||||
this.overwriteFiles = false;
|
||||
this.rollupSpec = null;
|
||||
this.updaterJobSpec = null;
|
||||
this.ignoreInvalidRows = false;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withDataSource(String dataSource)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withTimestampSpec(TimestampSpec timestampSpec)
|
||||
{
|
||||
this.timestampSpec = timestampSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withDataSpec(DataSpec dataSpec)
|
||||
{
|
||||
this.dataSpec = dataSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withGranularitySpec(GranularitySpec granularitySpec)
|
||||
{
|
||||
this.granularitySpec = granularitySpec;
|
||||
return this;
|
||||
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withPathSpec(PathSpec pathSpec)
|
||||
{
|
||||
this.pathSpec = pathSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withWorkingPath(String workingPath)
|
||||
{
|
||||
this.workingPath = workingPath;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withSegmentOutputPath(String segmentOutputPath)
|
||||
{
|
||||
this.segmentOutputPath = segmentOutputPath;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withVersion(String version)
|
||||
{
|
||||
this.version = version;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withPartitionsSpec(PartitionsSpec partitionsSpec)
|
||||
{
|
||||
this.partitionsSpec = partitionsSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withLeaveIntermediate(boolean leaveIntermediate)
|
||||
{
|
||||
this.leaveIntermediate = leaveIntermediate;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withCleanupOnFailure(boolean cleanupOnFailure)
|
||||
{
|
||||
this.cleanupOnFailure = cleanupOnFailure;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withShardSpecs(Map<DateTime, List<HadoopyShardSpec>> shardSpecs)
|
||||
{
|
||||
this.shardSpecs = shardSpecs;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withOverwriteFiles(boolean overwriteFiles)
|
||||
{
|
||||
this.overwriteFiles = overwriteFiles;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withRollupSpec(DataRollupSpec rollupSpec)
|
||||
{
|
||||
this.rollupSpec = rollupSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withUpdaterJobSpec(DbUpdaterJobSpec updaterJobSpec)
|
||||
{
|
||||
this.updaterJobSpec = updaterJobSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withIgnoreInvalidRows(boolean ignoreInvalidRows)
|
||||
{
|
||||
this.ignoreInvalidRows = ignoreInvalidRows;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfigBuilder withSchema(HadoopDruidIndexerSchema schema)
|
||||
{
|
||||
this.dataSource = schema.getDataSource();
|
||||
this.timestampSpec = schema.getTimestampSpec();
|
||||
this.dataSpec = schema.getDataSpec();
|
||||
this.granularitySpec = schema.getGranularitySpec();
|
||||
this.pathSpec = HadoopDruidIndexerConfig.jsonMapper.convertValue(schema.getPathSpec(), PathSpec.class);
|
||||
this.workingPath = schema.getWorkingPath();
|
||||
this.segmentOutputPath = schema.getSegmentOutputPath();
|
||||
this.version = schema.getVersion();
|
||||
this.partitionsSpec = schema.getPartitionsSpec();
|
||||
this.leaveIntermediate = schema.isLeaveIntermediate();
|
||||
this.cleanupOnFailure = schema.isCleanupOnFailure();
|
||||
this.shardSpecs = schema.getShardSpecs();
|
||||
this.overwriteFiles = schema.isOverwriteFiles();
|
||||
this.rollupSpec = schema.getRollupSpec();
|
||||
this.updaterJobSpec = schema.getUpdaterJobSpec();
|
||||
this.ignoreInvalidRows = schema.isIgnoreInvalidRows();
|
||||
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerConfig build()
|
||||
{
|
||||
return new HadoopDruidIndexerConfig(
|
||||
dataSource,
|
||||
timestampSpec,
|
||||
dataSpec,
|
||||
granularitySpec,
|
||||
pathSpec,
|
||||
workingPath,
|
||||
segmentOutputPath,
|
||||
version,
|
||||
partitionsSpec,
|
||||
leaveIntermediate,
|
||||
cleanupOnFailure,
|
||||
shardSpecs,
|
||||
overwriteFiles,
|
||||
rollupSpec,
|
||||
updaterJobSpec,
|
||||
ignoreInvalidRows,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
}
|
|
@ -46,6 +46,7 @@ public class HadoopDruidIndexerJob implements Jobby
|
|||
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
private final DbUpdaterJob dbUpdaterJob;
|
||||
|
||||
private IndexGeneratorJob indexJob;
|
||||
private volatile List<DataSegment> publishedSegments = null;
|
||||
|
||||
|
|
|
@ -1,123 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexer;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class HadoopDruidIndexerMain
|
||||
{
|
||||
private static final Logger log = new Logger(HadoopDruidIndexerMain.class);
|
||||
|
||||
public static void main(String[] args) throws Exception
|
||||
{
|
||||
if (args.length != 1) {
|
||||
printHelp();
|
||||
System.exit(2);
|
||||
}
|
||||
|
||||
HadoopDruidIndexerNode node = HadoopDruidIndexerNode.builder().build();
|
||||
|
||||
node.setArgumentSpec(args[0]);
|
||||
|
||||
Lifecycle lifecycle = new Lifecycle();
|
||||
lifecycle.addManagedInstance(node);
|
||||
|
||||
try {
|
||||
lifecycle.start();
|
||||
}
|
||||
catch (Throwable t) {
|
||||
log.info(t, "Throwable caught at startup, committing seppuku");
|
||||
Thread.sleep(500);
|
||||
printHelp();
|
||||
System.exit(1);
|
||||
}
|
||||
}
|
||||
|
||||
private static final List<Pair<String, String>> expectedFields =
|
||||
ImmutableList.<Pair<String, String>>builder()
|
||||
.add(Pair.of("dataSource", "Name of dataSource"))
|
||||
.add(Pair.of("timestampColumn", "Column name of the timestamp column"))
|
||||
.add(Pair.of("timestampFormat", "Format name of the timestamp column (posix or iso)"))
|
||||
.add(
|
||||
Pair.of(
|
||||
"dataSpec",
|
||||
"A JSON object with fields "
|
||||
+
|
||||
"format=(json, csv, tsv), "
|
||||
+
|
||||
"columns=JSON array of column names for the delimited text input file (only for csv and tsv formats),"
|
||||
+
|
||||
"dimensions=JSON array of dimensionn names (must match names in columns),"
|
||||
+
|
||||
"delimiter=delimiter of the data (only for tsv format)"
|
||||
)
|
||||
)
|
||||
.add(Pair.of("granularitySpec", "A JSON object indicating the Granularity that segments should be created at."))
|
||||
.add(
|
||||
Pair.of(
|
||||
"pathSpec",
|
||||
"A JSON object with fields type=granularity, inputPath, filePattern, dataGranularity"
|
||||
)
|
||||
)
|
||||
.add(
|
||||
Pair.of(
|
||||
"rollupSpec",
|
||||
"JSON object with fields rollupGranularity, aggs=JSON Array of Aggregator specs"
|
||||
)
|
||||
)
|
||||
.add(Pair.of("workingPath", "Path to store intermediate output data. Deleted when finished."))
|
||||
.add(Pair.of("segmentOutputPath", "Path to store output segments."))
|
||||
.add(
|
||||
Pair.of(
|
||||
"updaterJobSpec",
|
||||
"JSON object with fields type=db, connectURI of the database, username, password, and segment table name"
|
||||
)
|
||||
)
|
||||
.add(Pair.of("cleanupOnFailure", "Clean up intermediate files on failure? (default: true)"))
|
||||
.add(Pair.of("leaveIntermediate", "Leave intermediate files. (default: false)"))
|
||||
.add(Pair.of("partitionDimension", "Dimension to partition by (optional)"))
|
||||
.add(
|
||||
Pair.of(
|
||||
"targetPartitionSize",
|
||||
"Integer representing the target number of rows in a partition (required if partitionDimension != null)"
|
||||
)
|
||||
)
|
||||
.build();
|
||||
|
||||
public static void printHelp()
|
||||
{
|
||||
System.out.println("Usage: <java invocation> <config_spec>");
|
||||
System.out.println("<config_spec> is either a JSON object or the path to a file that contains a JSON object.");
|
||||
System.out.println();
|
||||
System.out.println("JSON object description:");
|
||||
System.out.println("{");
|
||||
for (Pair<String, String> expectedField : expectedFields) {
|
||||
System.out.printf(" \"%s\": %s%n", expectedField.lhs, expectedField.rhs);
|
||||
}
|
||||
System.out.println("}");
|
||||
}
|
||||
}
|
|
@ -38,7 +38,7 @@ public abstract class HadoopDruidIndexerMapper<KEYOUT, VALUEOUT> extends Mapper<
|
|||
protected void setup(Context context)
|
||||
throws IOException, InterruptedException
|
||||
{
|
||||
config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration());
|
||||
config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration());
|
||||
parser = config.getParser();
|
||||
}
|
||||
|
||||
|
|
|
@ -1,138 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexer;
|
||||
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.CharStreams;
|
||||
import com.google.common.io.InputSupplier;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class HadoopDruidIndexerNode
|
||||
{
|
||||
public static Builder builder()
|
||||
{
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
private String intervalSpec = null;
|
||||
private String argumentSpec = null;
|
||||
|
||||
public String getIntervalSpec()
|
||||
{
|
||||
return intervalSpec;
|
||||
}
|
||||
|
||||
public String getArgumentSpec()
|
||||
{
|
||||
return argumentSpec;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerNode setIntervalSpec(String intervalSpec)
|
||||
{
|
||||
this.intervalSpec = intervalSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HadoopDruidIndexerNode setArgumentSpec(String argumentSpec)
|
||||
{
|
||||
this.argumentSpec = argumentSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public HadoopDruidIndexerNode registerJacksonSubtype(Class<?>... clazzes)
|
||||
{
|
||||
HadoopDruidIndexerConfig.jsonMapper.registerSubtypes(clazzes);
|
||||
return this;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public HadoopDruidIndexerNode registerJacksonSubtype(NamedType... namedTypes)
|
||||
{
|
||||
HadoopDruidIndexerConfig.jsonMapper.registerSubtypes(namedTypes);
|
||||
return this;
|
||||
}
|
||||
|
||||
@LifecycleStart
|
||||
public void start() throws Exception
|
||||
{
|
||||
Preconditions.checkNotNull(argumentSpec, "argumentSpec");
|
||||
|
||||
final HadoopDruidIndexerConfig config;
|
||||
if (argumentSpec.startsWith("{")) {
|
||||
config = HadoopDruidIndexerConfig.fromString(argumentSpec);
|
||||
} else if (argumentSpec.startsWith("s3://")) {
|
||||
final Path s3nPath = new Path(String.format("s3n://%s", argumentSpec.substring("s3://".length())));
|
||||
final FileSystem fs = s3nPath.getFileSystem(new Configuration());
|
||||
|
||||
String configString = CharStreams.toString(new InputSupplier<InputStreamReader>()
|
||||
{
|
||||
@Override
|
||||
public InputStreamReader getInput() throws IOException
|
||||
{
|
||||
return new InputStreamReader(fs.open(s3nPath));
|
||||
}
|
||||
});
|
||||
|
||||
config = HadoopDruidIndexerConfig.fromString(configString);
|
||||
} else {
|
||||
config = HadoopDruidIndexerConfig.fromFile(new File(argumentSpec));
|
||||
}
|
||||
|
||||
if (intervalSpec != null) {
|
||||
final List<Interval> dataInterval = Lists.transform(
|
||||
Arrays.asList(intervalSpec.split(",")),
|
||||
new StringIntervalFunction()
|
||||
);
|
||||
|
||||
config.setIntervals(dataInterval);
|
||||
}
|
||||
|
||||
new HadoopDruidIndexerJob(config).run();
|
||||
}
|
||||
|
||||
@LifecycleStop
|
||||
public void stop()
|
||||
{
|
||||
}
|
||||
|
||||
public static class Builder
|
||||
{
|
||||
public HadoopDruidIndexerNode build()
|
||||
{
|
||||
return new HadoopDruidIndexerNode();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,194 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexer;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.druid.data.input.impl.DataSpec;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.indexer.granularity.GranularitySpec;
|
||||
import io.druid.indexer.partitions.PartitionsSpec;
|
||||
import io.druid.indexer.rollup.DataRollupSpec;
|
||||
import io.druid.indexer.updater.DbUpdaterJobSpec;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class HadoopDruidIndexerSchema
|
||||
{
|
||||
private final String dataSource;
|
||||
private final TimestampSpec timestampSpec;
|
||||
private final DataSpec dataSpec;
|
||||
private final GranularitySpec granularitySpec;
|
||||
private final Map<String, Object> pathSpec; // This cannot just be a PathSpec object
|
||||
private final String workingPath;
|
||||
private final String segmentOutputPath;
|
||||
private final String version;
|
||||
private final PartitionsSpec partitionsSpec;
|
||||
private final boolean leaveIntermediate;
|
||||
private final boolean cleanupOnFailure;
|
||||
private final Map<DateTime, List<HadoopyShardSpec>> shardSpecs;
|
||||
private final boolean overwriteFiles;
|
||||
private final DataRollupSpec rollupSpec;
|
||||
private final DbUpdaterJobSpec updaterJobSpec;
|
||||
private final boolean ignoreInvalidRows;
|
||||
|
||||
@JsonCreator
|
||||
public HadoopDruidIndexerSchema(
|
||||
final @JsonProperty("dataSource") String dataSource,
|
||||
final @JsonProperty("timestampSpec") TimestampSpec timestampSpec,
|
||||
final @JsonProperty("dataSpec") DataSpec dataSpec,
|
||||
final @JsonProperty("granularitySpec") GranularitySpec granularitySpec,
|
||||
final @JsonProperty("pathSpec") Map<String, Object> pathSpec,
|
||||
final @JsonProperty("workingPath") String workingPath,
|
||||
final @JsonProperty("segmentOutputPath") String segmentOutputPath,
|
||||
final @JsonProperty("version") String version,
|
||||
final @JsonProperty("partitionsSpec") PartitionsSpec partitionsSpec,
|
||||
final @JsonProperty("leaveIntermediate") boolean leaveIntermediate,
|
||||
final @JsonProperty("cleanupOnFailure") Boolean cleanupOnFailure,
|
||||
final @JsonProperty("shardSpecs") Map<DateTime, List<HadoopyShardSpec>> shardSpecs,
|
||||
final @JsonProperty("overwriteFiles") boolean overwriteFiles,
|
||||
final @JsonProperty("rollupSpec") DataRollupSpec rollupSpec,
|
||||
final @JsonProperty("updaterJobSpec") DbUpdaterJobSpec updaterJobSpec,
|
||||
final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows,
|
||||
// These fields are deprecated and will be removed in the future
|
||||
final @JsonProperty("timestampColumn") String timestampColumn,
|
||||
final @JsonProperty("timestampFormat") String timestampFormat
|
||||
|
||||
)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
this.timestampSpec = (timestampSpec == null) ? new TimestampSpec(timestampColumn, timestampFormat) : timestampSpec;
|
||||
this.dataSpec = dataSpec;
|
||||
this.granularitySpec = granularitySpec;
|
||||
this.pathSpec = pathSpec;
|
||||
this.workingPath = workingPath;
|
||||
this.segmentOutputPath = segmentOutputPath;
|
||||
this.version = version == null ? new DateTime().toString() : version;
|
||||
this.partitionsSpec = partitionsSpec;
|
||||
this.leaveIntermediate = leaveIntermediate;
|
||||
this.cleanupOnFailure = (cleanupOnFailure == null ? true : cleanupOnFailure);
|
||||
this.shardSpecs = (shardSpecs == null ? ImmutableMap.<DateTime, List<HadoopyShardSpec>>of() : shardSpecs);
|
||||
this.overwriteFiles = overwriteFiles;
|
||||
this.rollupSpec = rollupSpec;
|
||||
this.updaterJobSpec = updaterJobSpec;
|
||||
this.ignoreInvalidRows = ignoreInvalidRows;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public TimestampSpec getTimestampSpec()
|
||||
{
|
||||
return timestampSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public DataSpec getDataSpec()
|
||||
{
|
||||
return dataSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public GranularitySpec getGranularitySpec()
|
||||
{
|
||||
return granularitySpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<String, Object> getPathSpec()
|
||||
{
|
||||
return pathSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getWorkingPath()
|
||||
{
|
||||
return workingPath;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getSegmentOutputPath()
|
||||
{
|
||||
return segmentOutputPath;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getVersion()
|
||||
{
|
||||
return version;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public PartitionsSpec getPartitionsSpec()
|
||||
{
|
||||
return partitionsSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isLeaveIntermediate()
|
||||
{
|
||||
return leaveIntermediate;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isCleanupOnFailure()
|
||||
{
|
||||
return cleanupOnFailure;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Map<DateTime, List<HadoopyShardSpec>> getShardSpecs()
|
||||
{
|
||||
return shardSpecs;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isOverwriteFiles()
|
||||
{
|
||||
return overwriteFiles;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public DataRollupSpec getRollupSpec()
|
||||
{
|
||||
return rollupSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public DbUpdaterJobSpec getUpdaterJobSpec()
|
||||
{
|
||||
return updaterJobSpec;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public boolean isIgnoreInvalidRows()
|
||||
{
|
||||
return ignoreInvalidRows;
|
||||
}
|
||||
}
|
|
@ -86,6 +86,7 @@ public class IndexGeneratorJob implements Jobby
|
|||
private static final Logger log = new Logger(IndexGeneratorJob.class);
|
||||
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
|
||||
private IndexGeneratorStats jobStats;
|
||||
|
||||
public IndexGeneratorJob(
|
||||
|
@ -243,7 +244,7 @@ public class IndexGeneratorJob implements Jobby
|
|||
protected void setup(Context context)
|
||||
throws IOException, InterruptedException
|
||||
{
|
||||
config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration());
|
||||
config = HadoopDruidIndexerConfigBuilder.fromConfiguration(context.getConfiguration());
|
||||
|
||||
for (AggregatorFactory factory : config.getRollupSpec().getAggs()) {
|
||||
metricNames.add(factory.getName().toLowerCase());
|
||||
|
|
|
@ -60,7 +60,7 @@ public class JobHelper
|
|||
|
||||
final Configuration conf = groupByJob.getConfiguration();
|
||||
final FileSystem fs = FileSystem.get(conf);
|
||||
Path distributedClassPath = new Path(config.getJobOutputDir(), "classpath");
|
||||
Path distributedClassPath = new Path(config.getWorkingPath(), "classpath");
|
||||
|
||||
if (fs instanceof LocalFileSystem) {
|
||||
return;
|
||||
|
|
|
@ -63,12 +63,16 @@ public class GranularUnprocessedPathSpec extends GranularityPathSpec
|
|||
// This PathSpec breaks so many abstractions that we might as break some more
|
||||
Preconditions.checkState(
|
||||
config.getGranularitySpec() instanceof UniformGranularitySpec,
|
||||
String.format("Cannot use %s without %s", GranularUnprocessedPathSpec.class.getSimpleName(), UniformGranularitySpec.class.getSimpleName())
|
||||
String.format(
|
||||
"Cannot use %s without %s",
|
||||
GranularUnprocessedPathSpec.class.getSimpleName(),
|
||||
UniformGranularitySpec.class.getSimpleName()
|
||||
)
|
||||
);
|
||||
|
||||
final Path betaInput = new Path(getInputPath());
|
||||
final FileSystem fs = betaInput.getFileSystem(job.getConfiguration());
|
||||
final Granularity segmentGranularity = ((UniformGranularitySpec)config.getGranularitySpec()).getGranularity();
|
||||
final Granularity segmentGranularity = ((UniformGranularitySpec) config.getGranularitySpec()).getGranularity();
|
||||
|
||||
Map<DateTime, Long> inputModifiedTimes = new TreeMap<DateTime, Long>(
|
||||
Comparators.inverse(Comparators.<Comparable>comparable())
|
||||
|
@ -87,7 +91,11 @@ public class GranularUnprocessedPathSpec extends GranularityPathSpec
|
|||
DateTime timeBucket = entry.getKey();
|
||||
long mTime = entry.getValue();
|
||||
|
||||
String bucketOutput = String.format("%s/%s", config.getSegmentOutputDir(), segmentGranularity.toPath(timeBucket));
|
||||
String bucketOutput = String.format(
|
||||
"%s/%s",
|
||||
config.getSegmentOutputPath(),
|
||||
segmentGranularity.toPath(timeBucket)
|
||||
);
|
||||
for (FileStatus fileStatus : FSSpideringIterator.spiderIterable(fs, new Path(bucketOutput))) {
|
||||
if (fileStatus.getModificationTime() > mTime) {
|
||||
bucketsToRun.add(new Interval(timeBucket, segmentGranularity.increment(timeBucket)));
|
||||
|
|
|
@ -40,7 +40,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
|
||||
@Test
|
||||
public void testGranularitySpec() {
|
||||
public void testGranularitySpec()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
|
@ -54,7 +55,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -74,7 +76,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGranularitySpecLegacy() {
|
||||
public void testGranularitySpecLegacy()
|
||||
{
|
||||
// Deprecated and replaced by granularitySpec, but still supported
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
|
@ -86,7 +89,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -106,40 +110,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGranularitySpecPostConstructorIntervals() {
|
||||
// Deprecated and replaced by granularitySpec, but still supported
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
cfg = jsonMapper.readValue(
|
||||
"{"
|
||||
+ "\"segmentGranularity\":\"day\""
|
||||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
cfg.setIntervals(Lists.newArrayList(new Interval("2012-03-01/P1D")));
|
||||
|
||||
final UniformGranularitySpec granularitySpec = (UniformGranularitySpec) cfg.getGranularitySpec();
|
||||
|
||||
Assert.assertEquals(
|
||||
"getIntervals",
|
||||
Lists.newArrayList(new Interval("2012-03-01/P1D")),
|
||||
granularitySpec.getIntervals()
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"getGranularity",
|
||||
"DAY",
|
||||
granularitySpec.getGranularity().toString()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidGranularityCombination() {
|
||||
public void testInvalidGranularityCombination()
|
||||
{
|
||||
boolean thrown = false;
|
||||
try {
|
||||
final HadoopDruidIndexerConfig cfg = jsonReadWriteRead(
|
||||
|
@ -154,7 +126,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
thrown = true;
|
||||
}
|
||||
|
||||
|
@ -162,7 +135,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testPartitionsSpecAutoDimension() {
|
||||
public void testPartitionsSpecAutoDimension()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
|
@ -174,7 +148,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -200,7 +175,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testPartitionsSpecSpecificDimension() {
|
||||
public void testPartitionsSpecSpecificDimension()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
|
@ -213,7 +189,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -245,7 +222,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testPartitionsSpecLegacy() {
|
||||
public void testPartitionsSpecLegacy()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
|
@ -256,7 +234,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -288,7 +267,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testPartitionsSpecMaxPartitionSize() {
|
||||
public void testPartitionsSpecMaxPartitionSize()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
|
@ -302,7 +282,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -334,7 +315,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidPartitionsCombination() {
|
||||
public void testInvalidPartitionsCombination()
|
||||
{
|
||||
boolean thrown = false;
|
||||
try {
|
||||
final HadoopDruidIndexerConfig cfg = jsonReadWriteRead(
|
||||
|
@ -346,7 +328,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
thrown = true;
|
||||
}
|
||||
|
||||
|
@ -382,7 +365,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testDefaultSettings() {
|
||||
public void testDefaultSettings()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
|
@ -390,7 +374,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
"{}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -414,7 +399,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testNoCleanupOnFailure() {
|
||||
public void testNoCleanupOnFailure()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
|
@ -422,7 +408,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
"{\"cleanupOnFailure\":false}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -435,23 +422,25 @@ public class HadoopDruidIndexerConfigTest
|
|||
|
||||
|
||||
@Test
|
||||
public void shouldMakeHDFSCompliantSegmentOutputPath() {
|
||||
public void shouldMakeHDFSCompliantSegmentOutputPath()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
cfg = jsonReadWriteRead(
|
||||
"{"
|
||||
+ "\"dataSource\": \"source\","
|
||||
+ " \"granularitySpec\":{"
|
||||
+ " \"type\":\"uniform\","
|
||||
+ " \"gran\":\"hour\","
|
||||
+ " \"intervals\":[\"2012-07-10/P1D\"]"
|
||||
+ " },"
|
||||
+ "\"segmentOutputPath\": \"hdfs://server:9100/tmp/druid/datatest\""
|
||||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
"{"
|
||||
+ "\"dataSource\": \"source\","
|
||||
+ " \"granularitySpec\":{"
|
||||
+ " \"type\":\"uniform\","
|
||||
+ " \"gran\":\"hour\","
|
||||
+ " \"intervals\":[\"2012-07-10/P1D\"]"
|
||||
+ " },"
|
||||
+ "\"segmentOutputPath\": \"hdfs://server:9100/tmp/druid/datatest\""
|
||||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -466,23 +455,25 @@ public class HadoopDruidIndexerConfigTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void shouldMakeDefaultSegmentOutputPathIfNotHDFS() {
|
||||
public void shouldMakeDefaultSegmentOutputPathIfNotHDFS()
|
||||
{
|
||||
final HadoopDruidIndexerConfig cfg;
|
||||
|
||||
try {
|
||||
cfg = jsonReadWriteRead(
|
||||
"{"
|
||||
+ "\"dataSource\": \"the:data:source\","
|
||||
+ " \"granularitySpec\":{"
|
||||
+ " \"type\":\"uniform\","
|
||||
+ " \"gran\":\"hour\","
|
||||
+ " \"intervals\":[\"2012-07-10/P1D\"]"
|
||||
+ " },"
|
||||
+ "\"segmentOutputPath\": \"/tmp/dru:id/data:test\""
|
||||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
"{"
|
||||
+ "\"dataSource\": \"the:data:source\","
|
||||
+ " \"granularitySpec\":{"
|
||||
+ " \"type\":\"uniform\","
|
||||
+ " \"gran\":\"hour\","
|
||||
+ " \"intervals\":[\"2012-07-10/P1D\"]"
|
||||
+ " },"
|
||||
+ "\"segmentOutputPath\": \"/tmp/dru:id/data:test\""
|
||||
+ "}",
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
} catch(Exception e) {
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
|
@ -490,7 +481,10 @@ public class HadoopDruidIndexerConfigTest
|
|||
|
||||
Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712);
|
||||
Path path = cfg.makeSegmentOutputPath(new LocalFileSystem(), bucket);
|
||||
Assert.assertEquals("/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:version/4712", path.toString());
|
||||
Assert.assertEquals(
|
||||
"/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:version/4712",
|
||||
path.toString()
|
||||
);
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -20,38 +20,33 @@
|
|||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-indexing-service</artifactId>
|
||||
<name>druid-indexing-service</name>
|
||||
<description>druid-indexing-service</description>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-common</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-indexing-hadoop</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<artifactId>druid-realtime</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.metamx</groupId>
|
||||
|
|
|
@ -17,28 +17,33 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexing.worker.executor;
|
||||
package io.druid.guice;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import io.druid.indexing.overlord.TaskRunner;
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import io.druid.indexing.common.index.EventReceiverFirehoseFactory;
|
||||
import io.druid.initialization.DruidModule;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.List;
|
||||
|
||||
public class ExecutorLifecycleFactory
|
||||
public class IndexingServiceFirehoseModule implements DruidModule
|
||||
{
|
||||
private final File taskFile;
|
||||
private final File statusFile;
|
||||
|
||||
public ExecutorLifecycleFactory(File taskFile, File statusFile)
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
this.taskFile = taskFile;
|
||||
this.statusFile = statusFile;
|
||||
}
|
||||
|
||||
public ExecutorLifecycle build(TaskRunner taskRunner, ObjectMapper jsonMapper)
|
||||
{
|
||||
return new ExecutorLifecycle(
|
||||
new ExecutorLifecycleConfig().setTaskFile(taskFile).setStatusFile(statusFile), taskRunner, jsonMapper
|
||||
return ImmutableList.<Module>of(
|
||||
new SimpleModule("IndexingServiceFirehoseModule")
|
||||
.registerSubtypes(
|
||||
new NamedType(EventReceiverFirehoseFactory.class, "receiver")
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
}
|
||||
}
|
|
@ -84,7 +84,7 @@ public class RemoteTaskActionClient implements TaskActionClient
|
|||
|
||||
final String response;
|
||||
|
||||
log.info("Submitting action for task[%s] to coordinator[%s]: %s", task.getId(), serviceUri, taskAction);
|
||||
log.info("Submitting action for task[%s] to overlord[%s]: %s", task.getId(), serviceUri, taskAction);
|
||||
|
||||
try {
|
||||
response = httpClient.post(serviceUri.toURL())
|
||||
|
|
|
@ -37,7 +37,6 @@ import io.druid.query.QueryRunner;
|
|||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
public abstract class AbstractTask implements Task
|
||||
|
@ -189,13 +188,12 @@ public abstract class AbstractTask implements Task
|
|||
{
|
||||
final List<TaskLock> locks = toolbox.getTaskActionClient().submit(new LockListAction());
|
||||
|
||||
if (locks.isEmpty()) {
|
||||
return Arrays.asList(
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new LockAcquireAction(getImplicitLockInterval().get()))
|
||||
);
|
||||
if (locks.isEmpty() && getImplicitLockInterval().isPresent()) {
|
||||
// In the Peon's local mode, the implicit lock interval is not pre-acquired, so we need to try it here.
|
||||
toolbox.getTaskActionClient().submit(new LockAcquireAction(getImplicitLockInterval().get()));
|
||||
return toolbox.getTaskActionClient().submit(new LockListAction());
|
||||
} else {
|
||||
return locks;
|
||||
}
|
||||
|
||||
return locks;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,34 +22,57 @@ package io.druid.indexing.common.task;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfig;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfigBuilder;
|
||||
import io.druid.indexer.HadoopDruidIndexerJob;
|
||||
import io.druid.indexer.HadoopDruidIndexerSchema;
|
||||
import io.druid.indexing.common.TaskLock;
|
||||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.indexing.common.TaskToolbox;
|
||||
import io.druid.indexing.common.actions.SegmentInsertAction;
|
||||
import io.druid.initialization.Initialization;
|
||||
import io.druid.server.initialization.ExtensionsConfig;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.tesla.aether.internal.DefaultTeslaAether;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.File;
|
||||
import java.lang.reflect.Method;
|
||||
import java.net.URL;
|
||||
import java.net.URLClassLoader;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
public class HadoopIndexTask extends AbstractTask
|
||||
{
|
||||
@JsonIgnore
|
||||
private final HadoopDruidIndexerConfig config;
|
||||
|
||||
private static final Logger log = new Logger(HadoopIndexTask.class);
|
||||
private static String defaultHadoopCoordinates = "org.apache.hadoop:hadoop-core:1.0.3";
|
||||
|
||||
private static final ExtensionsConfig extensionsConfig;
|
||||
|
||||
static {
|
||||
extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class);
|
||||
}
|
||||
|
||||
@JsonIgnore
|
||||
private final HadoopDruidIndexerSchema schema;
|
||||
|
||||
@JsonIgnore
|
||||
private final String hadoopCoordinates;
|
||||
|
||||
/**
|
||||
* @param config is used by the HadoopDruidIndexerJob to set up the appropriate parameters
|
||||
* @param schema is used by the HadoopDruidIndexerJob to set up the appropriate parameters
|
||||
* for creating Druid index segments. It may be modified.
|
||||
* <p/>
|
||||
* Here, we will ensure that the DbConnectorConfig field of the config is set to null, such that the
|
||||
* Here, we will ensure that the DbConnectorConfig field of the schema is set to null, such that the
|
||||
* job does not push a list of published segments the database. Instead, we will use the method
|
||||
* IndexGeneratorJob.getPublishedSegments() to simply return a list of the published
|
||||
* segments, and let the indexing service report these segments to the database.
|
||||
|
@ -58,21 +81,24 @@ public class HadoopIndexTask extends AbstractTask
|
|||
@JsonCreator
|
||||
public HadoopIndexTask(
|
||||
@JsonProperty("id") String id,
|
||||
@JsonProperty("config") HadoopDruidIndexerConfig config
|
||||
@JsonProperty("config") HadoopDruidIndexerSchema schema,
|
||||
@JsonProperty("hadoopCoordinates") String hadoopCoordinates
|
||||
)
|
||||
{
|
||||
super(
|
||||
id != null ? id : String.format("index_hadoop_%s_%s", config.getDataSource(), new DateTime()),
|
||||
config.getDataSource(),
|
||||
JodaUtils.umbrellaInterval(config.getIntervals())
|
||||
id != null ? id : String.format("index_hadoop_%s_%s", schema.getDataSource(), new DateTime()),
|
||||
schema.getDataSource(),
|
||||
JodaUtils.umbrellaInterval(JodaUtils.condenseIntervals(schema.getGranularitySpec().bucketIntervals()))
|
||||
);
|
||||
|
||||
// Some HadoopDruidIndexerConfig stuff doesn't make sense in the context of the indexing service
|
||||
Preconditions.checkArgument(config.getSegmentOutputDir() == null, "segmentOutputPath must be absent");
|
||||
Preconditions.checkArgument(config.getJobOutputDir() == null, "workingPath must be absent");
|
||||
Preconditions.checkArgument(!config.isUpdaterJobSpecSet(), "updaterJobSpec must be absent");
|
||||
|
||||
this.config = config;
|
||||
// Some HadoopDruidIndexerSchema stuff doesn't make sense in the context of the indexing service
|
||||
Preconditions.checkArgument(schema.getSegmentOutputPath() == null, "segmentOutputPath must be absent");
|
||||
Preconditions.checkArgument(schema.getWorkingPath() == null, "workingPath must be absent");
|
||||
Preconditions.checkArgument(schema.getUpdaterJobSpec() == null, "updaterJobSpec must be absent");
|
||||
|
||||
this.schema = schema;
|
||||
this.hadoopCoordinates = (hadoopCoordinates == null ? defaultHadoopCoordinates : hadoopCoordinates);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -81,34 +107,64 @@ public class HadoopIndexTask extends AbstractTask
|
|||
return "index_hadoop";
|
||||
}
|
||||
|
||||
|
||||
@JsonProperty("config")
|
||||
public HadoopDruidIndexerSchema getSchema()
|
||||
{
|
||||
return schema;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getHadoopCoordinates()
|
||||
{
|
||||
return hadoopCoordinates;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
// Copy config so we don't needlessly modify our provided one
|
||||
// Also necessary to make constructor validations work upon serde-after-run
|
||||
final HadoopDruidIndexerConfig configCopy = toolbox.getObjectMapper()
|
||||
.readValue(
|
||||
toolbox.getObjectMapper().writeValueAsBytes(config),
|
||||
HadoopDruidIndexerConfig.class
|
||||
);
|
||||
// setup Hadoop
|
||||
final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig);
|
||||
final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates(
|
||||
aetherClient, hadoopCoordinates
|
||||
);
|
||||
final URL[] urLs = ((URLClassLoader) hadoopLoader).getURLs();
|
||||
|
||||
final URL[] nonHadoopUrls = ((URLClassLoader) HadoopIndexTask.class.getClassLoader()).getURLs();
|
||||
|
||||
List<URL> theURLS = Lists.newArrayList();
|
||||
theURLS.addAll(Arrays.asList(urLs));
|
||||
theURLS.addAll(Arrays.asList(nonHadoopUrls));
|
||||
|
||||
final URLClassLoader loader = new URLClassLoader(theURLS.toArray(new URL[theURLS.size()]), null);
|
||||
Thread.currentThread().setContextClassLoader(loader);
|
||||
|
||||
System.setProperty("druid.hadoop.internal.classpath", Joiner.on(File.pathSeparator).join(nonHadoopUrls));
|
||||
|
||||
final Class<?> mainClass = loader.loadClass(HadoopIndexTaskInnerProcessing.class.getName());
|
||||
final Method mainMethod = mainClass.getMethod("runTask", String[].class);
|
||||
|
||||
// We should have a lock from before we started running
|
||||
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
|
||||
final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox));
|
||||
log.info("Setting version to: %s", myLock.getVersion());
|
||||
configCopy.setVersion(myLock.getVersion());
|
||||
|
||||
// Set workingPath to some reasonable default
|
||||
configCopy.setJobOutputDir(toolbox.getConfig().getHadoopWorkingPath());
|
||||
String[] args = new String[]{
|
||||
toolbox.getObjectMapper().writeValueAsString(schema),
|
||||
myLock.getVersion(),
|
||||
toolbox.getConfig().getHadoopWorkingPath(),
|
||||
toolbox.getSegmentPusher().getPathForHadoop(getDataSource()),
|
||||
};
|
||||
|
||||
configCopy.setSegmentOutputDir(toolbox.getSegmentPusher().getPathForHadoop(getDataSource()));
|
||||
String segments = (String) mainMethod.invoke(null, new Object[]{args});
|
||||
|
||||
HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(configCopy);
|
||||
configCopy.verify();
|
||||
|
||||
log.info("Starting a hadoop index generator job...");
|
||||
if (job.run()) {
|
||||
List<DataSegment> publishedSegments = job.getPublishedSegments();
|
||||
|
||||
if (segments != null) {
|
||||
List<DataSegment> publishedSegments = toolbox.getObjectMapper().readValue(
|
||||
segments, new TypeReference<List<DataSegment>>()
|
||||
{
|
||||
}
|
||||
);
|
||||
// Request segment pushes
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments)));
|
||||
|
||||
|
@ -117,12 +173,41 @@ public class HadoopIndexTask extends AbstractTask
|
|||
} else {
|
||||
return TaskStatus.failure(getId());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public HadoopDruidIndexerConfig getConfig()
|
||||
public static class HadoopIndexTaskInnerProcessing
|
||||
{
|
||||
return config;
|
||||
public static String runTask(String[] args) throws Exception
|
||||
{
|
||||
final String schema = args[0];
|
||||
final String version = args[1];
|
||||
final String workingPath = args[2];
|
||||
final String segmentOutputPath = args[3];
|
||||
|
||||
final HadoopDruidIndexerSchema theSchema = HadoopDruidIndexerConfig.jsonMapper
|
||||
.readValue(
|
||||
schema,
|
||||
HadoopDruidIndexerSchema.class
|
||||
);
|
||||
final HadoopDruidIndexerConfig config =
|
||||
new HadoopDruidIndexerConfigBuilder().withSchema(theSchema)
|
||||
.withVersion(version)
|
||||
.withWorkingPath(
|
||||
workingPath
|
||||
)
|
||||
.withSegmentOutputPath(
|
||||
segmentOutputPath
|
||||
)
|
||||
.build();
|
||||
|
||||
HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(config);
|
||||
|
||||
log.info("Starting a hadoop index generator job...");
|
||||
if (job.run()) {
|
||||
return HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(job.getPublishedSegments());
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import io.druid.guice.annotations.Self;
|
|||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig;
|
||||
import io.druid.indexing.worker.config.WorkerConfig;
|
||||
import io.druid.server.DruidNode;
|
||||
import io.druid.tasklogs.TaskLogPusher;
|
||||
import io.druid.tasklogs.TaskLogStreamer;
|
||||
|
@ -84,6 +85,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
@Inject
|
||||
public ForkingTaskRunner(
|
||||
ForkingTaskRunnerConfig config,
|
||||
WorkerConfig workerConfig,
|
||||
Properties props,
|
||||
TaskLogPusher taskLogPusher,
|
||||
ObjectMapper jsonMapper,
|
||||
|
@ -96,7 +98,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
this.jsonMapper = jsonMapper;
|
||||
this.node = node;
|
||||
|
||||
this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(config.maxForks()));
|
||||
this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(workerConfig.getCapacity()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -204,7 +206,8 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
command.add(statusFile.toString());
|
||||
String nodeType = task.getNodeType();
|
||||
if (nodeType != null) {
|
||||
command.add(String.format("--nodeType %s", nodeType));
|
||||
command.add("--nodeType");
|
||||
command.add(nodeType);
|
||||
}
|
||||
|
||||
jsonMapper.writeValue(taskFile, task);
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.inject.Inject;
|
||||
import io.druid.guice.annotations.Self;
|
||||
import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig;
|
||||
import io.druid.indexing.worker.config.WorkerConfig;
|
||||
import io.druid.server.DruidNode;
|
||||
import io.druid.tasklogs.TaskLogPusher;
|
||||
|
||||
|
@ -33,6 +34,7 @@ import java.util.Properties;
|
|||
public class ForkingTaskRunnerFactory implements TaskRunnerFactory
|
||||
{
|
||||
private final ForkingTaskRunnerConfig config;
|
||||
private final WorkerConfig workerConfig;
|
||||
private final Properties props;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final TaskLogPusher persistentTaskLogs;
|
||||
|
@ -41,12 +43,14 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory
|
|||
@Inject
|
||||
public ForkingTaskRunnerFactory(
|
||||
final ForkingTaskRunnerConfig config,
|
||||
final WorkerConfig workerConfig,
|
||||
final Properties props,
|
||||
final ObjectMapper jsonMapper,
|
||||
final TaskLogPusher persistentTaskLogs,
|
||||
@Self DruidNode node
|
||||
) {
|
||||
this.config = config;
|
||||
this.workerConfig = workerConfig;
|
||||
this.props = props;
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.persistentTaskLogs = persistentTaskLogs;
|
||||
|
@ -56,6 +60,6 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory
|
|||
@Override
|
||||
public TaskRunner build()
|
||||
{
|
||||
return new ForkingTaskRunner(config, props, persistentTaskLogs, jsonMapper, node);
|
||||
return new ForkingTaskRunner(config, workerConfig, props, persistentTaskLogs, jsonMapper, node);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -748,7 +748,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer
|
|||
);
|
||||
sortedWorkers.addAll(zkWorkers.values());
|
||||
final String configMinWorkerVer = workerSetupData.get().getMinVersion();
|
||||
final String minWorkerVer = configMinWorkerVer == null ? config.getWorkerVersion() : configMinWorkerVer;
|
||||
final String minWorkerVer = configMinWorkerVer == null ? config.getMinWorkerVersion() : configMinWorkerVer;
|
||||
|
||||
for (ZkWorker zkWorker : sortedWorkers) {
|
||||
if (zkWorker.canRunTask(task) && zkWorker.isValidVersion(minWorkerVer)) {
|
||||
|
|
|
@ -1,37 +0,0 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.indexing.overlord.config;
|
||||
|
||||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
import org.skife.config.DefaultNull;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class EC2AutoScalingStrategyConfig
|
||||
{
|
||||
@Config("druid.indexer.worker.port")
|
||||
@Default("8080")
|
||||
public abstract String getWorkerPort();
|
||||
|
||||
@Config("druid.indexer.worker.version")
|
||||
@DefaultNull
|
||||
public abstract String getWorkerVersion();
|
||||
}
|
|
@ -29,10 +29,6 @@ import java.util.List;
|
|||
|
||||
public class ForkingTaskRunnerConfig
|
||||
{
|
||||
@JsonProperty
|
||||
@Min(1)
|
||||
private int maxForks = 1;
|
||||
|
||||
@JsonProperty
|
||||
@NotNull
|
||||
private String taskDir = "/tmp/persistent";
|
||||
|
@ -69,11 +65,6 @@ public class ForkingTaskRunnerConfig
|
|||
"file.encoding"
|
||||
);
|
||||
|
||||
public int maxForks()
|
||||
{
|
||||
return maxForks;
|
||||
}
|
||||
|
||||
public String getTaskDir()
|
||||
{
|
||||
return taskDir;
|
||||
|
|
|
@ -37,7 +37,7 @@ public class RemoteTaskRunnerConfig
|
|||
private boolean compressZnodes = false;
|
||||
|
||||
@JsonProperty
|
||||
private String workerVersion = null;
|
||||
private String minWorkerVersion = null;
|
||||
|
||||
@JsonProperty
|
||||
@Min(10 * 1024)
|
||||
|
@ -53,9 +53,9 @@ public class RemoteTaskRunnerConfig
|
|||
return compressZnodes;
|
||||
}
|
||||
|
||||
public String getWorkerVersion()
|
||||
public String getMinWorkerVersion()
|
||||
{
|
||||
return workerVersion;
|
||||
return minWorkerVersion;
|
||||
}
|
||||
|
||||
public long getMaxZnodeBytes()
|
||||
|
|
|
@ -30,11 +30,11 @@ public class WorkerConfig
|
|||
{
|
||||
@JsonProperty
|
||||
@NotNull
|
||||
private String ip = null;
|
||||
private String ip = "localhost";
|
||||
|
||||
@JsonProperty
|
||||
@NotNull
|
||||
private String version = null;
|
||||
private String version = "0";
|
||||
|
||||
@JsonProperty
|
||||
@Min(1)
|
||||
|
|
|
@ -22,12 +22,13 @@ package io.druid.indexing.common.task;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.metamx.common.Granularity;
|
||||
import io.druid.data.input.impl.JSONDataSpec;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfig;
|
||||
import io.druid.indexer.HadoopDruidIndexerSchema;
|
||||
import io.druid.indexer.granularity.UniformGranularitySpec;
|
||||
import io.druid.indexer.path.StaticPathSpec;
|
||||
import io.druid.indexer.rollup.DataRollupSpec;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
|
@ -338,17 +339,12 @@ public class TaskSerdeTest
|
|||
{
|
||||
final HadoopIndexTask task = new HadoopIndexTask(
|
||||
null,
|
||||
new HadoopDruidIndexerConfig(
|
||||
null,
|
||||
new HadoopDruidIndexerSchema(
|
||||
"foo",
|
||||
"timestamp",
|
||||
"auto",
|
||||
new TimestampSpec("timestamp", "auto"),
|
||||
new JSONDataSpec(ImmutableList.of("foo"), null),
|
||||
null,
|
||||
new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P1D"))),
|
||||
new StaticPathSpec("bar"),
|
||||
null,
|
||||
null,
|
||||
ImmutableMap.<String, Object>of("paths", "bar"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
|
@ -359,8 +355,11 @@ public class TaskSerdeTest
|
|||
false,
|
||||
new DataRollupSpec(ImmutableList.<AggregatorFactory>of(), QueryGranularity.NONE),
|
||||
null,
|
||||
false
|
||||
)
|
||||
false,
|
||||
null,
|
||||
null
|
||||
),
|
||||
null
|
||||
);
|
||||
|
||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||
|
|
|
@ -45,7 +45,7 @@ public class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig
|
|||
}
|
||||
|
||||
@Override
|
||||
public String getWorkerVersion()
|
||||
public String getMinWorkerVersion()
|
||||
{
|
||||
return "";
|
||||
}
|
||||
|
|
8
pom.xml
8
pom.xml
|
@ -20,10 +20,10 @@
|
|||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<packaging>pom</packaging>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
<name>druid</name>
|
||||
<description>druid</description>
|
||||
<scm>
|
||||
|
@ -60,7 +60,7 @@
|
|||
<dependency>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-api</artifactId>
|
||||
<version>0.1.0-SNAPSHOT</version>
|
||||
<version>0.1.3</version>
|
||||
</dependency>
|
||||
|
||||
<!-- Compile Scope -->
|
||||
|
@ -177,7 +177,7 @@
|
|||
<dependency>
|
||||
<groupId>it.uniroma3.mat</groupId>
|
||||
<artifactId>extendedset</artifactId>
|
||||
<version>1.3.2</version>
|
||||
<version>1.3.4</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
|
|
|
@ -20,20 +20,20 @@
|
|||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<name>druid-processing</name>
|
||||
<description>A module that is everything required to understands Druid Segments</description>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-common</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
|
|
|
@ -133,6 +133,11 @@ public class SpatialDimensionRowFormatter
|
|||
return (retVal == null) ? row.getDimension(dimension) : retVal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getRaw(String dimension) {
|
||||
return row.getRaw(dimension);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloatMetric(String metric)
|
||||
{
|
||||
|
|
|
@ -20,15 +20,15 @@
|
|||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid.extensions</groupId>
|
||||
<artifactId>druid-s3-extensions</artifactId>
|
||||
<name>druid-s3-extensions</name>
|
||||
<description>druid-s3-extensions</description>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -93,7 +93,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller
|
|||
S3Object s3Obj = null;
|
||||
|
||||
try {
|
||||
s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path);
|
||||
s3Obj = s3Client.getObject(s3Coords.bucket, s3Coords.path);
|
||||
|
||||
InputStream in = null;
|
||||
try {
|
||||
|
|
|
@ -1 +1 @@
|
|||
io.druid.storage.cassandra.S3StorageDruidModule
|
||||
io.druid.storage.s3.S3StorageDruidModule
|
|
@ -18,23 +18,22 @@
|
|||
~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<name>druid-server</name>
|
||||
<description>Druid Server</description>
|
||||
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
|
@ -221,7 +220,7 @@
|
|||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<type>test-jar</type>
|
||||
|
|
|
@ -113,14 +113,14 @@ public class Initialization
|
|||
}
|
||||
|
||||
for (String coordinate : config.getCoordinates()) {
|
||||
log.info("Loading extension[%s]", coordinate);
|
||||
log.info("Loading extension[%s] for class[%s]", coordinate, clazz.getName());
|
||||
try {
|
||||
URLClassLoader loader = getClassLoaderForCoordinates(aether, coordinate);
|
||||
|
||||
final ServiceLoader<T> serviceLoader = ServiceLoader.load(clazz, loader);
|
||||
|
||||
for (T module : serviceLoader) {
|
||||
log.info("Adding extension module[%s]", module.getClass());
|
||||
log.info("Adding extension module[%s] for class[%s]", module.getClass(), clazz.getName());
|
||||
retVal.add(module);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -209,6 +209,12 @@ class WikipediaIrcDecoder implements IrcDecoder
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getRaw(String dimension) {
|
||||
return dimensions.get(dimension);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public float getFloatMetric(String metric)
|
||||
{
|
||||
|
|
|
@ -29,6 +29,7 @@ public class BalancerSegmentHolder
|
|||
private final DruidServer fromServer;
|
||||
private final DataSegment segment;
|
||||
|
||||
// This is a pretty fugly hard coding of the maximum lifetime
|
||||
private volatile int lifetime = 15;
|
||||
|
||||
public BalancerSegmentHolder(
|
||||
|
|
|
@ -493,7 +493,7 @@ public class DruidCoordinator
|
|||
)
|
||||
)
|
||||
),
|
||||
config.getCoordinatorSegmentMergerPeriod()
|
||||
config.getCoordinatorIndexingPeriod()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -38,9 +38,9 @@ public abstract class DruidCoordinatorConfig
|
|||
@Default("PT60s")
|
||||
public abstract Duration getCoordinatorPeriod();
|
||||
|
||||
@Config("druid.coordinator.period.segmentMerger")
|
||||
@Config("druid.coordinator.period.indexingPeriod")
|
||||
@Default("PT1800s")
|
||||
public abstract Duration getCoordinatorSegmentMergerPeriod();
|
||||
public abstract Duration getCoordinatorIndexingPeriod();
|
||||
|
||||
@Config("druid.coordinator.merge.on")
|
||||
public boolean isMergeSegments()
|
||||
|
|
|
@ -44,7 +44,6 @@ public class EmittingRequestLoggerProvider implements RequestLoggerProvider
|
|||
@Inject
|
||||
public void injectMe(Injector injector)
|
||||
{
|
||||
System.out.println("YAYAYAYAYAYA!!!");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -150,6 +150,12 @@ public class RealtimeManagerTest
|
|||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getRaw(String dimension)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -80,6 +80,12 @@ public class SinkTest
|
|||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getRaw(String dimension)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
||||
FireHydrant currHydrant = sink.getCurrIndex();
|
||||
|
@ -113,6 +119,12 @@ public class SinkTest
|
|||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getRaw(String dimension)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
||||
Assert.assertEquals(currHydrant, swapHydrant);
|
||||
|
|
|
@ -89,7 +89,7 @@ public class DruidCoordinatorTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public Duration getCoordinatorSegmentMergerPeriod()
|
||||
public Duration getCoordinatorIndexingPeriod()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -19,51 +19,45 @@
|
|||
-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-services</artifactId>
|
||||
<name>druid-services</name>
|
||||
<description>druid-services</description>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.6.0-SNAPSHOT</version>
|
||||
<version>0.6.1-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-examples</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-indexing-hadoop</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-indexing-service</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<artifactId>druid-realtime</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-examples</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.metamx.druid</groupId>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-indexing-service</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
|
|
|
@ -42,6 +42,13 @@
|
|||
</includes>
|
||||
<outputDirectory>config/historical</outputDirectory>
|
||||
</fileSet>
|
||||
<fileSet>
|
||||
<directory>../examples/config/overlord</directory>
|
||||
<includes>
|
||||
<include>*</include>
|
||||
</includes>
|
||||
<outputDirectory>config/overlord</outputDirectory>
|
||||
</fileSet>
|
||||
<fileSet>
|
||||
<directory>../examples/bin</directory>
|
||||
<includes>
|
||||
|
|
|
@ -41,7 +41,7 @@ import java.util.List;
|
|||
*/
|
||||
@Command(
|
||||
name = "hadoop",
|
||||
description = "Runs the batch Hadoop Druid Indexer, see https://github.com/metamx/druid/wiki/Batch-ingestion for a description."
|
||||
description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.0/Batch-ingestion.html for a description."
|
||||
)
|
||||
public class CliHadoopIndexer implements Runnable
|
||||
{
|
||||
|
|
|
@ -24,8 +24,8 @@ import com.metamx.common.logger.Logger;
|
|||
import io.airlift.command.Arguments;
|
||||
import io.airlift.command.Command;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfig;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfigBuilder;
|
||||
import io.druid.indexer.HadoopDruidIndexerJob;
|
||||
import io.druid.timeline.partition.SingleDimensionShardSpec;
|
||||
|
||||
import java.io.File;
|
||||
|
||||
|
@ -46,12 +46,6 @@ public class CliInternalHadoopIndexer implements Runnable
|
|||
public void run()
|
||||
{
|
||||
try {
|
||||
System.out.println(
|
||||
HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(
|
||||
new SingleDimensionShardSpec("billy", "a", "b", 1)
|
||||
)
|
||||
);
|
||||
|
||||
final HadoopDruidIndexerJob job = new HadoopDruidIndexerJob(getHadoopDruidIndexerConfig());
|
||||
job.run();
|
||||
}
|
||||
|
@ -64,9 +58,9 @@ public class CliInternalHadoopIndexer implements Runnable
|
|||
{
|
||||
try {
|
||||
if (argumentSpec.startsWith("{")) {
|
||||
return HadoopDruidIndexerConfig.fromString(argumentSpec);
|
||||
return HadoopDruidIndexerConfigBuilder.fromString(argumentSpec);
|
||||
} else {
|
||||
return HadoopDruidIndexerConfig.fromFile(new File(argumentSpec));
|
||||
return HadoopDruidIndexerConfigBuilder.fromFile(new File(argumentSpec));
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.inject.Provides;
|
|||
import com.google.inject.util.Providers;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.airlift.command.Command;
|
||||
import io.druid.guice.IndexingServiceFirehoseModule;
|
||||
import io.druid.guice.IndexingServiceModuleHelper;
|
||||
import io.druid.guice.Jerseys;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
|
@ -101,7 +102,8 @@ public class CliMiddleManager extends ServerRunnable
|
|||
config.getVersion()
|
||||
);
|
||||
}
|
||||
}
|
||||
},
|
||||
new IndexingServiceFirehoseModule()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import com.google.inject.servlet.GuiceFilter;
|
|||
import com.google.inject.util.Providers;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.airlift.command.Command;
|
||||
import io.druid.guice.IndexingServiceFirehoseModule;
|
||||
import io.druid.guice.IndexingServiceModuleHelper;
|
||||
import io.druid.guice.JacksonConfigProvider;
|
||||
import io.druid.guice.Jerseys;
|
||||
|
@ -206,7 +207,8 @@ public class CliOverlord extends ServerRunnable
|
|||
|
||||
JsonConfigProvider.bind(binder, "druid.indexer.autoscale", SimpleResourceManagementConfig.class);
|
||||
}
|
||||
}
|
||||
},
|
||||
new IndexingServiceFirehoseModule()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -19,13 +19,12 @@
|
|||
|
||||
package io.druid.cli;
|
||||
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.multibindings.MapBinder;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
@ -33,6 +32,7 @@ import io.airlift.command.Arguments;
|
|||
import io.airlift.command.Command;
|
||||
import io.airlift.command.Option;
|
||||
import io.druid.guice.Binders;
|
||||
import io.druid.guice.IndexingServiceFirehoseModule;
|
||||
import io.druid.guice.Jerseys;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.LazySingleton;
|
||||
|
@ -49,7 +49,6 @@ import io.druid.indexing.common.actions.TaskActionClientFactory;
|
|||
import io.druid.indexing.common.actions.TaskActionToolbox;
|
||||
import io.druid.indexing.common.config.TaskConfig;
|
||||
import io.druid.indexing.common.index.ChatHandlerProvider;
|
||||
import io.druid.indexing.common.index.EventReceiverFirehoseFactory;
|
||||
import io.druid.indexing.common.index.NoopChatHandlerProvider;
|
||||
import io.druid.indexing.common.index.ServiceAnnouncingChatHandlerProvider;
|
||||
import io.druid.indexing.overlord.HeapMemoryTaskStorage;
|
||||
|
@ -100,7 +99,7 @@ public class CliPeon extends GuiceRunnable
|
|||
protected List<Object> getModules()
|
||||
{
|
||||
return ImmutableList.<Object>of(
|
||||
new DruidModule()
|
||||
new Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
|
@ -122,7 +121,7 @@ public class CliPeon extends GuiceRunnable
|
|||
binder.bind(TaskToolboxFactory.class).in(LazySingleton.class);
|
||||
|
||||
JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.worker.taskActionClient.retry", RetryPolicyConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.peon.taskActionClient.retry", RetryPolicyConfig.class);
|
||||
|
||||
configureTaskActionClient(binder);
|
||||
|
||||
|
@ -179,16 +178,8 @@ public class CliPeon extends GuiceRunnable
|
|||
.to(RemoteTaskActionClientFactory.class).in(LazySingleton.class);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<? extends com.fasterxml.jackson.databind.Module> getJacksonModules()
|
||||
{
|
||||
return Arrays.asList(
|
||||
new SimpleModule("PeonModule")
|
||||
.registerSubtypes(new NamedType(EventReceiverFirehoseFactory.class, "receiver"))
|
||||
);
|
||||
}
|
||||
}
|
||||
},
|
||||
new IndexingServiceFirehoseModule()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,6 +24,7 @@ import io.airlift.command.Cli;
|
|||
import io.airlift.command.Help;
|
||||
import io.airlift.command.ParseException;
|
||||
import io.druid.cli.convert.ConvertProperties;
|
||||
import io.druid.cli.validate.DruidJsonValidator;
|
||||
import io.druid.initialization.Initialization;
|
||||
import io.druid.server.initialization.ExtensionsConfig;
|
||||
|
||||
|
@ -58,7 +59,7 @@ public class Main
|
|||
builder.withGroup("tools")
|
||||
.withDescription("Various tools for working with Druid")
|
||||
.withDefaultCommand(Help.class)
|
||||
.withCommands(ConvertProperties.class);
|
||||
.withCommands(ConvertProperties.class, DruidJsonValidator.class);
|
||||
|
||||
builder.withGroup("index")
|
||||
.withDescription("Run indexing for druid")
|
||||
|
|
|
@ -61,7 +61,7 @@ public class ConvertProperties implements Runnable
|
|||
new Rename("com.metamx.emitter.logging", "druid.emitter.logging"),
|
||||
new Rename("com.metamx.emitter.logging.level", "druid.emitter.logging.logLevel"),
|
||||
new Rename("com.metamx.emitter.http", "druid.emitter.http"),
|
||||
new Rename("com.metamx.emitter.http.url", "druid.emitter.http.url"),
|
||||
new Rename("com.metamx.emitter.http.url", "druid.emitter.http.recipientBaseUrl"),
|
||||
new Rename("com.metamx.druid.emitter.period", "druid.emitter.emissionPeriod"),
|
||||
new PrefixRename("com.metamx.emitter", "druid.emitter"),
|
||||
new PrefixRename("com.metamx.druid.emitter", "druid.emitter"),
|
||||
|
@ -85,7 +85,7 @@ public class ConvertProperties implements Runnable
|
|||
new Rename("druid.indexer.fork.startport", "druid.indexer.runner.startPort"),
|
||||
new Rename("druid.indexer.properties.prefixes", "druid.indexer.runner.allowedPrefixes"),
|
||||
new Rename("druid.indexer.taskAssignmentTimeoutDuration", "druid.indexer.runner.taskAssignmentTimeout"),
|
||||
new Rename("druid.indexer.worker.version", "druid.indexer.runner.workerVersion"),
|
||||
new Rename("druid.indexer.worker.version", "druid.indexer.runner.minWorkerVersion"),
|
||||
new Rename("druid.zk.maxNumBytes", "druid.indexer.runner.maxZnodeBytes"),
|
||||
new Rename("druid.indexer.provisionResources.duration", "druid.indexer.autoscale.provisionPeriod"),
|
||||
new Rename("druid.indexer.terminateResources.duration", "druid.indexer.autoscale.terminatePeriod"),
|
||||
|
@ -108,6 +108,7 @@ public class ConvertProperties implements Runnable
|
|||
new Rename("druid.worker.taskActionClient.retry.minWaitMillis", "druid.worker.taskActionClient.retry.minWait"),
|
||||
new Rename("druid.worker.taskActionClient.retry.maxWaitMillis", "druid.worker.taskActionClient.retry.maxWait"),
|
||||
new Rename("druid.master.merger.service", "druid.selectors.indexing.serviceName"),
|
||||
new Rename("druid.master.period.segmentMerger", "druid.coordinator.period.indexingPeriod"),
|
||||
new Rename("druid.master.merger.on", "druid.coordinator.merge.on"),
|
||||
new PrefixRename("druid.master", "druid.coordinator"),
|
||||
new PrefixRename("druid.pusher", "druid.storage"),
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue