{
35 |
36 | /**
37 | * Thrift-based import requires us to provide {@link org.apache.cassandra.thrift.Mutation}.
38 | * Therefore we convert each input line into one.
39 | *
40 | * @param inputRow byte representation of the input row as it was read from Avro file
41 | * @return wraps the record into something that blends nicely into Crunch
42 | */
43 | @Override
44 | public ThriftRecord map(ByteBuffer inputRow) {
45 | LegacyInputFormat row = LegacyInputFormat.parse(inputRow);
46 | ByteBuffer key = CassandraRecordUtils.toByteBuffer(row.getRowkey());
47 | long ts = Objects.firstNonNull(row.getTimestamp(), DateTimeUtils.currentTimeMillis());
48 | int ttl = Objects.firstNonNull(row.getTtl(), 0l).intValue();
49 | Mutation mutation = CassandraRecordUtils.createMutation(
50 | row.getColname(), row.getColval(), ts, ttl);
51 | return ThriftRecord.of(key, mutation);
52 | }
53 |
54 | }
55 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/LegacyInputFormat.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass;
17 |
18 | import org.apache.cassandra.utils.ByteBufferUtil;
19 | import org.apache.crunch.CrunchRuntimeException;
20 | import org.joda.time.DateTimeUtils;
21 |
22 | import java.nio.ByteBuffer;
23 | import java.nio.charset.CharacterCodingException;
24 |
25 | /**
26 | * Represents tab-separated input line.
27 | *
28 | *
29 | * This used to be the only supported format of hdfs2cass. Now it's deprecated and should not be
30 | * used. The format of a line is:
31 | *
32 | * Hdfs2Cassandra\t\t\t\t[]\t[]\t
33 | *
34 | * - timestamp and ttl are optional
35 | * - version 1 means timestamp and ttl is not present
36 | * - version 2 means ttl is not present
37 | * - version 3 means all fields are present
38 | *
39 | */
40 | public class LegacyInputFormat {
41 |
42 | private final String rowkey;
43 | private final String colname;
44 | private final String colvalue;
45 | private final long timestamp;
46 | private final long ttl;
47 |
48 | public LegacyInputFormat(String rowkey, String colname, String colvalue, long timestamp,
49 | long ttl) {
50 | this.rowkey = rowkey;
51 | this.colname = colname;
52 | this.colvalue = colvalue;
53 | this.timestamp = timestamp;
54 | this.ttl = ttl;
55 | }
56 |
57 | public static LegacyInputFormat parse (ByteBuffer row) {
58 | try {
59 | return parse(ByteBufferUtil.string(row));
60 | } catch (CharacterCodingException e) {
61 | throw new CrunchRuntimeException(e);
62 | }
63 | }
64 |
65 | public static LegacyInputFormat parse(String row) {
66 | String[] parts = row.split("\t");
67 | String rowkey = parts[2];
68 | String colname = parts[3];
69 | String value;
70 | long ts = DateTimeUtils.currentTimeMillis();
71 | long ttl = 0;
72 | if (!parts[0].equals("HdfsToCassandra")) {
73 | throw new CrunchRuntimeException("Found malformed row. The rows must start with 'HdfsToCassandra'");
74 | }
75 | switch (Integer.valueOf(parts[1])) {
76 | case 1:
77 | if (parts.length != 5) {
78 | throw new CrunchRuntimeException("Found malformed row. Check correct row format.");
79 | }
80 | value = parts[4];
81 | break;
82 | case 2:
83 | if (parts.length != 6) {
84 | throw new CrunchRuntimeException("Found malformed row. Check correct row format.");
85 | }
86 | ts = parseNumber(parts[4]);
87 | value = parts[5];
88 | break;
89 | case 3:
90 | if (parts.length != 7) {
91 | throw new CrunchRuntimeException("Found malformed row. Check correct row format.");
92 | }
93 | ts = parseNumber(parts[4]);
94 | ttl = parseNumber(parts[5]);
95 | value = parts[6];
96 | break;
97 | default:
98 | throw new CrunchRuntimeException("Unknown format version");
99 | }
100 | return new LegacyInputFormat(rowkey, colname, value, ts, ttl);
101 | }
102 |
103 | public String getRowkey() {
104 | return rowkey;
105 | }
106 |
107 | public String getColname() {
108 | return colname;
109 | }
110 |
111 | public String getColval() {
112 | return colvalue;
113 | }
114 |
115 | public long getTimestamp() {
116 | return timestamp;
117 | }
118 |
119 | public long getTtl() {
120 | return ttl;
121 | }
122 |
123 | private static long parseNumber(String str) throws CrunchRuntimeException {
124 | try {
125 | return Integer.valueOf(str);
126 | } catch (NumberFormatException e) {
127 | throw new CrunchRuntimeException(e);
128 | }
129 | }
130 |
131 | }
132 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/cql/CrunchCqlBulkOutputFormat.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | *
18 | * The modifications to the upstream file is Copyright 2014 Spotify AB.
19 | * The original upstream file can be found at
20 | * https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkOutputFormat.java
21 | */
22 | package com.spotify.hdfs2cass.cassandra.cql;
23 |
24 | import com.spotify.hdfs2cass.crunch.cql.CQLRecord;
25 | import org.apache.cassandra.hadoop.AbstractBulkOutputFormat;
26 | import org.apache.crunch.CrunchRuntimeException;
27 | import org.apache.hadoop.conf.Configuration;
28 | import org.apache.hadoop.fs.FileSystem;
29 | import org.apache.hadoop.mapred.JobConf;
30 | import org.apache.hadoop.mapreduce.TaskAttemptContext;
31 | import org.apache.hadoop.util.Progressable;
32 |
33 | import java.io.IOException;
34 | import java.nio.ByteBuffer;
35 |
36 | /**
37 | * This is an almost-copy of {@link org.apache.cassandra.hadoop.cql3.CqlBulkOutputFormat}
38 | *
39 | * We return {@link com.spotify.hdfs2cass.cassandra.cql.CrunchCqlBulkRecordWriter}
40 | * with our improvements and resolving conflicts with Crunch. This issue is tracked in
41 | * https://issues.apache.org/jira/browse/CASSANDRA-8367
42 | *
43 | */
44 | public class CrunchCqlBulkOutputFormat extends AbstractBulkOutputFormat {
45 |
46 | private static final String OUTPUT_CQL_SCHEMA_PREFIX = "cassandra.columnfamily.schema.";
47 | private static final String OUTPUT_CQL_INSERT_PREFIX = "cassandra.columnfamily.insert.";
48 | private static final String OUTPUT_CQL_SCHEMA_COLUMNS = "cassandra.columnfamily.columns.";
49 |
50 | /**
51 | * Not used anyway, so do not bother implementing.
52 | */
53 | @Deprecated
54 | @Override
55 | public CrunchCqlBulkRecordWriter getRecordWriter(FileSystem filesystem, JobConf job, String name, Progressable progress) throws IOException {
56 | throw new CrunchRuntimeException("Use getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext)");
57 | }
58 |
59 | @Override
60 | public CrunchCqlBulkRecordWriter getRecordWriter(final TaskAttemptContext context) throws IOException, InterruptedException {
61 | return new CrunchCqlBulkRecordWriter(context);
62 | }
63 |
64 | public static void setColumnFamilySchema(Configuration conf, String columnFamily, String schema) {
65 | conf.set(OUTPUT_CQL_SCHEMA_PREFIX + columnFamily, schema);
66 | }
67 |
68 | public static void setColumnFamilyInsertStatement(Configuration conf, String columnFamily, String insertStatement) {
69 | conf.set(OUTPUT_CQL_INSERT_PREFIX + columnFamily, insertStatement);
70 | }
71 |
72 | public static String getColumnFamilySchema(Configuration conf, String columnFamily) {
73 | String schema = conf.get(OUTPUT_CQL_SCHEMA_PREFIX + columnFamily);
74 | if (schema == null) {
75 | throw new UnsupportedOperationException("You must set the ColumnFamily schema using setColumnFamilySchema.");
76 | }
77 | return schema;
78 | }
79 |
80 | public static String getColumnFamilyInsertStatement(Configuration conf, String columnFamily) {
81 | String insert = conf.get(OUTPUT_CQL_INSERT_PREFIX + columnFamily);
82 | if (insert == null) {
83 | throw new UnsupportedOperationException("You must set the ColumnFamily insert statement using setColumnFamilySchema.");
84 | }
85 | return insert;
86 | }
87 |
88 | public static void setColumnIndex(Configuration conf, String columnFamily, String column,
89 | int index) {
90 | conf.set(String.format("%s%s.%s", OUTPUT_CQL_SCHEMA_COLUMNS, columnFamily, column), String.valueOf(index));
91 | }
92 |
93 | public static int getColumnIndex(Configuration conf, String columnFamily, String column) {
94 | String columnNames = conf.get(String.format("%s%s.%s", OUTPUT_CQL_SCHEMA_COLUMNS, columnFamily, column));
95 | if (columnNames == null) {
96 | throw new UnsupportedOperationException(String.format("Column name '%s' for table '%s' not found in configuration", column, columnFamily));
97 | }
98 | return Integer.valueOf(columnNames);
99 | }
100 |
101 | }
102 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/cql/CrunchCqlBulkRecordWriter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | *
18 | * The modifications to the upstream file is Copyright 2014 Spotify AB.
19 | * The original upstream file can be found at
20 | * https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
21 | */
22 | package com.spotify.hdfs2cass.cassandra.cql;
23 |
24 | import com.google.common.collect.Lists;
25 | import com.google.common.util.concurrent.Uninterruptibles;
26 | import com.spotify.hdfs2cass.cassandra.thrift.ProgressHeartbeat;
27 | import com.spotify.hdfs2cass.cassandra.thrift.ProgressIndicator;
28 | import com.spotify.hdfs2cass.crunch.CrunchConfigHelper;
29 | import com.spotify.hdfs2cass.crunch.cql.CQLRecord;
30 | import org.apache.cassandra.exceptions.InvalidRequestException;
31 | import org.apache.cassandra.hadoop.AbstractBulkRecordWriter;
32 | import org.apache.cassandra.hadoop.BulkRecordWriter;
33 | import org.apache.cassandra.hadoop.ConfigHelper;
34 | import org.apache.cassandra.hadoop.HadoopCompat;
35 | import org.apache.cassandra.io.sstable.CQLSSTableWriter;
36 | import org.apache.cassandra.io.sstable.SSTableLoader;
37 | import org.apache.cassandra.streaming.StreamState;
38 | import org.apache.cassandra.utils.ByteBufferUtil;
39 | import org.apache.crunch.CrunchRuntimeException;
40 | import org.apache.hadoop.mapreduce.TaskAttemptContext;
41 | import org.slf4j.Logger;
42 | import org.slf4j.LoggerFactory;
43 |
44 | import java.io.File;
45 | import java.io.IOException;
46 | import java.net.InetAddress;
47 | import java.nio.ByteBuffer;
48 | import java.util.Collections;
49 | import java.util.List;
50 | import java.util.concurrent.ExecutionException;
51 | import java.util.concurrent.Future;
52 |
53 | /**
54 | * This is an almost-copy of {@link org.apache.cassandra.hadoop.cql3.CqlBulkRecordWriter}
55 | *
56 | * We had to re-implement this class because of https://issues.apache.org/jira/browse/CASSANDRA-8367
57 | *
58 | */
59 | public class CrunchCqlBulkRecordWriter extends AbstractBulkRecordWriter {
60 |
61 | private static final Logger LOG = LoggerFactory.getLogger(CrunchCqlBulkRecordWriter.class);
62 |
63 | private String keyspace;
64 | private final ProgressHeartbeat heartbeat;
65 |
66 | private String columnFamily;
67 | private String schema;
68 | private String insertStatement;
69 | private File outputDir;
70 |
71 | public CrunchCqlBulkRecordWriter(TaskAttemptContext context) {
72 | super(context);
73 | setConfigs();
74 | heartbeat = new ProgressHeartbeat(context, 120);
75 | }
76 |
77 | private void setConfigs()
78 | {
79 | // if anything is missing, exceptions will be thrown here, instead of on write()
80 | keyspace = ConfigHelper.getOutputKeyspace(conf);
81 | columnFamily = CrunchConfigHelper.getOutputColumnFamily(conf);
82 | schema = CrunchCqlBulkOutputFormat.getColumnFamilySchema(conf, columnFamily);
83 | insertStatement = CrunchCqlBulkOutputFormat.getColumnFamilyInsertStatement(conf, columnFamily);
84 | outputDir = getColumnFamilyDirectory();
85 | }
86 |
87 | private void prepareWriter() {
88 | try {
89 | if (writer == null) {
90 | writer = CQLSSTableWriter.builder()
91 | .forTable(schema)
92 | .using(insertStatement)
93 | .withPartitioner(ConfigHelper.getOutputPartitioner(conf))
94 | .inDirectory(outputDir)
95 | .sorted()
96 | .build();
97 | }
98 | if (loader == null) {
99 | CrunchExternalClient externalClient = new CrunchExternalClient(conf);
100 | externalClient.addKnownCfs(keyspace, schema);
101 | this.loader = new SSTableLoader(outputDir, externalClient,
102 | new BulkRecordWriter.NullOutputHandler());
103 | }
104 | } catch (Exception e) {
105 | throw new CrunchRuntimeException(e);
106 | }
107 | }
108 |
109 | @Override
110 | public void write(final ByteBuffer ignoredKey, final CQLRecord record) {
111 | prepareWriter();
112 | // To ensure Crunch doesn't reuse CQLSSTableWriter's objects
113 | List bb = Lists.newArrayList();
114 | for (ByteBuffer v : record.getValues()) {
115 | bb.add(ByteBufferUtil.clone(v));
116 | }
117 | try {
118 | ((CQLSSTableWriter) writer).rawAddRow(bb);
119 | if (null != progress)
120 | progress.progress();
121 | if (null != context)
122 | HadoopCompat.progress(context);
123 | } catch (InvalidRequestException | IOException e) {
124 | LOG.error(e.getMessage());
125 | throw new CrunchRuntimeException("Error adding row : " + e.getMessage());
126 | }
127 | }
128 |
129 | private File getColumnFamilyDirectory() {
130 | try {
131 | File dir = new File(String.format("%s%s%s%s%s",
132 | getOutputLocation(), File.separator, keyspace, File.separator, columnFamily));
133 | if (!dir.exists() && !dir.mkdirs()) {
134 | throw new CrunchRuntimeException("Failed to created output directory: " + dir);
135 | }
136 | return dir;
137 | } catch (IOException e) {
138 | throw new CrunchRuntimeException(e);
139 | }
140 | }
141 |
142 | @Override
143 | public void close(TaskAttemptContext context) throws InterruptedException {
144 | close();
145 | }
146 |
147 | @Override
148 | @Deprecated
149 | public void close(org.apache.hadoop.mapred.Reporter reporter) {
150 | close();
151 | }
152 |
153 | private void close() {
154 | LOG.info("SSTables built. Now starting streaming");
155 | context.setStatus("streaming");
156 | heartbeat.startHeartbeat();
157 | try {
158 | if (writer != null) {
159 | writer.close();
160 | Future future =
161 | loader.stream(Collections.emptySet(), new ProgressIndicator());
162 | try {
163 | StreamState streamState = Uninterruptibles.getUninterruptibly(future);
164 | if (streamState.hasFailedSession()) {
165 | LOG.warn("Some streaming sessions failed");
166 | } else {
167 | LOG.info("Streaming finished successfully");
168 | }
169 | } catch (ExecutionException e) {
170 | throw new CrunchRuntimeException("Streaming to the following hosts failed: " +
171 | loader.getFailedHosts(), e);
172 | }
173 | } else {
174 | LOG.info("SSTableWriter wasn't instantiated, no streaming happened.");
175 | }
176 | } catch (IOException e) {
177 | throw new CrunchRuntimeException(e);
178 | } finally {
179 | heartbeat.stopHeartbeat();
180 | }
181 | }
182 | }
183 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/cql/CrunchExternalClient.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | *
18 | * The modifications to the upstream file is Copyright 2014 Spotify AB.
19 | * The original upstream file can be found at
20 | * https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java
21 | */
22 | package com.spotify.hdfs2cass.cassandra.cql;
23 |
24 | import org.apache.cassandra.config.CFMetaData;
25 | import org.apache.cassandra.hadoop.AbstractBulkRecordWriter;
26 | import org.apache.hadoop.conf.Configuration;
27 |
28 | import java.util.HashMap;
29 | import java.util.Map;
30 |
31 | /**
32 | * This is an almost-copy of {@link org.apache.cassandra.hadoop.cql3.CqlBulkRecordWriter.ExternalClient}
33 | *
34 | * We had to re-implement this class because of https://issues.apache.org/jira/browse/CASSANDRA-8367
35 | *
36 | */
37 | public class CrunchExternalClient extends AbstractBulkRecordWriter.ExternalClient {
38 | private Map> knownCqlCfs = new HashMap<>();
39 |
40 | public CrunchExternalClient(Configuration conf) {
41 | super(conf);
42 | }
43 |
44 | public void addKnownCfs(String keyspace, String cql) {
45 | Map cfs = knownCqlCfs.get(keyspace);
46 |
47 | if (cfs == null) {
48 | cfs = new HashMap<>();
49 | knownCqlCfs.put(keyspace, cfs);
50 | }
51 | CFMetaData metadata = CFMetaData.compile(cql, keyspace);
52 | cfs.put(metadata.cfName, metadata);
53 | }
54 |
55 | @Override
56 | public CFMetaData getCFMetaData(String keyspace, String cfName) {
57 | CFMetaData metadata = super.getCFMetaData(keyspace, cfName);
58 | if (metadata != null) {
59 | return metadata;
60 | }
61 | Map cfs = knownCqlCfs.get(keyspace);
62 | return cfs != null ? cfs.get(cfName) : null;
63 | }
64 | }
65 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/thrift/CrunchBulkOutputFormat.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | *
18 | * The modifications to the upstream file is Copyright 2014 Spotify AB.
19 | * The original upstream file can be found at
20 | * https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/hadoop/BulkOutputFormat.java
21 | */
22 | package com.spotify.hdfs2cass.cassandra.thrift;
23 |
24 | import org.apache.cassandra.hadoop.AbstractBulkOutputFormat;
25 | import org.apache.cassandra.io.sstable.CrunchBulkRecordWriter;
26 | import org.apache.cassandra.thrift.Mutation;
27 | import org.apache.crunch.CrunchRuntimeException;
28 | import org.apache.hadoop.mapreduce.TaskAttemptContext;
29 | import org.slf4j.Logger;
30 | import org.slf4j.LoggerFactory;
31 |
32 | import java.nio.ByteBuffer;
33 | import java.util.List;
34 |
35 | /**
36 | * This is an almost-copy of {@link org.apache.cassandra.hadoop.BulkOutputFormat}
37 | *
38 | * We had to re-implement this class (and its inner private classes) because of clash between
39 | * Cassandra's and Crunch's MapReduce configs.
40 | * See https://issues.apache.org/jira/browse/CASSANDRA-8367 for more info.
41 | *
42 | * This is a temporary workaround and will be removed in the future.
43 | *
44 | * We return {@link com.spotify.hdfs2cass.cassandra.cql.CrunchCqlBulkRecordWriter}.
45 | *
46 | */
47 | public class CrunchBulkOutputFormat extends AbstractBulkOutputFormat> {
48 | private final Logger logger = LoggerFactory.getLogger(CrunchBulkOutputFormat.class);
49 |
50 | /**
51 | * Not used anyway, so do not bother implementing.
52 | */
53 | @Deprecated
54 | public CrunchBulkRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress) {
55 | throw new CrunchRuntimeException("Use getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext)");
56 | }
57 |
58 | @Override
59 | public CrunchBulkRecordWriter getRecordWriter(final TaskAttemptContext context) {
60 | return new CrunchBulkRecordWriter(context);
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/thrift/ExternalSSTableLoaderClient.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | *
18 | * The modifications to the upstream file is Copyright 2014 Spotify AB.
19 | * The original upstream file can be found at
20 | * https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
21 | */
22 | package com.spotify.hdfs2cass.cassandra.thrift;
23 |
24 | import com.google.common.collect.Maps;
25 | import com.google.common.collect.Sets;
26 | import org.apache.cassandra.auth.IAuthenticator;
27 | import org.apache.cassandra.config.CFMetaData;
28 | import org.apache.cassandra.dht.Range;
29 | import org.apache.cassandra.dht.Token;
30 | import org.apache.cassandra.io.sstable.SSTableLoader;
31 | import org.apache.cassandra.thrift.AuthenticationRequest;
32 | import org.apache.cassandra.thrift.Cassandra;
33 | import org.apache.cassandra.thrift.CfDef;
34 | import org.apache.cassandra.thrift.KsDef;
35 | import org.apache.cassandra.thrift.TokenRange;
36 | import org.apache.crunch.CrunchRuntimeException;
37 | import org.apache.thrift.protocol.TProtocol;
38 | import org.apache.thrift.transport.TFramedTransport;
39 | import org.apache.thrift.transport.TSocket;
40 | import org.apache.thrift.transport.TTransport;
41 | import org.apache.thrift.transport.TTransportException;
42 |
43 | import java.net.InetAddress;
44 | import java.net.UnknownHostException;
45 | import java.util.HashMap;
46 | import java.util.Iterator;
47 | import java.util.List;
48 | import java.util.Map;
49 | import java.util.Set;
50 |
51 | /**
52 | * This is an almost-copy of {@link org.apache.cassandra.hadoop.AbstractBulkRecordWriter.ExternalClient}
53 | *
54 | * We had to re-implement this class because of https://issues.apache.org/jira/browse/CASSANDRA-8367
55 | *
56 | */
57 | public class ExternalSSTableLoaderClient extends SSTableLoader.Client {
58 | private final Map> knownCfs = new HashMap<>();
59 | private final String hostlist;
60 | private final int rpcPort;
61 | private final String username;
62 | private final String password;
63 |
64 | public ExternalSSTableLoaderClient(String hostlist, int port, String username, String password) {
65 | super();
66 | this.hostlist = hostlist;
67 | this.rpcPort = port;
68 | this.username = username;
69 | this.password = password;
70 | }
71 |
72 | public void init(String keyspace) {
73 | Set hosts = Sets.newHashSet();
74 | String[] nodes = hostlist.split(",");
75 | for (String node : nodes) {
76 | try {
77 | hosts.add(InetAddress.getByName(node));
78 | } catch (UnknownHostException e) {
79 | throw new RuntimeException(e);
80 | }
81 | }
82 |
83 | Iterator hostiter = hosts.iterator();
84 | while (hostiter.hasNext()) {
85 | try {
86 | InetAddress host = hostiter.next();
87 | Cassandra.Client client = createThriftClient(host.getHostAddress(), rpcPort);
88 |
89 | // log in
90 | client.set_keyspace(keyspace);
91 | if (username != null) {
92 | Map creds = Maps.newHashMap();
93 | creds.put(IAuthenticator.USERNAME_KEY, username);
94 | creds.put(IAuthenticator.PASSWORD_KEY, password);
95 | AuthenticationRequest authRequest = new AuthenticationRequest(creds);
96 | client.login(authRequest);
97 | }
98 |
99 | List tokenRanges = client.describe_ring(keyspace);
100 | List ksDefs = client.describe_keyspaces();
101 |
102 | setPartitioner(client.describe_partitioner());
103 | Token.TokenFactory tkFactory = getPartitioner().getTokenFactory();
104 |
105 | for (TokenRange tr : tokenRanges) {
106 | Range range = new Range<>(tkFactory.fromString(tr.start_token), tkFactory.fromString(tr.end_token));
107 | for (String ep : tr.endpoints) {
108 | addRangeForEndpoint(range, InetAddress.getByName(ep));
109 | }
110 | }
111 |
112 | for (KsDef ksDef : ksDefs) {
113 | Map cfs = new HashMap<>(ksDef.cf_defs.size());
114 | for (CfDef cfDef : ksDef.cf_defs)
115 | cfs.put(cfDef.name, CFMetaData.fromThrift(cfDef));
116 | knownCfs.put(ksDef.name, cfs);
117 | }
118 | break;
119 | } catch (Exception e) {
120 | throw new CrunchRuntimeException("Could not retrieve endpoint ranges: ", e);
121 | }
122 | }
123 | }
124 |
125 | public CFMetaData getCFMetaData(String keyspace, String cfName) {
126 | Map cfs = knownCfs.get(keyspace);
127 | return cfs != null ? cfs.get(cfName) : null;
128 | }
129 |
130 | private static Cassandra.Client createThriftClient(String host, int port) throws TTransportException {
131 | TSocket socket = new TSocket(host, port);
132 | TTransport trans = new TFramedTransport(socket);
133 | trans.open();
134 | TProtocol protocol = new org.apache.thrift.protocol.TBinaryProtocol(trans);
135 | return new Cassandra.Client(protocol);
136 | }
137 | }
138 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/thrift/ProgressHeartbeat.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.cassandra.thrift;
17 |
18 | import org.apache.hadoop.util.Progressable;
19 | import org.slf4j.Logger;
20 | import org.slf4j.LoggerFactory;
21 |
22 | /**
23 | * Runs a heartbeat thread in the background that calls progress every SLEEP_MINS in order to keep
24 | * DoFns from timing out. The heartbeat will stop calling progress() after stopAfterMins.
25 | */
26 | public class ProgressHeartbeat extends Thread {
27 |
28 | private static final Logger LOG = LoggerFactory.getLogger(ProgressHeartbeat.class);
29 |
30 | private static final int SLEEP_MINS = 1;
31 |
32 | private final Progressable progressable;
33 | private final int stopAfterMins;
34 |
35 | private boolean isCancelled;
36 |
37 | public ProgressHeartbeat(Progressable progressable, int stopAfterMins) {
38 | setDaemon(true);
39 | this.progressable = progressable;
40 | this.stopAfterMins = stopAfterMins;
41 | this.isCancelled = false;
42 | }
43 |
44 | public void startHeartbeat() {
45 | this.start();
46 | }
47 |
48 | public void stopHeartbeat() {
49 | isCancelled = true;
50 | }
51 |
52 | @Override
53 | public void run() {
54 | int minsRunning = 0;
55 | while (!isCancelled && minsRunning < stopAfterMins) {
56 | LOG.debug("Heartbeat invoked");
57 | progressable.progress();
58 | try {
59 | Thread.sleep(1000L * 60L * SLEEP_MINS);
60 | } catch (InterruptedException e) {
61 | Thread.currentThread().interrupt();
62 | }
63 | minsRunning += SLEEP_MINS;
64 | }
65 | }
66 | }
67 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/thrift/ProgressIndicator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.cassandra.thrift;
17 |
18 | import com.google.common.collect.Maps;
19 | import com.google.common.collect.Sets;
20 | import org.apache.cassandra.streaming.ProgressInfo;
21 | import org.apache.cassandra.streaming.SessionInfo;
22 | import org.apache.cassandra.streaming.StreamEvent;
23 | import org.apache.cassandra.streaming.StreamEventHandler;
24 | import org.apache.cassandra.streaming.StreamState;
25 | import org.slf4j.Logger;
26 | import org.slf4j.LoggerFactory;
27 |
28 | import java.net.InetAddress;
29 | import java.util.Map;
30 | import java.util.Set;
31 | import java.util.concurrent.ConcurrentHashMap;
32 | import java.util.concurrent.TimeUnit;
33 |
34 | /**
35 | * Return true when everything is at 100%
36 | */
37 | public class ProgressIndicator implements StreamEventHandler {
38 |
39 | private static final Logger LOG = LoggerFactory.getLogger(ProgressIndicator.class);
40 |
41 | private final Map sessionsByHost = new ConcurrentHashMap<>();
42 | private final Map> progressByHost = new ConcurrentHashMap<>();
43 |
44 | private long start;
45 | private long lastProgress;
46 | private long lastTime;
47 |
48 | public ProgressIndicator() {
49 | start = lastTime = System.nanoTime();
50 | }
51 |
52 | @Override
53 | public void onSuccess(StreamState finalState) {
54 | }
55 |
56 | @Override
57 | public void onFailure(Throwable t) {
58 | }
59 |
60 | @Override
61 | public void handleStreamEvent(StreamEvent event) {
62 |
63 | LOG.debug("Handling stream event");
64 |
65 | if (event.eventType == StreamEvent.Type.STREAM_PREPARED) {
66 |
67 | SessionInfo session = ((StreamEvent.SessionPreparedEvent) event).session;
68 | sessionsByHost.put(session.peer, session);
69 | LOG.info(String.format("Session to %s created", session.connecting.getHostAddress()));
70 |
71 | } else if (event.eventType == StreamEvent.Type.STREAM_COMPLETE ) {
72 |
73 | StreamEvent.SessionCompleteEvent completionEvent = ((StreamEvent.SessionCompleteEvent) event);
74 | if (completionEvent.success) {
75 | LOG.info(String.format("Stream to %s successful.", completionEvent.peer.getHostAddress()));
76 | } else {
77 | LOG.info(String.format("Stream to %s failed.", completionEvent.peer.getHostAddress()));
78 | }
79 | } else if (event.eventType == StreamEvent.Type.FILE_PROGRESS) {
80 |
81 | ProgressInfo progressInfo = ((StreamEvent.ProgressEvent) event).progress;
82 |
83 | // update progress
84 | Set progresses = progressByHost.get(progressInfo.peer);
85 | if (progresses == null) {
86 | progresses = Sets.newSetFromMap(Maps.newConcurrentMap());
87 | progressByHost.put(progressInfo.peer, progresses);
88 | }
89 | if (progresses.contains(progressInfo)) {
90 | progresses.remove(progressInfo);
91 | }
92 | progresses.add(progressInfo);
93 |
94 | // craft status update string
95 | StringBuilder sb = new StringBuilder();
96 | sb.append("progress: ");
97 |
98 | long totalProgress = 0;
99 | long totalSize = 0;
100 | for (Map.Entry> entry : progressByHost.entrySet()) {
101 | SessionInfo session = sessionsByHost.get(entry.getKey());
102 |
103 | long size = session.getTotalSizeToSend();
104 | long current = 0;
105 | int completed = 0;
106 | for (ProgressInfo progress : entry.getValue()) {
107 | if (progress.currentBytes == progress.totalBytes) {
108 | completed++;
109 | }
110 | current += progress.currentBytes;
111 | }
112 | totalProgress += current;
113 | totalSize += size;
114 | sb.append("[").append(entry.getKey());
115 | sb.append(" ").append(completed).append("/").append(session.getTotalFilesToSend());
116 | sb.append(" (").append(size == 0 ? 100L : current * 100L / size).append("%)] ");
117 | }
118 | long time = System.nanoTime();
119 | long deltaTime = TimeUnit.NANOSECONDS.toMillis(time - lastTime);
120 | lastTime = time;
121 | long deltaProgress = totalProgress - lastProgress;
122 | lastProgress = totalProgress;
123 |
124 | sb.append("[total: ").append(totalSize == 0 ? 100L : totalProgress * 100L / totalSize).append("% - ");
125 | sb.append(mbPerSec(deltaProgress, deltaTime)).append("MB/s");
126 | sb.append(" (avg: ").append(mbPerSec(totalProgress, TimeUnit.NANOSECONDS.toMillis(time - start))).append("MB/s)]");
127 |
128 | LOG.info(sb.toString());
129 | }
130 | }
131 |
132 | private int mbPerSec(long bytes, long timeInMs) {
133 | double bytesPerMs = ((double) bytes) / timeInMs;
134 | return (int) ((bytesPerMs * 1000) / (1024 * 2024));
135 | }
136 | }
137 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/utils/CassandraClusterInfo.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.cassandra.utils;
17 |
18 | import com.datastax.driver.core.Cluster;
19 | import com.datastax.driver.core.ColumnMetadata;
20 | import com.datastax.driver.core.KeyspaceMetadata;
21 | import com.datastax.driver.core.Metadata;
22 | import com.datastax.driver.core.TableMetadata;
23 | import com.google.common.base.Optional;
24 | import com.google.common.collect.Lists;
25 | import com.spotify.hdfs2cass.cassandra.thrift.ExternalSSTableLoaderClient;
26 | import org.apache.cassandra.config.Config;
27 | import org.apache.cassandra.hadoop.ConfigHelper;
28 | import org.apache.crunch.CrunchRuntimeException;
29 | import org.apache.hadoop.conf.Configuration;
30 | import org.slf4j.Logger;
31 | import org.slf4j.LoggerFactory;
32 |
33 | import java.io.Serializable;
34 | import java.util.List;
35 |
36 | public class CassandraClusterInfo implements Serializable {
37 |
38 | private static final Logger logger = LoggerFactory.getLogger(CassandraClusterInfo.class);
39 |
40 | private final String host;
41 | private final int port;
42 | private String partitionerClass;
43 | private int numClusterNodes;
44 | private String keyspace;
45 | private String columnFamily;
46 | private String cqlSchema;
47 | private List columns;
48 | private int[] partitionKeyIndexes;
49 |
50 | /**
51 | * Uses DataStax JavaDriver to fetch Cassandra cluster metadata.
52 | *
53 | * @param host Hostname of a node in the cluster.
54 | * @param port Binary/cql protocol port. Optional.
55 | */
56 | public CassandraClusterInfo(final String host, final int port) {
57 | this.host = host;
58 | this.port = port;
59 | }
60 |
61 | public void init(final String keyspace, final String columnFamily) {
62 |
63 | this.keyspace = keyspace;
64 | this.columnFamily = columnFamily;
65 |
66 | // connect to the cluster
67 | Cluster.Builder clusterBuilder = Cluster.builder();
68 | clusterBuilder.addContactPoints(host);
69 | if (port != -1) {
70 | clusterBuilder.withPort(port);
71 | }
72 |
73 | // ask for some metadata
74 | logger.info("getting cluster metadata for {}.{}", keyspace, columnFamily);
75 | final TableMetadata tableMetadata;
76 | try (final Cluster cluster = clusterBuilder.build()) {
77 | Metadata clusterMetadata = cluster.getMetadata();
78 | KeyspaceMetadata keyspaceMetadata = clusterMetadata.getKeyspace('"' + keyspace + '"');
79 | tableMetadata = keyspaceMetadata.getTable('"' + columnFamily + '"');
80 | cqlSchema = tableMetadata.asCQLQuery();
81 | partitionerClass = clusterMetadata.getPartitioner();
82 | Class.forName(partitionerClass);
83 | numClusterNodes = clusterMetadata.getAllHosts().size();
84 | columns = tableMetadata.getColumns();
85 | } catch (ClassNotFoundException cnfe) {
86 | throw new CrunchRuntimeException("No such partitioner: " + partitionerClass, cnfe);
87 | } catch (NullPointerException npe) {
88 | String msg = String.format("No such keyspace/table: %s/%s", keyspace, columnFamily);
89 | throw new CrunchRuntimeException(msg, npe);
90 | }
91 |
92 | // map the partition key columns
93 | final List partitionKeyColumns = tableMetadata.getPartitionKey();
94 | partitionKeyIndexes = new int[partitionKeyColumns.size()];
95 | for (int i = 0; i < partitionKeyColumns.size(); i++) {
96 | final String keyColName = partitionKeyColumns.get(i).getName();
97 | int j;
98 | for (j = 0; j < columns.size(); j++) {
99 | if (columns.get(j).getName().equals(keyColName)) {
100 | partitionKeyIndexes[i] = j;
101 | logger.info("partition key column {} index {}", keyColName, j);
102 | break;
103 | }
104 | }
105 | if (j == columns.size()) {
106 | throw new CrunchRuntimeException("no matching column for key " + keyColName);
107 | }
108 | }
109 | }
110 |
111 | /**
112 | * The partitioner used by the Cassandra cluster
113 | *
114 | * @return The full class name of the partitioner or null, if error
115 | */
116 | public String getPartitionerClass() {
117 | return partitionerClass;
118 | }
119 |
120 | /**
121 | * The number of nodes participating in the cluster
122 | *
123 | * @return The number of nodes or zero, if error
124 | */
125 | public int getNumClusterNodes() {
126 | return numClusterNodes;
127 | }
128 |
129 | /**
130 | * CQL schema of the table data is imported to
131 | *
132 | * @return valid CQL command to create the table
133 | */
134 | public String getCqlSchema() {
135 | return cqlSchema;
136 | }
137 |
138 | public int[] getPartitionKeyIndexes() {
139 | return partitionKeyIndexes;
140 | }
141 |
142 | /**
143 | * Get all column names from table metadata. Used if
144 | * {@link com.spotify.hdfs2cass.cassandra.utils.CassandraParams} don't specify column names.
145 | */
146 | public String[] getAllColumnNames() {
147 | List colNames = Lists.newArrayList();
148 | for (ColumnMetadata col : columns) {
149 | colNames.add(col.getName());
150 | }
151 | return colNames.toArray(new String[colNames.size()]);
152 | }
153 |
154 | /**
155 | * Prepare the insert statement with column names ordered as they appear in columnNames.
156 | *
157 | * @param columnNames array of column names
158 | * @return Prepared insert statement, e.g. 'INSERT INTO ks.table (column) VALUES (?);'
159 | */
160 | public String buildPreparedStatement(String[] columnNames) {
161 | StringBuilder colNames = new StringBuilder();
162 | StringBuilder valueTemplates = new StringBuilder();
163 | for (String col : columnNames) {
164 | colNames.append(String.format("%s, ", col));
165 | valueTemplates.append("?, ");
166 | }
167 | // remove last ','
168 | colNames.deleteCharAt(colNames.lastIndexOf(","));
169 | valueTemplates.deleteCharAt(valueTemplates.lastIndexOf(","));
170 | return String.format("INSERT INTO %s.%s (%s) VALUES (%s) USING TIMESTAMP ? AND TTL ?;",
171 | keyspace, columnFamily, colNames.toString(), valueTemplates.toString());
172 | }
173 |
174 | public void validateThriftAccessible(final Optional rpcPort) {
175 | Config.setClientMode(true);
176 |
177 | int port = rpcPort.or(ConfigHelper.getOutputRpcPort(new Configuration()));
178 |
179 | ExternalSSTableLoaderClient client = new ExternalSSTableLoaderClient(this.host, port, null, null);
180 | client.init(this.keyspace);
181 | if (client.getCFMetaData(this.keyspace, this.columnFamily) == null) {
182 | throw new CrunchRuntimeException("Column family not accessible: " + this.keyspace + "." + this.columnFamily);
183 | }
184 | }
185 | }
186 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/utils/CassandraKeyComparator.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2016 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 |
17 | package com.spotify.hdfs2cass.cassandra.utils;
18 |
19 | import com.google.common.base.Throwables;
20 | import org.apache.avro.io.BinaryDecoder;
21 | import org.apache.avro.io.DecoderFactory;
22 | import org.apache.avro.mapred.AvroKey;
23 | import org.apache.cassandra.dht.IPartitioner;
24 | import org.apache.hadoop.conf.Configurable;
25 | import org.apache.hadoop.conf.Configuration;
26 | import org.apache.hadoop.io.RawComparator;
27 |
28 | import java.io.IOException;
29 | import java.nio.ByteBuffer;
30 |
31 | /**
32 | * A comparator for sorting keys in sstable order. This is used in the shuffle
33 | * to ensure that the reducer sees inputs in the correct order and can append
34 | * them to sstables without sorting again.
35 | */
36 | public class CassandraKeyComparator implements RawComparator>, Configurable {
37 | private static final DecoderFactory DECODER_FACTORY = DecoderFactory.get();
38 |
39 | private Configuration conf;
40 | private IPartitioner> partitioner;
41 |
42 | @Override
43 | public int compare(byte[] o1, int s1, int l1, byte[] o2, int s2, int l2) {
44 | try {
45 | final BinaryDecoder d1 = DECODER_FACTORY.binaryDecoder(o1, s1, l1, null);
46 | final ByteBuffer key1 = d1.readBytes(null);
47 |
48 | // re-use the decoder instance, but do not re-use the byte buffer,
49 | // because DecoratedKey stores a reference
50 | final BinaryDecoder d2 = DECODER_FACTORY.binaryDecoder(o2, s2, l2, d1);
51 | final ByteBuffer key2 = d2.readBytes(null);
52 |
53 | return compare(key1, key2);
54 | } catch (final IOException e) {
55 | throw Throwables.propagate(e);
56 | }
57 | }
58 |
59 | @Override
60 | public int compare(AvroKey o1, AvroKey o2) {
61 | final ByteBuffer key1 = o1.datum();
62 | final ByteBuffer key2 = o2.datum();
63 | return compare(key1, key2);
64 | }
65 |
66 | private int compare(final ByteBuffer key1, final ByteBuffer key2) {
67 | assert key1 != key2 : "bug - unsafe buffer re-use";
68 | return partitioner.decorateKey(key1).compareTo(partitioner.decorateKey(key2));
69 | }
70 |
71 | @Override
72 | public Configuration getConf() {
73 | return conf;
74 | }
75 |
76 | @Override
77 | public void setConf(Configuration conf) {
78 | this.conf = conf;
79 | final String partitionerParam = conf.get(CassandraParams.SCRUB_CASSANDRACLUSTER_PARTITIONER_CONFIG);
80 | if (partitionerParam == null) {
81 | throw new RuntimeException("Didn't get any cassandra partitioner information");
82 | }
83 | try {
84 | partitioner = (IPartitioner>) Class.forName(partitionerParam).newInstance();
85 | } catch (final Exception e) {
86 | throw Throwables.propagate(e);
87 | }
88 | }
89 | }
90 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/utils/CassandraParams.java:
--------------------------------------------------------------------------------
1 | package com.spotify.hdfs2cass.cassandra.utils;
2 |
3 | import com.google.common.base.Objects;
4 | import com.google.common.base.Optional;
5 | import com.google.common.collect.Maps;
6 | import com.spotify.hdfs2cass.crunch.CrunchConfigHelper;
7 | import com.spotify.hdfs2cass.crunch.cql.CQLRecord;
8 | import org.apache.cassandra.dht.Murmur3Partitioner;
9 | import org.apache.cassandra.dht.RandomPartitioner;
10 | import org.apache.cassandra.hadoop.ConfigHelper;
11 | import org.apache.cassandra.tools.BulkLoader;
12 | import org.apache.commons.lang.StringUtils;
13 | import org.apache.crunch.CrunchRuntimeException;
14 | import org.apache.crunch.GroupingOptions;
15 | import org.apache.crunch.MapFn;
16 | import org.apache.hadoop.conf.Configuration;
17 | import org.apache.hadoop.mapred.JobConf;
18 | import org.slf4j.Logger;
19 | import org.slf4j.LoggerFactory;
20 |
21 | import java.io.Serializable;
22 | import java.math.BigInteger;
23 | import java.net.URI;
24 | import java.nio.ByteBuffer;
25 | import java.util.ArrayList;
26 | import java.util.Collections;
27 | import java.util.Map;
28 | import java.util.Random;
29 |
30 | public class CassandraParams implements Serializable {
31 | private static final Logger logger = LoggerFactory.getLogger(CassandraParams.class);
32 |
33 | public static final String SCRUB_CASSANDRACLUSTER_PARTITIONER_CONFIG = "scrub.cassandracluster.com.spotify.cassandra.thrift.partitioner";
34 | public static final String SCRUB_CASSANDRACLUSTER_RANGE_PER_REDUCER_CONFIG = "scrub.cassandracluster.com.spotify.cassandra.thrift.rangeperreducer";
35 | public static final String SCRUB_CASSANDRACLUSTER_REDUCERS_CONFIG = "scrub.cassandracluster.com.spotify.cassandra.thrift.reducers";
36 | public static final String SCRUB_CASSANDRACLUSTER_DISTRIBUTE_RANDOMLY_CONFIG = "scrub.cassandracluster.com.spotify.cassandra.thrift.distributerandomly";
37 |
38 | private CassandraClusterInfo clusterInfo;
39 |
40 | private String seedNodeHost;
41 | private int seedNodePort;
42 | private String columnFamily;
43 | private String keyspace;
44 | private Optional rpcPort = Optional.absent();
45 |
46 | private String partitioner;
47 |
48 | private Optional streamThrottleMBits = Optional.absent();
49 | private Optional compressionClass = Optional.absent();
50 | private int reducers = 0;
51 | private boolean distributeRandomly = false;
52 | private String schema;
53 | private String statement;
54 | private String[] columnNames;
55 |
56 | /**
57 | * Configures CassandraProvider based on the target hdfs2cass resource URI.
58 | * The URI has schema:
59 | * (thrift|cql)://seedNodeHost[:port]/keySpace/colFamily?query_string
60 | * query_string keys:
61 | * - columnnames
62 | * - compressionclass
63 | * - distributerandomly
64 | * - reducers
65 | * - streamthrottlembits
66 | * - rpcport
67 | */
68 | private CassandraParams() {
69 | }
70 |
71 | public static CassandraParams parse(URI dataResource) {
72 | String queryString = Objects.firstNonNull(dataResource.getQuery(), "");
73 | Map query = parseQuery(queryString);
74 |
75 | CassandraParams params = new CassandraParams();
76 | params.seedNodeHost = dataResource.getHost();
77 | params.seedNodePort = dataResource.getPort();
78 | String[] path = dataResource.getPath().split("/");
79 | params.keyspace = path[1];
80 | params.columnFamily = path[2];
81 |
82 | params.clusterInfo = new CassandraClusterInfo(params.seedNodeHost, params.seedNodePort);
83 | params.clusterInfo.init(params.keyspace, params.columnFamily);
84 | params.partitioner = params.clusterInfo.getPartitionerClass();
85 |
86 | params.schema = params.clusterInfo.getCqlSchema();
87 | String[] columnNames;
88 | if (query.containsKey("columnnames")) {
89 | columnNames = query.get("columnnames").split(",");
90 | } else {
91 | columnNames = params.clusterInfo.getAllColumnNames();
92 | }
93 | params.statement = params.clusterInfo.buildPreparedStatement(columnNames);
94 | params.columnNames = columnNames;
95 |
96 | if (query.containsKey("streamthrottlembits")) {
97 | params.streamThrottleMBits = Optional.of(Integer.parseInt(query.get("streamthrottlembits")));
98 | logger.info("setting streamthrottlembits to " + params.streamThrottleMBits.get());
99 | } else {
100 | logger.warn("No throttling specified");
101 | }
102 |
103 | if (query.containsKey("compressionclass")) {
104 | params.compressionClass = Optional.of(query.get("compressionclass"));
105 | }
106 |
107 | if (query.containsKey("mappers")) {
108 | logger.warn("mappers argument has been deprecated and is now ignored.");
109 | }
110 |
111 | if (query.containsKey("reducers")) {
112 | params.reducers = Integer.parseInt(query.get("reducers"));
113 | } else {
114 | params.reducers = params.clusterInfo.getNumClusterNodes();
115 | }
116 |
117 | if (query.containsKey("copiers")) {
118 | logger.warn("copiers argument has been deprecated and is now ignored.");
119 | }
120 |
121 | if (query.containsKey("distributerandomly")) {
122 | params.distributeRandomly = Boolean.parseBoolean(query.get("distributerandomly"));
123 | }
124 |
125 | if (query.containsKey("rpcport")) {
126 | params.rpcPort = Optional.of(Integer.parseInt(query.get("rpcport")));
127 | }
128 |
129 | if ("thrift".equals(dataResource.getScheme())) {
130 | logger.warn("Thrift support is deprecated and will be removed, please use CQL instead");
131 | params.clusterInfo.validateThriftAccessible(params.rpcPort);
132 | }
133 |
134 | return params;
135 | }
136 |
137 | public static Map parseQuery(String query) {
138 | final Map result = Maps.newHashMap();
139 | final String[] pairs = query.split("&");
140 | for (String pair : pairs) {
141 | if (pair.isEmpty())
142 | continue;
143 |
144 | final int idx = pair.indexOf("=");
145 | if (idx > -1) {
146 | result.put(pair.substring(0, idx), pair.substring(idx + 1));
147 | } else {
148 | result.put(pair, "true");
149 | }
150 | }
151 | return result;
152 | }
153 |
154 | public void configure(final Configuration conf) {
155 | if (conf instanceof JobConf) {
156 | configure((JobConf) conf);
157 | } else {
158 | String msg = String.format("Attempting to run a job with unknown config type: %s",
159 | conf.toString());
160 | throw new CrunchRuntimeException(msg);
161 | }
162 | }
163 |
164 | private void configure(final JobConf conf) {
165 | ConfigHelper.setOutputInitialAddress(conf, this.getSeedNodeHost());
166 | CrunchConfigHelper.setOutputColumnFamily(conf, this.getKeyspace(), this.getColumnFamily());
167 | ConfigHelper.setOutputPartitioner(conf, this.getPartitioner());
168 |
169 | if (this.getStreamThrottleMBits().isPresent()) {
170 | conf.set("mapreduce.output.bulkoutputformat.streamthrottlembits",
171 | this.getStreamThrottleMBits().get().toString());
172 | }
173 |
174 | if (this.getCompressionClass().isPresent()) {
175 | ConfigHelper.setOutputCompressionClass(conf, this.getCompressionClass().get());
176 | }
177 |
178 | if (this.getRpcPort().isPresent()) {
179 | ConfigHelper.setOutputRpcPort(conf, String.valueOf(this.getRpcPort().get()));
180 | }
181 |
182 | conf.setJarByClass(BulkLoader.class);
183 | }
184 |
185 | /**
186 | * A Cassandra host used to fetch information about the Cassandra cluster.
187 | *
188 | * @return hostname
189 | */
190 | public String getSeedNodeHost() {
191 | return seedNodeHost;
192 | }
193 |
194 | /**
195 | * Cassandra column family hdfs2cass is imported to.
196 | *
197 | * @return column family name
198 | */
199 | public String getColumnFamily() {
200 | return columnFamily;
201 | }
202 |
203 | /**
204 | * Cassandra keyspace hdfs2cass is imported to.
205 | *
206 | * @return keyspace name
207 | */
208 | public String getKeyspace() {
209 | return keyspace;
210 | }
211 |
212 | /**
213 | * Cassandra partitioner the cluster is using.
214 | *
215 | * @return full class name
216 | */
217 | public String getPartitioner() {
218 | return partitioner;
219 | }
220 |
221 | /**
222 | * Maximum throughput the streaming of SSTables can happen with.
223 | *
224 | * @return
225 | */
226 | public Optional getStreamThrottleMBits() {
227 | return streamThrottleMBits;
228 | }
229 |
230 | /**
231 | * Compression used when writing SSTables.
232 | *
233 | * @return full or simple class name
234 | */
235 | public Optional getCompressionClass() {
236 | return compressionClass;
237 | }
238 |
239 | /**
240 | * Number of reducers for the import job
241 | *
242 | * @return
243 | */
244 | public int getReducers() {
245 | return reducers;
246 | }
247 |
248 | /**
249 | * Override Cassandra partitioner and distribute hdfs2cass randomly.
250 | *
251 | * @return
252 | */
253 | public boolean getDistributeRandomly() {
254 | return distributeRandomly;
255 | }
256 |
257 | /**
258 | * If using CQL, get schema of table being imported.
259 | *
260 | * @return
261 | */
262 | public String getSchema() {
263 | return schema;
264 | }
265 |
266 | /**
267 | * If using CQL, get prepared statement for inserting values.
268 | *
269 | * @return
270 | */
271 | public String getStatement() {
272 | return statement;
273 | }
274 |
275 | /**
276 | * If using CQL, get a list of column names as they appear in the insert statement.
277 | *
278 | * @return
279 | */
280 | public String[] getColumnNames() {
281 | return columnNames;
282 | }
283 |
284 | public Optional getRpcPort() {
285 | return rpcPort;
286 | }
287 |
288 | public GroupingOptions createGroupingOptions() {
289 | logger.info("GroupingOptions.numReducers: " + this.getReducers());
290 | GroupingOptions.Builder builder = GroupingOptions.builder()
291 | .partitionerClass(CassandraPartitioner.class)
292 | .sortComparatorClass(CassandraKeyComparator.class)
293 | .numReducers(this.getReducers());
294 |
295 | final BigInteger maxToken;
296 | final BigInteger minToken;
297 | switch (clusterInfo.getPartitionerClass()) {
298 | case "org.apache.cassandra.dht.RandomPartitioner":
299 | maxToken = RandomPartitioner.MAXIMUM.subtract(BigInteger.ONE);
300 | minToken = RandomPartitioner.ZERO;
301 | break;
302 | case "org.apache.cassandra.dht.Murmur3Partitioner":
303 | maxToken = BigInteger.valueOf(Murmur3Partitioner.MAXIMUM);
304 | minToken = BigInteger.valueOf(Murmur3Partitioner.MINIMUM.token);
305 | break;
306 | default:
307 | throw new IllegalArgumentException("Unknown partitioner class: " + clusterInfo.getPartitionerClass());
308 | }
309 |
310 | final BigInteger[] rangeWidth = maxToken
311 | .subtract(minToken)
312 | .add(BigInteger.ONE)
313 | .divideAndRemainder(BigInteger.valueOf(this.getReducers()));
314 | if (!rangeWidth[1].equals(BigInteger.ZERO)) {
315 | rangeWidth[0] = rangeWidth[0].add(BigInteger.ONE);
316 | }
317 | BigInteger rangePerReducer = rangeWidth[0];
318 |
319 | ArrayList reducerList = new ArrayList<>(this.getReducers());
320 | for (int i = 0; i < this.getReducers(); i++) {
321 | reducerList.add(i);
322 | }
323 |
324 | Collections.shuffle(reducerList, new Random());
325 |
326 | builder.conf(SCRUB_CASSANDRACLUSTER_PARTITIONER_CONFIG, clusterInfo.getPartitionerClass());
327 | builder.conf(SCRUB_CASSANDRACLUSTER_RANGE_PER_REDUCER_CONFIG, rangePerReducer.toString());
328 | builder.conf(SCRUB_CASSANDRACLUSTER_REDUCERS_CONFIG, StringUtils.join(reducerList, ","));
329 | if (this.getDistributeRandomly()) {
330 | builder.conf(SCRUB_CASSANDRACLUSTER_DISTRIBUTE_RANDOMLY_CONFIG, Boolean.TRUE.toString());
331 | }
332 |
333 | return builder.build();
334 | }
335 |
336 | /**
337 | * @return a map function to extract the partition key from a record
338 | */
339 | public MapFn getKeyFn() {
340 | return makeKeyFn(clusterInfo.getPartitionKeyIndexes());
341 | }
342 |
343 | private static MapFn makeKeyFn(final int[] partitionKeyIndexes) {
344 | return new MapFn() {
345 | @Override
346 | public ByteBuffer map(final CQLRecord record) {
347 | return CassandraRecordUtils.getPartitionKey(record.getValues(), partitionKeyIndexes);
348 | }
349 | };
350 | }
351 | }
352 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/utils/CassandraPartitioner.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.cassandra.utils;
17 |
18 | import org.apache.avro.mapred.AvroKey;
19 | import org.apache.cassandra.dht.AbstractPartitioner;
20 | import org.apache.cassandra.dht.BigIntegerToken;
21 | import org.apache.cassandra.dht.LongToken;
22 | import org.apache.cassandra.dht.Murmur3Partitioner;
23 | import org.apache.cassandra.dht.Token;
24 | import org.apache.commons.lang.StringUtils;
25 | import org.apache.hadoop.conf.Configurable;
26 | import org.apache.hadoop.conf.Configuration;
27 | import org.apache.hadoop.mapreduce.Partitioner;
28 | import org.slf4j.Logger;
29 | import org.slf4j.LoggerFactory;
30 |
31 | import java.io.Serializable;
32 | import java.math.BigInteger;
33 | import java.nio.ByteBuffer;
34 | import java.util.ArrayList;
35 | import java.util.Arrays;
36 | import java.util.List;
37 | import java.util.Random;
38 |
39 | /**
40 | * Uses the cassandra topology to send a key to a particular set of reducers
41 | */
42 | public class CassandraPartitioner extends Partitioner, Object> implements Configurable, Serializable {
43 |
44 | private static final Logger logger = LoggerFactory.getLogger(CassandraPartitioner.class);
45 |
46 | private static final BigInteger MURMUR3_SCALE =
47 | BigInteger.valueOf(Murmur3Partitioner.MINIMUM.token).abs();
48 |
49 | private AbstractPartitioner partitioner;
50 | private BigInteger rangePerReducer;
51 | private List reducers;
52 | private boolean distributeRandomly;
53 | private Random random;
54 | private Configuration conf;
55 |
56 | @Override
57 | public int getPartition(AvroKey key, Object value, int numReducers) {
58 | if (distributeRandomly) {
59 | return reducers.get(random.nextInt(reducers.size()));
60 | }
61 |
62 | final Token token = partitioner.getToken(key.datum());
63 | BigInteger bigIntToken;
64 | if (token instanceof BigIntegerToken) {
65 | bigIntToken = ((BigIntegerToken) token).token.abs();
66 | } else if (token instanceof LongToken) {
67 | bigIntToken = BigInteger.valueOf(((LongToken) token).token).add(MURMUR3_SCALE);
68 | } else {
69 | throw new RuntimeException("Invalid partitioner Token type. Only BigIntegerToken and LongToken supported");
70 | }
71 | return reducers.get(bigIntToken.divide(rangePerReducer).intValue());
72 | }
73 |
74 | @Override
75 | public void setConf(Configuration conf) {
76 | this.conf = conf;
77 |
78 | final String partitionerParam = conf.get(CassandraParams.SCRUB_CASSANDRACLUSTER_PARTITIONER_CONFIG);
79 | logger.info(CassandraParams.SCRUB_CASSANDRACLUSTER_PARTITIONER_CONFIG + ": " + partitionerParam);
80 | if (partitionerParam == null) {
81 | throw new RuntimeException("Didn't get any cassandra partitioner information");
82 | }
83 |
84 | try {
85 | partitioner = (AbstractPartitioner) Class.forName(partitionerParam).newInstance();
86 | } catch (Exception ex) {
87 | throw new RuntimeException("Invalid partitioner class name: " + partitionerParam);
88 | }
89 |
90 | final String rangePerReducerStr = conf.get(CassandraParams.SCRUB_CASSANDRACLUSTER_RANGE_PER_REDUCER_CONFIG);
91 | if (rangePerReducerStr == null) {
92 | throw new RuntimeException("Didn't get cassandra range per reducer");
93 | }
94 |
95 | rangePerReducer = new BigInteger(rangePerReducerStr);
96 |
97 | final String reducersStr = conf.get(CassandraParams.SCRUB_CASSANDRACLUSTER_REDUCERS_CONFIG);
98 | if (reducersStr == null) {
99 | throw new RuntimeException("Failed to get list of reducers");
100 | }
101 |
102 | final String[] parts = StringUtils.splitByWholeSeparatorPreserveAllTokens(reducersStr, ",");
103 | if ((parts == null) || (parts.length == 0)) {
104 | throw new RuntimeException("Didn't get any valid list of reducers");
105 | }
106 |
107 | reducers = new ArrayList<>(parts.length);
108 | for (String part : parts) {
109 | reducers.add(Integer.parseInt(part));
110 | }
111 |
112 | distributeRandomly = conf.getBoolean(CassandraParams.SCRUB_CASSANDRACLUSTER_DISTRIBUTE_RANDOMLY_CONFIG, false);
113 | if (distributeRandomly) {
114 | random = new Random();
115 | }
116 |
117 | logger.info("CP: range per reducer: {}, reducers: {}, distribute randomly: {}",
118 | new Object[]{rangePerReducerStr,
119 | Arrays.toString(reducers.toArray()),
120 | distributeRandomly});
121 | }
122 |
123 | @Override
124 | public Configuration getConf() {
125 | return conf;
126 | }
127 | }
128 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/cassandra/utils/CassandraRecordUtils.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.cassandra.utils;
17 |
18 | import com.google.common.collect.Lists;
19 |
20 | import com.google.common.collect.Maps;
21 | import com.google.common.collect.Sets;
22 | import org.apache.avro.Schema;
23 | import org.apache.avro.generic.GenericData;
24 | import org.apache.avro.specific.SpecificRecord;
25 | import org.apache.avro.util.Utf8;
26 | import org.apache.cassandra.db.marshal.CompositeType;
27 | import org.apache.cassandra.serializers.BooleanSerializer;
28 | import org.apache.cassandra.serializers.DecimalSerializer;
29 | import org.apache.cassandra.serializers.DoubleSerializer;
30 | import org.apache.cassandra.serializers.FloatSerializer;
31 | import org.apache.cassandra.serializers.InetAddressSerializer;
32 | import org.apache.cassandra.serializers.Int32Serializer;
33 | import org.apache.cassandra.serializers.IntegerSerializer;
34 | import org.apache.cassandra.serializers.ListSerializer;
35 | import org.apache.cassandra.serializers.LongSerializer;
36 | import org.apache.cassandra.serializers.MapSerializer;
37 | import org.apache.cassandra.serializers.SetSerializer;
38 | import org.apache.cassandra.serializers.TimestampSerializer;
39 | import org.apache.cassandra.serializers.TypeSerializer;
40 | import org.apache.cassandra.serializers.UTF8Serializer;
41 | import org.apache.cassandra.serializers.UUIDSerializer;
42 | import org.apache.cassandra.thrift.Column;
43 | import org.apache.cassandra.thrift.ColumnOrSuperColumn;
44 | import org.apache.cassandra.thrift.Mutation;
45 | import org.apache.cassandra.utils.ByteBufferUtil;
46 | import org.apache.crunch.CrunchRuntimeException;
47 | import org.joda.time.DateTimeUtils;
48 |
49 | import java.io.Serializable;
50 | import java.math.BigDecimal;
51 | import java.math.BigInteger;
52 | import java.net.InetAddress;
53 | import java.nio.ByteBuffer;
54 | import java.util.Date;
55 | import java.util.HashMap;
56 | import java.util.List;
57 | import java.util.Map;
58 | import java.util.Set;
59 | import java.util.UUID;
60 |
61 | public final class CassandraRecordUtils implements Serializable {
62 |
63 | private static final Map, TypeSerializer>> serializers;
64 | static {
65 | serializers = new HashMap<>();
66 | serializers.put(BigInteger.class, IntegerSerializer.instance);
67 | serializers.put(Boolean.class, BooleanSerializer.instance);
68 | serializers.put(BigDecimal.class, DecimalSerializer.instance);
69 | serializers.put(Date.class, TimestampSerializer.instance);
70 | serializers.put(Double.class, DoubleSerializer.instance);
71 | serializers.put(Float.class, FloatSerializer.instance);
72 | serializers.put(InetAddress.class, InetAddressSerializer.instance);
73 | serializers.put(Integer.class, Int32Serializer.instance);
74 | serializers.put(Long.class, LongSerializer.instance);
75 | serializers.put(String.class, UTF8Serializer.instance);
76 | serializers.put(UUID.class, UUIDSerializer.instance);
77 | // serializers.put(Utf8.class, UTF8Serializer.instance);
78 | }
79 |
80 | public static ByteBuffer toByteBuffer(final Object value) {
81 | if (value == null) {
82 | return ByteBufferUtil.EMPTY_BYTE_BUFFER;
83 | } else if (value instanceof CharSequence) {
84 | return ByteBufferUtil.bytes(value.toString());
85 | } else if (value instanceof Double) {
86 | return ByteBufferUtil.bytes((Double) value);
87 | } else if (value instanceof Float) {
88 | return ByteBufferUtil.bytes((Float) value);
89 | } else if (value instanceof Integer) {
90 | return ByteBufferUtil.bytes((Integer) value);
91 | } else if (value instanceof Long) {
92 | return ByteBufferUtil.bytes((Long) value);
93 | } else if (value instanceof ByteBuffer) {
94 | return ByteBufferUtil.clone((ByteBuffer) value);
95 | } else if (value instanceof GenericData.Array) {
96 | return serializeList((GenericData.Array)value);
97 | } else if (value instanceof SpecificRecord) {
98 | List buffers = Lists.newArrayList();
99 | SpecificRecord record = (SpecificRecord) value;
100 | for (Schema.Field field : record.getSchema().getFields()) {
101 | buffers.add(toByteBuffer(record.get(field.pos())));
102 | }
103 | return CompositeType.build(buffers.toArray(new ByteBuffer[0]));
104 | } else if (value instanceof Map) {
105 | return serializeMap((Map, ?>) value);
106 | } else if (value instanceof Set) {
107 | return serializeSet((Set>) value);
108 | } else if (value instanceof List) {
109 | return serializeList((List>) value);
110 | } else if (value instanceof UUID) {
111 | return ByteBufferUtil.bytes((UUID) value);
112 | }
113 |
114 |
115 | throw new CrunchRuntimeException("Can not transform field (class: " + value.getClass() + ") to ByteBuffer");
116 | }
117 |
118 | /**
119 | * Serialize a map using Cassandra's map serializer.
120 | * Avro's Utf8 can't be cast to String and needs to be converted manually. This applies to both
121 | * List and Set.
122 | */
123 | private static ByteBuffer serializeMap(Map, ?> map) {
124 | TypeSerializer keySerializer = null;
125 | TypeSerializer valueSerializer = null;
126 | // no need to pass a serializer for elements if the collection is empty
127 | if (!map.isEmpty()) {
128 | // need to derive the type of the keys and values of the map
129 | Map.Entry, ?> firstEntry = map.entrySet().iterator().next();
130 | if (firstEntry.getKey() instanceof Utf8) {
131 | return serializeMap(updateKeysToString(map));
132 | }
133 | if (firstEntry.getValue() instanceof Utf8) {
134 | return serializeMap(updateValuesToString(map));
135 | }
136 | Class> keyType = firstEntry.getKey().getClass();
137 | Class> valueType = firstEntry.getValue().getClass();
138 | keySerializer = getSerializer(Map.class, keyType);
139 | valueSerializer = getSerializer(Map.class, valueType);
140 | }
141 | return MapSerializer.getInstance(keySerializer, valueSerializer).serialize(map);
142 | }
143 |
144 | /**
145 | * Serialize a list using Cassandra's list serializer.
146 | */
147 | private static ByteBuffer serializeList(List> list) {
148 | TypeSerializer elementSerializer = null;
149 | if (!list.isEmpty()) {
150 | Object first = list.iterator().next();
151 | if (first instanceof Utf8) {
152 | return serializeList(toIterableOfStrings(list));
153 | }
154 | elementSerializer = getSerializer(List.class, first.getClass());
155 | }
156 | return ListSerializer.getInstance(elementSerializer).serialize(list);
157 | }
158 |
159 | /**
160 | * Serialize a set using Cassandra's set serializer.
161 | */
162 | private static ByteBuffer serializeSet(Set> set) {
163 | TypeSerializer elementSerializer = null;
164 | if (!set.isEmpty()) {
165 | Object first = set.iterator().next();
166 | if (first instanceof Utf8) {
167 | return serializeSet(Sets.newLinkedHashSet(toIterableOfStrings(set)));
168 | }
169 | elementSerializer = getSerializer(Set.class, first.getClass());
170 | }
171 | return SetSerializer.getInstance(elementSerializer).serialize(set);
172 | }
173 |
174 | /**
175 | * Calls .toString() on each element in the iterable
176 | * @return new list with Strings in it
177 | */
178 | private static List> toIterableOfStrings(Iterable> list) {
179 | List newList = Lists.newArrayList();
180 | for (Object o : list) {
181 | newList.add(o.toString());
182 | }
183 | return newList;
184 | }
185 |
186 | private static Map, ?> updateKeysToString(Map, ?> oldMap) {
187 | Map newMap = Maps.newLinkedHashMap();
188 | for (Object oldKey : oldMap.keySet()) {
189 | newMap.put(oldKey.toString(), oldMap.get(oldKey));
190 | }
191 | return newMap;
192 | }
193 |
194 | private static Map, ?> updateValuesToString(Map, ?> oldMap) {
195 | Map newMap = Maps.newLinkedHashMap();
196 | for (Object oldKey : oldMap.keySet()) {
197 | newMap.put(oldKey, oldMap.get(oldKey).toString());
198 | }
199 | return newMap;
200 | }
201 |
202 | private static TypeSerializer getSerializer(Class> collectionType, Class> clazz) {
203 | if (!serializers.containsKey(clazz)) {
204 | throw new CrunchRuntimeException(
205 | "Can not transform " + collectionType + " with element types of " + clazz
206 | + " to ByteBuffer");
207 | }
208 | return serializers.get(clazz);
209 | }
210 |
211 | public static Mutation createMutation(final Object name, final Object value) {
212 | return createMutation(name, value, DateTimeUtils.currentTimeMillis(), 0);
213 | }
214 |
215 | public static Mutation createMutation(Object name, Object value, long timestamp, int ttl) {
216 | Column column = new Column();
217 | column.setName(toByteBuffer(name));
218 | column.setValue(toByteBuffer(value));
219 | column.setTimestamp(timestamp);
220 | if (ttl > 0) {
221 | column.setTtl(ttl);
222 | }
223 |
224 | Mutation mutation = new Mutation();
225 | mutation.column_or_supercolumn = new ColumnOrSuperColumn();
226 | mutation.column_or_supercolumn.column = column;
227 | return mutation;
228 | }
229 |
230 | public static ByteBuffer getPartitionKey(final List values,
231 | final int[] keyIndexes) {
232 | if (keyIndexes.length == 1) {
233 | return values.get(keyIndexes[0]);
234 | } else {
235 | final ByteBuffer[] components = new ByteBuffer[keyIndexes.length];
236 | for (int i = 0; i < components.length; i++) {
237 | components[i] = values.get(keyIndexes[i]);
238 | }
239 | return compose(components);
240 | }
241 | }
242 |
243 | /**
244 | * Serialize a composite key.
245 | */
246 | private static ByteBuffer compose(final ByteBuffer[] buffers) {
247 | int totalLength = 0;
248 | for (final ByteBuffer bb : buffers)
249 | totalLength += 2 + bb.remaining() + 1;
250 |
251 | final ByteBuffer out = ByteBuffer.allocate(totalLength);
252 | for (final ByteBuffer buffer : buffers)
253 | {
254 | final ByteBuffer bb = buffer.duplicate();
255 | putShortLength(out, bb.remaining());
256 | out.put(bb);
257 | out.put((byte) 0);
258 | }
259 | out.flip();
260 | return out;
261 | }
262 |
263 | private static void putShortLength(final ByteBuffer bb, final int length) {
264 | bb.put((byte) ((length >> 8) & 0xFF));
265 | bb.put((byte) (length & 0xFF));
266 | }
267 |
268 | private CassandraRecordUtils() {
269 | }
270 |
271 | }
272 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/crunch/CrunchConfigHelper.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.crunch;
17 |
18 | import org.apache.cassandra.hadoop.ConfigHelper;
19 | import org.apache.hadoop.conf.Configuration;
20 |
21 | public class CrunchConfigHelper {
22 | public static final String COLUMN_FAMILY_CONFIG = "spotify.cassandra.column.family";
23 |
24 | /**
25 | * Set the column family for the output of this job.
26 | *
27 | * Use this instead of
28 | * {@link org.apache.cassandra.hadoop.ConfigHelper#setOutputColumnFamily(org.apache.hadoop.conf.Configuration, String)}
29 | *
30 | */
31 | public static void setOutputColumnFamily(Configuration conf, String columnFamily) {
32 | conf.set(COLUMN_FAMILY_CONFIG, columnFamily);
33 | }
34 |
35 | /**
36 | * Set the keyspace and column family for the output of this job.
37 | *
38 | * Use this instead of
39 | * {@link org.apache.cassandra.hadoop.ConfigHelper#setOutputColumnFamily(org.apache.hadoop.conf.Configuration, String, String)}
40 | *
41 | */
42 | public static void setOutputColumnFamily(Configuration conf, String keyspace, String columnFamily) {
43 | ConfigHelper.setOutputKeyspace(conf, keyspace);
44 | setOutputColumnFamily(conf, columnFamily);
45 | }
46 |
47 | public static String getOutputColumnFamily(final Configuration conf) {
48 | return conf.get(COLUMN_FAMILY_CONFIG);
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/crunch/cql/CQLConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.crunch.cql;
17 |
18 | import org.apache.crunch.Pair;
19 | import org.apache.crunch.types.Converter;
20 |
21 | import java.nio.ByteBuffer;
22 |
23 | public class CQLConverter implements Converter, Pair>> {
24 |
25 | @Override
26 | public Pair convertInput(final ByteBuffer k, final CQLRecord v) {
27 | return Pair.of(k, v);
28 | }
29 |
30 | @Override
31 | public Pair> convertIterableInput(
32 | final ByteBuffer k,
33 | final Iterable v) {
34 | return Pair.of(k, v);
35 | }
36 |
37 | @Override
38 | public ByteBuffer outputKey(final Pair value) {
39 | return value.first();
40 | }
41 |
42 | @Override
43 | public CQLRecord outputValue(final Pair value) {
44 | return value.second();
45 | }
46 |
47 | @Override
48 | public Class getKeyClass() {
49 | return ByteBuffer.class;
50 | }
51 |
52 | @Override
53 | public Class getValueClass() {
54 | return CQLRecord.class;
55 | }
56 |
57 | @Override
58 | public boolean applyPTypeTransforms() {
59 | return false;
60 | }
61 | }
62 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/crunch/cql/CQLRecord.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.crunch.cql;
17 |
18 | import com.google.common.collect.Lists;
19 | import com.spotify.hdfs2cass.cassandra.cql.CrunchCqlBulkOutputFormat;
20 | import com.spotify.hdfs2cass.cassandra.utils.CassandraRecordUtils;
21 | import com.spotify.hdfs2cass.crunch.CrunchConfigHelper;
22 | import org.apache.avro.generic.IndexedRecord;
23 | import org.apache.cassandra.utils.Hex;
24 | import org.apache.crunch.types.PType;
25 | import org.apache.crunch.types.avro.Avros;
26 | import org.apache.hadoop.conf.Configuration;
27 | import org.joda.time.DateTimeUtils;
28 |
29 | import java.io.Serializable;
30 | import java.nio.ByteBuffer;
31 | import java.util.List;
32 | import java.util.Map;
33 |
34 | /**
35 | * Data structure used when importing hdfs2cass to Cassandra column families with schema.
36 | * These are column families that have been created using CQL.
37 | *
38 | *
39 | * A CQLRecord consists of a List of values. The values are passed to
40 | * {@link org.apache.cassandra.hadoop.cql3.CqlBulkOutputFormat}. They are used as instances of
41 | * parameters to CqlBulkOutputFormat's prepared statement. hdfs2cass can figure the prepared
42 | * statement out automatically in {@link com.spotify.hdfs2cass.cassandra.utils.CassandraParams}, but
43 | * it's possible to shuffle the order of columns in the prepared statement around using the
44 | * 'columnnames' parameter in the Cassandra target URI.
45 | *
46 | * However, order of values must match the order of column names in the prepared statement.
47 | *
48 | */
49 | public class CQLRecord implements Serializable {
50 | public static PType PTYPE = Avros.reflects(CQLRecord.class);
51 |
52 | private final List values;
53 |
54 | /**
55 | * Constructor for Avro reflection-based serialization.
56 | */
57 | public CQLRecord() {
58 | this(Lists.newArrayList());
59 | }
60 |
61 | /**
62 | * @param values List of column values
63 | */
64 | private CQLRecord(final List values) {
65 | this.values = values;
66 | }
67 |
68 | public static CQLRecord create(final Configuration conf, final Map valueMap) {
69 | return create(conf, DateTimeUtils.currentTimeMillis(), 0, valueMap);
70 | }
71 |
72 | public static CQLRecord create(final Configuration conf, final long timestamp, final int ttl,
73 | final Map valueMap) {
74 | List values = Lists.newArrayList(new Object[valueMap.size()]);
75 | String cfName = CrunchConfigHelper.getOutputColumnFamily(conf);
76 | for (Map.Entry valueMapEntry : valueMap.entrySet()) {
77 | int columnIndex = CrunchCqlBulkOutputFormat.getColumnIndex(conf, cfName, valueMapEntry.getKey());
78 | values.set(columnIndex, valueMapEntry.getValue());
79 | }
80 | return create(timestamp, ttl, values);
81 | }
82 |
83 | public static CQLRecord create(final long timestamp, final List> values) {
84 | return create(timestamp, 0, values);
85 | }
86 |
87 | public static CQLRecord create(final long timestamp, final int ttl, final List> values) {
88 | List list = Lists.newArrayList();
89 | for (Object value : values) {
90 | list.add(CassandraRecordUtils.toByteBuffer(value));
91 | }
92 | list.add(CassandraRecordUtils.toByteBuffer(timestamp));
93 | list.add(CassandraRecordUtils.toByteBuffer(ttl));
94 | return new CQLRecord(list);
95 | }
96 |
97 | /**
98 | * @deprecated Use the overload without the {@code key} argument
99 | */
100 | @Deprecated
101 | public static CQLRecord create(final Configuration conf, final Object rowKey,
102 | final long timestamp, final int ttl,
103 | final Map valueMap) {
104 | return create(conf, timestamp, ttl, valueMap);
105 | }
106 |
107 | /**
108 | * @deprecated Use the overload without the {@code key} argument
109 | */
110 | @Deprecated
111 | public static CQLRecord create(final Configuration conf, final Object rowKey,
112 | final Map valueMap) {
113 | return create(conf, valueMap);
114 | }
115 |
116 | /**
117 | * @deprecated Use the overload without the {@code key} argument
118 | */
119 | @Deprecated
120 | public static CQLRecord create(final Configuration conf, final Object rowKey,
121 | final long timestamp, final Map valueMap) {
122 | return CQLRecord.create(conf, timestamp, 0, valueMap);
123 | }
124 |
125 | /**
126 | * @deprecated Use the overload without the {@code key} argument
127 | */
128 | @Deprecated
129 | public static CQLRecord create(final Object key, final List> values) {
130 | return create(DateTimeUtils.currentTimeMillis(), values);
131 | }
132 |
133 | /**
134 | * @deprecated Use the overload without the {@code key} argument
135 | */
136 | @Deprecated
137 | public static CQLRecord create(final Object key, final long timestamp, final List> values) {
138 | return create(timestamp, 0, values);
139 | }
140 |
141 | /**
142 | * @deprecated Use the overload without the {@code key} argument
143 | */
144 | @Deprecated
145 | public static CQLRecord create(final Object key, final long timestamp, final int ttl,
146 | final List> values) {
147 | return create(timestamp, ttl, values);
148 | }
149 |
150 | public static CQLRecord transform(final IndexedRecord record) {
151 | Object key = record.get(0);
152 | List values = Lists.newArrayList();
153 | for (int i = 0; i < record.getSchema().getFields().size(); i++) {
154 | values.add(record.get(i));
155 | }
156 |
157 | return create(key, values);
158 | }
159 |
160 | public List getValues() {
161 | return values;
162 | }
163 |
164 | @Override
165 | public boolean equals(final Object o) {
166 | if (this == o) return true;
167 | if (o == null || getClass() != o.getClass()) return false;
168 |
169 | CQLRecord cqlRecord = (CQLRecord) o;
170 |
171 | if (!values.equals(cqlRecord.values)) return false;
172 |
173 | return true;
174 | }
175 |
176 | @Override
177 | public int hashCode() {
178 | return values.hashCode();
179 | }
180 |
181 | @Override
182 | public String toString() {
183 | StringBuilder valuesAsStrings = new StringBuilder();
184 | valuesAsStrings.append("[");
185 | for (ByteBuffer value : values) {
186 | valuesAsStrings.append(Hex.bytesToHex(value.array()));
187 | valuesAsStrings.append(",");
188 | }
189 | valuesAsStrings.deleteCharAt(valuesAsStrings.length()-1);
190 | valuesAsStrings.append("]");
191 | return String.format("CQLRecord(key=%s, values=%s", valuesAsStrings);
192 | }
193 | }
194 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/crunch/cql/CQLTarget.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.crunch.cql;
17 |
18 | import com.google.common.collect.Maps;
19 | import com.spotify.hdfs2cass.cassandra.cql.CrunchCqlBulkOutputFormat;
20 | import com.spotify.hdfs2cass.cassandra.utils.CassandraParams;
21 | import com.spotify.hdfs2cass.crunch.CrunchConfigHelper;
22 | import org.apache.crunch.CrunchRuntimeException;
23 | import org.apache.crunch.SourceTarget;
24 | import org.apache.crunch.Target;
25 | import org.apache.crunch.io.CrunchOutputs;
26 | import org.apache.crunch.io.FormatBundle;
27 | import org.apache.crunch.io.MapReduceTarget;
28 | import org.apache.crunch.io.OutputHandler;
29 | import org.apache.crunch.types.Converter;
30 | import org.apache.crunch.types.PTableType;
31 | import org.apache.crunch.types.PType;
32 | import org.apache.hadoop.conf.Configuration;
33 | import org.apache.hadoop.fs.Path;
34 | import org.apache.hadoop.mapred.JobConf;
35 | import org.apache.hadoop.mapreduce.Job;
36 | import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
37 |
38 | import java.io.Serializable;
39 | import java.net.URI;
40 | import java.nio.ByteBuffer;
41 | import java.util.List;
42 | import java.util.Map;
43 |
44 | /**
45 | * Responsible for configuring the MapReduce job to use CQL version of bulk output format.
46 | */
47 | public class CQLTarget implements MapReduceTarget, Serializable {
48 | private Map extraConf = Maps.newHashMap();
49 |
50 | private final URI resource;
51 | private final CassandraParams params;
52 |
53 | public CQLTarget(final URI resource, final CassandraParams params) {
54 | this.resource = resource;
55 | this.params = params;
56 | }
57 |
58 | @Override
59 | public void configureForMapReduce(final Job job, final PType> pType, final Path outputPath, final String name) {
60 |
61 | if (name == null) {
62 | throw new CrunchRuntimeException("'name' arguments should not be null. We don't know why tho");
63 | }
64 |
65 | FileOutputFormat.setOutputPath(job, outputPath);
66 | job.setOutputFormatClass(CrunchCqlBulkOutputFormat.class);
67 |
68 | JobConf conf = new JobConf();
69 | params.configure(conf);
70 |
71 | for (Map.Entry e : extraConf.entrySet()) {
72 | conf.set(e.getKey(), e.getValue());
73 | }
74 |
75 | FormatBundle bundle = FormatBundle.forOutput(CrunchCqlBulkOutputFormat.class);
76 | for (Map.Entry e : conf) {
77 | bundle.set(e.getKey(), e.getValue());
78 | }
79 |
80 | Configuration jobConfiguration = job.getConfiguration();
81 |
82 | // we don't know why exactly this is needed, but without this, the actual streaming will not
83 | // see the the throttling and buffer size arguments
84 | params.configure(jobConfiguration);
85 |
86 | CrunchConfigHelper.setOutputColumnFamily(jobConfiguration, params.getKeyspace(),
87 | params.getColumnFamily());
88 | CrunchCqlBulkOutputFormat.setColumnFamilySchema(jobConfiguration, params.getColumnFamily(),
89 | params.getSchema());
90 | CrunchCqlBulkOutputFormat.setColumnFamilyInsertStatement(jobConfiguration,
91 | params.getColumnFamily(), params.getStatement());
92 |
93 | String[] colNames = params.getColumnNames();
94 | for(int i=0; i< colNames.length; i++) {
95 | CrunchCqlBulkOutputFormat.setColumnIndex(jobConfiguration, params.getColumnFamily(), colNames[i], i);
96 | }
97 |
98 | CrunchOutputs.addNamedOutput(job, name, bundle, ByteBuffer.class, List.class);
99 | }
100 |
101 | @Override
102 | public Target outputConf(final String key, final String value) {
103 | extraConf.put(key, value);
104 | return this;
105 | }
106 |
107 | @Override
108 | public boolean handleExisting(final WriteMode writeMode, final long lastModifiedAt, final Configuration conf) {
109 | return false;
110 | }
111 |
112 | @Override
113 | public boolean accept(final OutputHandler handler, final PType> pType) {
114 | if (pType instanceof PTableType) {
115 | final PTableType, ?> pTableType = (PTableType, ?>) pType;
116 | PType> keyType = pTableType.getKeyType();
117 | PType> valueType = pTableType.getValueType();
118 | if (ByteBuffer.class.equals(keyType.getTypeClass())
119 | && CQLRecord.class.equals(valueType.getTypeClass())) {
120 | handler.configure(this, pType);
121 | return true;
122 | }
123 | }
124 | return false;
125 | }
126 |
127 | @Override
128 | public Converter, ?, ?, ?> getConverter(final PType> pType) {
129 | return new CQLConverter();
130 | }
131 |
132 | @Override
133 | public SourceTarget asSourceTarget(final PType pType) {
134 | return null;
135 | }
136 |
137 | @Override
138 | public String toString() {
139 | return resource.toString();
140 | }
141 | }
142 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/crunch/thrift/ThriftConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.crunch.thrift;
17 |
18 | import org.apache.cassandra.thrift.Mutation;
19 | import org.apache.crunch.Pair;
20 | import org.apache.crunch.types.Converter;
21 |
22 | import java.nio.ByteBuffer;
23 | import java.util.Collection;
24 |
25 | public class ThriftConverter implements Converter, Pair>, Pair>>> {
26 | @Override
27 | public Pair> convertInput(final ByteBuffer key, final Collection value) {
28 | return Pair.of(key, value);
29 | }
30 |
31 | @Override
32 | public Pair>> convertIterableInput(final ByteBuffer key, final Iterable> value) {
33 | return Pair.of(key, value);
34 | }
35 |
36 | @Override
37 | public ByteBuffer outputKey(final Pair> value) {
38 | return value.first();
39 | }
40 |
41 | @Override
42 | public Collection outputValue(final Pair> value) {
43 | return value.second();
44 | }
45 |
46 | @Override
47 | public Class getKeyClass() {
48 | return ByteBuffer.class;
49 | }
50 |
51 | @Override
52 | public Class> getValueClass() {
53 | return (Class>) (Class>) Collection.class;
54 | }
55 |
56 | @Override
57 | public boolean applyPTypeTransforms() {
58 | return false;
59 | }
60 | }
61 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/crunch/thrift/ThriftRecord.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.crunch.thrift;
17 |
18 | import com.google.common.collect.Lists;
19 | import org.apache.cassandra.thrift.Mutation;
20 | import org.apache.crunch.MapFn;
21 | import org.apache.crunch.Pair;
22 | import org.apache.crunch.types.PTableType;
23 | import org.apache.crunch.types.PType;
24 | import org.apache.crunch.types.avro.Avros;
25 |
26 | import java.io.Serializable;
27 | import java.nio.ByteBuffer;
28 | import java.util.Collection;
29 | import java.util.List;
30 |
31 | /**
32 | * Data structure used when importing hdfs2cass to schema-less Cassandra column families.
33 | * Schema-less Cassandra column families are the ones that have been created without CQL.
34 | *
35 | * @deprecated Prefer CQL, see {@link com.spotify.hdfs2cass.crunch.cql.CQLRecord}
36 | */
37 | @Deprecated
38 | public class ThriftRecord implements Serializable {
39 | public static PType PTYPE = Avros.reflects(ThriftRecord.class);
40 |
41 | private ByteBuffer key;
42 | private List values;
43 |
44 | public ThriftRecord() {
45 | }
46 |
47 | /**
48 | * A ThriftRecord consists of Cassandra row key and a collection of
49 | * {@link org.apache.cassandra.thrift.Mutation}.
50 | * Mutations are passed to {@link org.apache.cassandra.hadoop.BulkOutputFormat}
51 | * and correspond to column insertions.
52 | * Mutations can be in any order. One row can be split into multiple ThriftRecords, Cassandra
53 | * will eventually handle this.
54 | * Placing 5,000+ mutations in one causes A LOT of memory pressure and should be avoided.
55 | *
56 | * @param key Cassandra row (i.e. partition) key
57 | * @param values List of columns belonging to this row
58 | */
59 | public ThriftRecord(final ByteBuffer key, final List values) {
60 | this.key = key;
61 | this.values = values;
62 | }
63 |
64 | public ByteBuffer getKey() {
65 | return key;
66 | }
67 |
68 | public static ThriftRecord of(final ByteBuffer key, final Mutation... values) {
69 | return of(key, Lists.newArrayList(values));
70 | }
71 |
72 |
73 |
74 | /**
75 | * @param key Cassandra row (i.e. partition) key
76 | * @param values List of columns belonging to this row
77 | * @return
78 | */
79 | public static ThriftRecord of(final ByteBuffer key, final List values) {
80 | return new ThriftRecord(key, values);
81 | }
82 |
83 | public Pair> asPair() {
84 | Collection collection = values;
85 | return Pair.of(key, collection);
86 | }
87 |
88 | public List getValues() {
89 | return values;
90 | }
91 |
92 | public static class AsPair extends MapFn>> {
93 | public static PTableType> PTYPE =
94 | Avros.tableOf(Avros.bytes(), Avros.collections(Avros.records(Mutation.class)));
95 |
96 | @Override
97 | public Pair> map(final ThriftRecord input) {
98 | return input.asPair();
99 | }
100 | }
101 |
102 | @Override
103 | public boolean equals(final Object o) {
104 | if (this == o) return true;
105 | if (o == null || getClass() != o.getClass()) return false;
106 |
107 | ThriftRecord that = (ThriftRecord) o;
108 |
109 | if (!key.equals(that.key)) return false;
110 | if (!values.equals(that.values)) return false;
111 |
112 | return true;
113 | }
114 |
115 | @Override
116 | public int hashCode() {
117 | int result = key.hashCode();
118 | result = 31 * result + values.hashCode();
119 | return result;
120 | }
121 |
122 | @Override
123 | public String toString() {
124 | return "ThriftRecord{" + "key=" + key + ", values=" + values + '}';
125 | }
126 | }
127 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/crunch/thrift/ThriftTarget.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.crunch.thrift;
17 |
18 | import com.google.common.collect.Maps;
19 | import com.spotify.hdfs2cass.cassandra.thrift.CrunchBulkOutputFormat;
20 | import com.spotify.hdfs2cass.crunch.CrunchConfigHelper;
21 | import com.spotify.hdfs2cass.cassandra.utils.CassandraParams;
22 | import org.apache.cassandra.thrift.Mutation;
23 | import org.apache.crunch.CrunchRuntimeException;
24 | import org.apache.crunch.SourceTarget;
25 | import org.apache.crunch.Target;
26 | import org.apache.crunch.io.CrunchOutputs;
27 | import org.apache.crunch.io.FormatBundle;
28 | import org.apache.crunch.io.MapReduceTarget;
29 | import org.apache.crunch.io.OutputHandler;
30 | import org.apache.crunch.types.Converter;
31 | import org.apache.crunch.types.PTableType;
32 | import org.apache.crunch.types.PType;
33 | import org.apache.hadoop.conf.Configuration;
34 | import org.apache.hadoop.fs.Path;
35 | import org.apache.hadoop.mapred.JobConf;
36 | import org.apache.hadoop.mapreduce.Job;
37 | import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
38 |
39 | import java.io.Serializable;
40 | import java.net.URI;
41 | import java.nio.ByteBuffer;
42 | import java.util.Collection;
43 | import java.util.List;
44 | import java.util.Map;
45 |
46 | /**
47 | * Responsible for configuring the MapReduce job to use Thrift version of bulk output format.
48 | */
49 | public class ThriftTarget implements MapReduceTarget, Serializable {
50 | private Map extraConf = Maps.newHashMap();
51 |
52 | private URI resource;
53 | private final CassandraParams params;
54 |
55 | public ThriftTarget(final URI resource, final CassandraParams params) {
56 | this.resource = resource;
57 | this.params = params;
58 | }
59 |
60 | @Override
61 | public void configureForMapReduce(final Job job, final PType> pType, final Path outputPath, final String name) {
62 |
63 | if (name == null) {
64 | throw new CrunchRuntimeException("'name' arguments should not be null. We don't know why tho");
65 | }
66 |
67 | FileOutputFormat.setOutputPath(job, outputPath);
68 | job.setOutputFormatClass(CrunchBulkOutputFormat.class);
69 |
70 | JobConf conf = new JobConf();
71 | params.configure(conf);
72 |
73 | for (Map.Entry e : extraConf.entrySet()) {
74 | conf.set(e.getKey(), e.getValue());
75 | }
76 |
77 | FormatBundle bundle = FormatBundle.forOutput(CrunchBulkOutputFormat.class);
78 | for (Map.Entry e : conf) {
79 | bundle.set(e.getKey(), e.getValue());
80 | }
81 |
82 | Configuration jobConfiguration = job.getConfiguration();
83 |
84 | // we don't know why exactly this is needed, but without this, the actual streaming will not
85 | // see the the throttling and buffer size arguments
86 | params.configure(jobConfiguration);
87 |
88 | CrunchConfigHelper.setOutputColumnFamily(jobConfiguration, params.getKeyspace(),
89 | params.getColumnFamily());
90 |
91 | CrunchOutputs.addNamedOutput(job, name, bundle, ByteBuffer.class, List.class);
92 | }
93 |
94 | @Override
95 | public Target outputConf(final String key, final String value) {
96 | extraConf.put(key, value);
97 | return this;
98 | }
99 |
100 | @Override
101 | public boolean handleExisting(final WriteMode writeMode, final long lastModifiedAt, final Configuration conf) {
102 | return false;
103 | }
104 |
105 | @Override
106 | public boolean accept(final OutputHandler handler, final PType> pType) {
107 | if (pType instanceof PTableType) {
108 | PTableType pTableType = (PTableType) pType;
109 | PType> keyType = pTableType.getKeyType();
110 | PType> valueType = pTableType.getValueType();
111 | List subTypes = valueType.getSubTypes();
112 |
113 | if (ByteBuffer.class.equals(keyType.getTypeClass())
114 | && Collection.class.equals(valueType.getTypeClass())
115 | && subTypes.size() == 1
116 | && Mutation.class.equals(subTypes.get(0).getTypeClass())) {
117 | handler.configure(this, pType);
118 | return true;
119 | }
120 | }
121 | return false;
122 | }
123 |
124 | @Override
125 | public Converter, ?, ?, ?> getConverter(final PType> pType) {
126 | return new ThriftConverter();
127 | }
128 |
129 | @Override
130 | public SourceTarget asSourceTarget(final PType pType) {
131 | return null;
132 | }
133 |
134 | @Override
135 | public String toString() {
136 | return resource.toString();
137 | }
138 | }
139 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/crunch/thrift/converters/Thrift.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.crunch.thrift.converters;
17 |
18 | import com.spotify.hdfs2cass.crunch.thrift.ThriftRecord;
19 | import org.apache.avro.specific.SpecificRecord;
20 | import org.apache.crunch.PCollection;
21 |
22 | public final class Thrift {
23 | public static final String DEFAULT_ROWKEY_FIELD_NAME = "rowkey";
24 | public static final String DEFAULT_TTL_FIELD_NAME = "ttl";
25 | public static final String DEFAULT_TIMESTAMP_FIELD_NAME = "timestamp";
26 |
27 | private Thrift() {
28 | }
29 |
30 | public static PCollection byConvention(final PCollection collection) {
31 | return byFieldNames(collection, DEFAULT_ROWKEY_FIELD_NAME, DEFAULT_TTL_FIELD_NAME, DEFAULT_TIMESTAMP_FIELD_NAME);
32 | }
33 |
34 | public static PCollection byFieldNames(
35 | final PCollection collection,
36 | final String rowKeyFieldName,
37 | final String ttlFieldName,
38 | final String timestampFieldName
39 | ) {
40 | final Class recordType = collection.getPType().getTypeClass();
41 | T record;
42 | try {
43 | record = recordType.getConstructor().newInstance();
44 | } catch (Exception e) {
45 | throw new RuntimeException("Could not create an instance of the record to determine it's schema", e);
46 | }
47 |
48 | ThriftByFieldNamesFn doFn = new ThriftByFieldNamesFn(record.getSchema(), rowKeyFieldName, ttlFieldName, timestampFieldName);
49 | return collection.parallelDo(doFn, ThriftRecord.PTYPE);
50 | }
51 |
52 | }
53 |
--------------------------------------------------------------------------------
/src/main/java/com/spotify/hdfs2cass/crunch/thrift/converters/ThriftByFieldNamesFn.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass.crunch.thrift.converters;
17 |
18 | import com.google.common.base.Optional;
19 | import com.google.common.collect.Lists;
20 | import com.spotify.hdfs2cass.crunch.thrift.ThriftRecord;
21 | import com.spotify.hdfs2cass.cassandra.utils.CassandraRecordUtils;
22 | import org.apache.avro.Schema;
23 | import org.apache.avro.specific.SpecificRecord;
24 | import org.apache.cassandra.thrift.Column;
25 | import org.apache.cassandra.thrift.ColumnOrSuperColumn;
26 | import org.apache.cassandra.thrift.Mutation;
27 | import org.apache.cassandra.utils.ByteBufferUtil;
28 | import org.apache.crunch.CrunchRuntimeException;
29 | import org.apache.crunch.MapFn;
30 | import org.joda.time.DateTimeUtils;
31 | import org.slf4j.Logger;
32 | import org.slf4j.LoggerFactory;
33 |
34 | import java.nio.ByteBuffer;
35 | import java.util.List;
36 |
37 | public class ThriftByFieldNamesFn extends MapFn {
38 | private final Logger logger = LoggerFactory.getLogger(ThriftByFieldNamesFn.class);
39 |
40 | private int rowKeyIndex = -1;
41 | private int ttlIndex = -1;
42 | private int timestampIndex = -1;
43 |
44 | public ThriftByFieldNamesFn() {
45 | }
46 |
47 | public ThriftByFieldNamesFn(final Schema schema, final String rowKeyFieldName, final String ttlFieldName, final String timestampFieldName) {
48 |
49 | Schema.Field rowKeyField = schema.getField(rowKeyFieldName);
50 | if (rowKeyField == null) {
51 | throw new CrunchRuntimeException("Row key field name not found: " + rowKeyFieldName);
52 | }
53 | rowKeyIndex = rowKeyField.pos();
54 |
55 | Schema.Field ttlField = schema.getField(ttlFieldName);
56 | if (ttlField == null) {
57 | logger.info("TTL field not found, TTL will not be set");
58 | } else {
59 | logger.info("Using TTL field name: " + ttlFieldName);
60 | ttlIndex = ttlField.pos();
61 | }
62 |
63 | Schema.Field timestampField = schema.getField(timestampFieldName);
64 | if (timestampField == null) {
65 | logger.info("Timestamp field not found, System.currentTimeMillis() will be used");
66 | } else {
67 | logger.info("Using timestamp field name: " + ttlFieldName);
68 | timestampIndex = timestampField.pos();
69 | }
70 | }
71 |
72 | @Override
73 | public ThriftRecord map(T input) {
74 | ByteBuffer key = getRowKey(input);
75 | List values = getMutations(input);
76 | return ThriftRecord.of(key, values);
77 | }
78 |
79 | private List getMutations(final T input) {
80 | List mutations = Lists.newArrayList();
81 |
82 | long timestamp = getTimestamp(input);
83 | Optional ttl = getTtl(input);
84 |
85 | for (Schema.Field field : input.getSchema().getFields()) {
86 | int fieldPos = field.pos();
87 | if (fieldPos == rowKeyIndex || fieldPos == ttlIndex || fieldPos == timestampIndex) {
88 | continue;
89 | }
90 |
91 | Object fieldValue = input.get(fieldPos);
92 |
93 | Column column = new Column();
94 | column.setName(ByteBufferUtil.bytes(field.name()));
95 | column.setTimestamp(timestamp);
96 | if (ttl.isPresent()) {
97 | column.setTtl(ttl.get());
98 | }
99 | column.setValue(CassandraRecordUtils.toByteBuffer(fieldValue));
100 |
101 | Mutation mutation = new Mutation();
102 | mutation.column_or_supercolumn = new ColumnOrSuperColumn();
103 | mutation.column_or_supercolumn.column = column;
104 |
105 | mutations.add(mutation);
106 | }
107 |
108 |
109 | return mutations;
110 | }
111 |
112 | private Optional getTtl(final T input) {
113 | if (ttlIndex > -1) {
114 | Object value = input.get(timestampIndex);
115 | if (value instanceof Long) {
116 | return Optional.fromNullable(Integer.class.cast(value));
117 | } else {
118 | throw new CrunchRuntimeException("Can not transform ttl field (class: " + value.getClass() + ") to Integer");
119 | }
120 | } else {
121 | return Optional.absent();
122 | }
123 | }
124 |
125 | private long getTimestamp(final T input) {
126 | if (timestampIndex > -1) {
127 | Object value = input.get(timestampIndex);
128 | if (value instanceof Long) {
129 | return (long) value;
130 | } else {
131 | throw new CrunchRuntimeException("Can not transform timestamp field (class: " + value.getClass() + ") to long");
132 | }
133 | } else {
134 | return DateTimeUtils.currentTimeMillis();
135 | }
136 | }
137 |
138 | public ByteBuffer getRowKey(final T input) {
139 | Object value = input.get(rowKeyIndex);
140 | return CassandraRecordUtils.toByteBuffer(value);
141 | }
142 | }
143 |
--------------------------------------------------------------------------------
/src/main/java/org/apache/cassandra/io/sstable/CrunchBulkRecordWriter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | *
18 | * The modifications to the upstream file is Copyright 2014 Spotify AB.
19 | * The original upstream file can be found at
20 | * https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java
21 | */
22 | package org.apache.cassandra.io.sstable;
23 |
24 | import com.google.common.util.concurrent.Uninterruptibles;
25 |
26 | import com.spotify.hdfs2cass.cassandra.thrift.ExternalSSTableLoaderClient;
27 | import com.spotify.hdfs2cass.cassandra.thrift.ProgressHeartbeat;
28 | import com.spotify.hdfs2cass.cassandra.thrift.ProgressIndicator;
29 | import com.spotify.hdfs2cass.crunch.CrunchConfigHelper;
30 |
31 | import org.apache.cassandra.config.Config;
32 | import org.apache.cassandra.config.DatabaseDescriptor;
33 | import org.apache.cassandra.db.marshal.AbstractType;
34 | import org.apache.cassandra.db.marshal.BytesType;
35 | import org.apache.cassandra.hadoop.ConfigHelper;
36 | import org.apache.cassandra.hadoop.HadoopCompat;
37 | import org.apache.cassandra.streaming.StreamState;
38 | import org.apache.cassandra.thrift.Column;
39 | import org.apache.cassandra.thrift.CounterColumn;
40 | import org.apache.cassandra.thrift.Mutation;
41 | import org.apache.cassandra.utils.OutputHandler;
42 | import org.apache.crunch.CrunchRuntimeException;
43 | import org.apache.hadoop.conf.Configuration;
44 | import org.apache.hadoop.mapreduce.RecordWriter;
45 | import org.apache.hadoop.mapreduce.TaskAttemptContext;
46 | import org.slf4j.Logger;
47 | import org.slf4j.LoggerFactory;
48 |
49 | import java.io.File;
50 | import java.io.IOException;
51 | import java.net.InetAddress;
52 | import java.nio.ByteBuffer;
53 | import java.nio.file.Paths;
54 | import java.util.Collections;
55 | import java.util.List;
56 | import java.util.concurrent.ExecutionException;
57 | import java.util.concurrent.Future;
58 |
59 | /**
60 | * This is an almost-copy of {@link org.apache.cassandra.hadoop.BulkRecordWriter}.
61 | *
62 | * We had to re-implement this class because of https://issues.apache.org/jira/browse/CASSANDRA-8367
63 | *
64 | */
65 | public class CrunchBulkRecordWriter
66 | extends RecordWriter> implements
67 | org.apache.hadoop.mapred.RecordWriter> {
68 |
69 | private final static Logger LOG = LoggerFactory.getLogger(CrunchBulkRecordWriter.class);
70 |
71 | private final static String OUTPUT_LOCATION = "mapreduce.output.bulkoutputformat.localdir";
72 | private final static String STREAM_THROTTLE_MBITS = "mapreduce.output.bulkoutputformat.streamthrottlembits";
73 | private final static String MAX_FAILED_HOSTS = "mapreduce.output.bulkoutputformat.maxfailedhosts";
74 |
75 | private final Configuration conf;
76 | private final ProgressHeartbeat heartbeat;
77 | private AbstractSSTableSimpleWriter writer;
78 | private SSTableLoader loader;
79 | private File outputdir;
80 | private TaskAttemptContext context;
81 |
82 | private enum CFType {
83 | NORMAL, SUPER
84 | }
85 |
86 | private enum ColType {
87 | NORMAL, COUNTER
88 | }
89 |
90 | private CFType cfType;
91 | private ColType colType;
92 |
93 | public CrunchBulkRecordWriter(TaskAttemptContext context) {
94 | Config.setClientMode(true);
95 | Config.setOutboundBindAny(true);
96 | this.conf = HadoopCompat.getConfiguration(context);
97 | this.context = context;
98 | int megabitsPerSec = Integer.parseInt(conf.get(STREAM_THROTTLE_MBITS, "0"));
99 | LOG.info("Setting stream throttling to " + megabitsPerSec);
100 | DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(megabitsPerSec);
101 | DatabaseDescriptor.setInterDCStreamThroughputOutboundMegabitsPerSec(megabitsPerSec);
102 | heartbeat = new ProgressHeartbeat(context, 120);
103 | }
104 |
105 | private String getOutputLocation() {
106 | String dir = conf.get(OUTPUT_LOCATION, System.getProperty("java.io.tmpdir"));
107 | if (dir == null) {
108 | throw new CrunchRuntimeException(
109 | "Output directory not defined, if hadoop is not setting java.io.tmpdir then define "
110 | + OUTPUT_LOCATION);
111 | }
112 | return dir;
113 | }
114 |
115 | private void setTypes(Mutation mutation) {
116 | if (cfType == null) {
117 | if (mutation.getColumn_or_supercolumn().isSetSuper_column()
118 | || mutation.getColumn_or_supercolumn().isSetCounter_super_column())
119 | cfType = CFType.SUPER;
120 | else
121 | cfType = CFType.NORMAL;
122 | if (mutation.getColumn_or_supercolumn().isSetCounter_column()
123 | || mutation.getColumn_or_supercolumn().isSetCounter_super_column())
124 | colType = ColType.COUNTER;
125 | else
126 | colType = ColType.NORMAL;
127 | }
128 | }
129 |
130 | private void prepareWriter() {
131 | String columnFamily = CrunchConfigHelper.getOutputColumnFamily(conf);
132 | String keyspace = ConfigHelper.getOutputKeyspace(conf);
133 |
134 | if (outputdir == null) {
135 | // dir must be named by ks/cf for the loader
136 | outputdir = Paths.get(getOutputLocation(), keyspace, columnFamily).toFile();
137 | outputdir.mkdirs();
138 | }
139 |
140 | if (writer == null) {
141 | AbstractType> subcomparator = null;
142 |
143 | if (cfType == CFType.SUPER)
144 | subcomparator = BytesType.instance;
145 |
146 | this.writer = new SSTableSimpleWriter(
147 | outputdir, ConfigHelper.getOutputPartitioner(conf),
148 | keyspace, columnFamily,
149 | BytesType.instance, subcomparator);
150 |
151 | ExternalSSTableLoaderClient externalClient = new ExternalSSTableLoaderClient(
152 | ConfigHelper.getOutputInitialAddress(conf),
153 | ConfigHelper.getOutputRpcPort(conf),
154 | ConfigHelper.getOutputKeyspaceUserName(conf),
155 | ConfigHelper.getOutputKeyspacePassword(conf));
156 |
157 | this.loader = new SSTableLoader(outputdir, externalClient,
158 | new OutputHandler.SystemOutput(true, true));
159 | }
160 | }
161 |
162 | @Override
163 | public void write(ByteBuffer keybuff, List value) throws IOException {
164 | ProgressHeartbeat heartbeat = new ProgressHeartbeat(context, 120);
165 | heartbeat.startHeartbeat();
166 | try {
167 | setTypes(value.get(0));
168 | prepareWriter();
169 | if (writer.currentKey() == null || !keybuff.equals(writer.currentKey().key)) {
170 | writer.newRow(keybuff);
171 | }
172 | for (Mutation mut : value) {
173 | if (cfType == CFType.SUPER) {
174 | writer.newSuperColumn(mut.getColumn_or_supercolumn().getSuper_column().name);
175 | if (colType == ColType.COUNTER)
176 | for (CounterColumn column : mut.getColumn_or_supercolumn().getCounter_super_column().columns)
177 | writer.addCounterColumn(column.name, column.value);
178 | else {
179 | for (Column column : mut.getColumn_or_supercolumn().getSuper_column().columns) {
180 | if (column.ttl == 0)
181 | writer.addColumn(column.name, column.value, column.timestamp);
182 | else
183 | writer.addExpiringColumn(column.name, column.value, column.timestamp, column.ttl,
184 | System.currentTimeMillis() + ((long) column.ttl * 1000));
185 | }
186 | }
187 | } else {
188 | if (colType == ColType.COUNTER) {
189 | writer.addCounterColumn(mut.getColumn_or_supercolumn().counter_column.name,
190 | mut.getColumn_or_supercolumn().counter_column.value);
191 | } else {
192 | if (mut.getColumn_or_supercolumn().column.ttl == 0) {
193 | writer.addColumn(mut.getColumn_or_supercolumn().column.name,
194 | mut.getColumn_or_supercolumn().column.value,
195 | mut.getColumn_or_supercolumn().column.timestamp);
196 | } else {
197 | writer.addExpiringColumn(mut.getColumn_or_supercolumn().column.name,
198 | mut.getColumn_or_supercolumn().column.value,
199 | mut.getColumn_or_supercolumn().column.timestamp,
200 | mut.getColumn_or_supercolumn().column.ttl, System.currentTimeMillis()
201 | + ((long) (mut.getColumn_or_supercolumn().column.ttl) * 1000));
202 | }
203 | }
204 | }
205 | }
206 | } finally {
207 | heartbeat.stopHeartbeat();
208 | }
209 | }
210 |
211 | @Override
212 | public void close(TaskAttemptContext context) throws IOException, InterruptedException {
213 | close();
214 | }
215 |
216 | /**
217 | * Fills the deprecated RecordWriter interface for streaming.
218 | */
219 | @Deprecated
220 | public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException {
221 | close();
222 | }
223 |
224 | private void close() throws IOException {
225 | LOG.info("SSTables built. Now starting streaming");
226 | heartbeat.startHeartbeat();
227 | try {
228 | if (writer != null) {
229 | writer.close();
230 | Future future =
231 | loader.stream(Collections.emptySet(), new ProgressIndicator());
232 | try {
233 | StreamState streamState = Uninterruptibles.getUninterruptibly(future);
234 | if (streamState.hasFailedSession()) {
235 | LOG.warn("Some streaming sessions failed");
236 | } else {
237 | LOG.info("Streaming finished successfully");
238 | }
239 | } catch (ExecutionException e) {
240 | throw new RuntimeException("Streaming to the following hosts failed: " +
241 | loader.getFailedHosts(), e);
242 | }
243 | } else {
244 | LOG.info("SSTableWriter wasn't instantiated, no streaming happened.");
245 | }
246 | } finally {
247 | heartbeat.stopHeartbeat();
248 | }
249 | LOG.info("Successfully closed bulk record writer");
250 | }
251 | }
252 |
--------------------------------------------------------------------------------
/src/test/java/com/spotify/hdfs2cass/LegacyInputFormatTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Spotify AB. All rights reserved.
3 | *
4 | * The contents of this file are licensed under the Apache License, Version
5 | * 2.0 (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
12 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
13 | * License for the specific language governing permissions and limitations
14 | * under the License.
15 | */
16 | package com.spotify.hdfs2cass;
17 |
18 | import org.apache.crunch.CrunchRuntimeException;
19 | import org.joda.time.DateTimeUtils;
20 | import org.junit.Test;
21 |
22 | import static org.junit.Assert.*;
23 |
24 | public class LegacyInputFormatTest {
25 |
26 | @Test
27 | public void testParseValid() throws Exception {
28 |
29 | DateTimeUtils.setCurrentMillisFixed(42l);
30 |
31 | String v1 = "HdfsToCassandra\t1\tkey\tcolName\tvalue";
32 | LegacyInputFormat r1 = LegacyInputFormat.parse(v1);
33 | assertEquals("key", r1.getRowkey());
34 | assertEquals("colName", r1.getColname());
35 | assertEquals("value", r1.getColval());
36 | assertEquals(42l, r1.getTimestamp());
37 | assertEquals(0, r1.getTtl());
38 |
39 | String v2 = "HdfsToCassandra\t2\tkey\tcolName\t23\tvalue";
40 | r1 = LegacyInputFormat.parse(v2);
41 | assertEquals("key", r1.getRowkey());
42 | assertEquals("colName", r1.getColname());
43 | assertEquals("value", r1.getColval());
44 | assertEquals(23l, r1.getTimestamp());
45 | assertEquals(0, r1.getTtl());
46 |
47 | String v3 = "HdfsToCassandra\t3\tkey\tcolName\t23\t666\tvalue";
48 | r1 = LegacyInputFormat.parse(v3);
49 | assertEquals("key", r1.getRowkey());
50 | assertEquals("colName", r1.getColname());
51 | assertEquals("value", r1.getColval());
52 | assertEquals(23l, r1.getTimestamp());
53 | assertEquals(666, r1.getTtl());
54 | }
55 |
56 | @Test(expected = CrunchRuntimeException.class)
57 | public void testParseInvalidTooFew() {
58 | String v1 = "HdfsToCassandra\t1\tkey\tcolName";
59 | LegacyInputFormat.parse(v1);
60 | }
61 |
62 | @Test(expected = CrunchRuntimeException.class)
63 | public void testParseInvalidTooManyV1() {
64 | String v1 = "HdfsToCassandra\t1\tkey\tcolName\tvalue\tfoo";
65 | LegacyInputFormat.parse(v1);
66 | }
67 |
68 | @Test(expected = CrunchRuntimeException.class)
69 | public void testParseInvalidTooManyV2() {
70 | String v1 = "HdfsToCassandra\t2\tkey\tcolName\t23\tvalue\tfoo";
71 | LegacyInputFormat.parse(v1);
72 | }
73 |
74 | @Test(expected = CrunchRuntimeException.class)
75 | public void testParseInvalidTooManyV3() {
76 | String v1 = "HdfsToCassandra\t3\tkey\tcolName\t23\t666\tvalue\tfoo";
77 | LegacyInputFormat.parse(v1);
78 | }
79 |
80 | @Test(expected = CrunchRuntimeException.class)
81 | public void testParseInvalidNumberFormat() {
82 | String v1 = "HdfsToCassandra\t3\tkey\tcolName\t2a3\t666\tvalue";
83 | LegacyInputFormat.parse(v1);
84 | }
85 |
86 | }
87 |
--------------------------------------------------------------------------------
/src/test/java/com/spotify/hdfs2cass/cassandra/utils/CassandraKeyComparatorTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | /*
3 | * Copyright 2016 Spotify AB. All rights reserved.
4 | *
5 | * The contents of this file are licensed under the Apache License, Version
6 | * 2.0 (the "License"); you may not use this file except in compliance with
7 | * the License. You may obtain a copy of the License at
8 | *
9 | * http://www.apache.org/licenses/LICENSE-2.0
10 | *
11 | * Unless required by applicable law or agreed to in writing, software
12 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
13 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
14 | * License for the specific language governing permissions and limitations
15 | * under the License.
16 | */
17 |
18 | package com.spotify.hdfs2cass.cassandra.utils;
19 |
20 | import static org.hamcrest.Matchers.greaterThan;
21 | import static org.hamcrest.Matchers.is;
22 | import static org.hamcrest.Matchers.lessThan;
23 | import static org.hamcrest.junit.MatcherAssert.assertThat;
24 |
25 | import org.apache.avro.io.BinaryEncoder;
26 | import org.apache.avro.io.EncoderFactory;
27 | import org.apache.cassandra.dht.Murmur3Partitioner;
28 | import org.apache.cassandra.dht.OrderPreservingPartitioner;
29 | import org.apache.hadoop.conf.Configuration;
30 | import org.junit.Test;
31 |
32 | import java.io.ByteArrayOutputStream;
33 | import java.io.IOException;
34 |
35 | public class CassandraKeyComparatorTest {
36 | private static final EncoderFactory ENCODERS = EncoderFactory.get();
37 |
38 | private final CassandraKeyComparator comparator = new CassandraKeyComparator();
39 | private final Configuration conf = new Configuration();
40 |
41 | @Test
42 | public void compareOrderPreservingPartitioner() throws IOException {
43 | conf.set(CassandraParams.SCRUB_CASSANDRACLUSTER_PARTITIONER_CONFIG,
44 | OrderPreservingPartitioner.class.getName());
45 | comparator.setConf(conf);
46 | checkOrder("abc", "def");
47 | checkOrder("1", "2");
48 | checkOrder("abc1", "abc2");
49 | checkOrder("abc", "abcdef");
50 | }
51 |
52 | @Test
53 | public void compareMurmur3Partitioner() throws IOException {
54 | conf.set(CassandraParams.SCRUB_CASSANDRACLUSTER_PARTITIONER_CONFIG,
55 | Murmur3Partitioner.class.getName());
56 | comparator.setConf(conf);
57 | // murmur3_128("foo")[0] = -2129773440516405919
58 | // murmur3_128("bar")[0] = -7911037993560119804
59 | // murmur3_128("baz")[0] = 8295379539955784970
60 | checkOrder("bar", "foo");
61 | checkOrder("foo", "baz");
62 | checkOrder("bar", "baz");
63 |
64 | // Murmur3Partitioner maps empty string to Long.MIN_VALUE
65 | checkOrder("", "foo");
66 | checkOrder("", "bar");
67 | }
68 |
69 | private void checkOrder(final String key1, final String key2) throws IOException {
70 | final byte[] buf1 = bytes(key1, 0);
71 | final int offset = 3;
72 | final byte[] buf2 = bytes(key2, offset);
73 |
74 | final int l1 = buf1.length;
75 | final int l2 = buf2.length - offset;
76 | assertThat(comparator.compare(buf1, 0, l1, buf2, offset, l2), lessThan(0));
77 | assertThat(comparator.compare(buf2, offset, l2, buf1, 0, l1), greaterThan(0));
78 | assertThat(comparator.compare(buf1, 0, l1, buf1, 0, l1), is(0));
79 | assertThat(comparator.compare(buf2, offset, l2, buf2, offset, l2), is(0));
80 | }
81 |
82 | private static byte[] bytes(final String s, final int offset)
83 | throws IOException {
84 | final ByteArrayOutputStream baos = new ByteArrayOutputStream(32);
85 | baos.write(new byte[offset], 0, offset);
86 | final BinaryEncoder enc = ENCODERS.directBinaryEncoder(baos, null);
87 | enc.writeString(s);
88 | return baos.toByteArray();
89 | }
90 | }
91 |
--------------------------------------------------------------------------------
/src/test/java/com/spotify/hdfs2cass/cassandra/utils/CassandraRecordUtilsTest.java:
--------------------------------------------------------------------------------
1 | package com.spotify.hdfs2cass.cassandra.utils;
2 |
3 | import com.google.common.collect.ImmutableList;
4 | import com.google.common.collect.ImmutableMap;
5 | import com.google.common.collect.ImmutableSet;
6 | import org.apache.avro.util.Utf8;
7 | import org.apache.cassandra.serializers.DecimalSerializer;
8 | import org.apache.cassandra.serializers.FloatSerializer;
9 | import org.apache.cassandra.serializers.Int32Serializer;
10 | import org.apache.cassandra.serializers.ListSerializer;
11 | import org.apache.cassandra.serializers.MapSerializer;
12 | import org.apache.cassandra.serializers.SetSerializer;
13 | import org.apache.cassandra.serializers.UTF8Serializer;
14 | import org.junit.Test;
15 |
16 | import java.math.BigDecimal;
17 | import java.nio.ByteBuffer;
18 | import java.util.List;
19 | import java.util.Map;
20 | import java.util.Set;
21 |
22 | import static org.junit.Assert.assertEquals;
23 |
24 | public class CassandraRecordUtilsTest {
25 |
26 | @Test
27 | public void testSerializeMap() {
28 | Map map = ImmutableMap.of("foo", 1, "bar", 2);
29 |
30 | ByteBuffer expected =
31 | MapSerializer.getInstance(UTF8Serializer.instance, Int32Serializer.instance).serialize(map);
32 | assertEquals(expected, CassandraRecordUtils.toByteBuffer(map));
33 | }
34 |
35 | @Test
36 | public void testSerializeMapUtf8() {
37 | Map map = ImmutableMap.of(new Utf8("foo"), 1, new Utf8("bar"), 2);
38 | Map expectedMap = ImmutableMap.of("foo", 1, "bar", 2);
39 | ByteBuffer expectedBytes =
40 | MapSerializer.getInstance(UTF8Serializer.instance, Int32Serializer.instance)
41 | .serialize(expectedMap);
42 | assertEquals(expectedBytes, CassandraRecordUtils.toByteBuffer(map));
43 | }
44 |
45 | @Test
46 | public void testSerializeList() {
47 | List list = ImmutableList.of(BigDecimal.valueOf(0),
48 | new BigDecimal("1.2"),
49 | new BigDecimal("3.4"));
50 |
51 | ByteBuffer expected = ListSerializer.getInstance(DecimalSerializer.instance).serialize(list);
52 | assertEquals(expected, CassandraRecordUtils.toByteBuffer(list));
53 | }
54 |
55 | @Test
56 | public void testSerializeListUtf8() {
57 | List list = ImmutableList.of(new Utf8("foo"), new Utf8("bar"), new Utf8("baz"));
58 | List expectedList = ImmutableList.of("foo", "bar", "baz");
59 | ByteBuffer expectedBytes = ListSerializer.getInstance(UTF8Serializer.instance)
60 | .serialize(expectedList);
61 | assertEquals(expectedBytes, CassandraRecordUtils.toByteBuffer(list));
62 | }
63 |
64 | @Test
65 | public void testSerializeSet() {
66 | Set set = ImmutableSet.of(1.0f, 2.0f, 3.0f);
67 | ByteBuffer expected = SetSerializer.getInstance(FloatSerializer.instance).serialize(set);
68 | assertEquals(expected, CassandraRecordUtils.toByteBuffer(set));
69 | }
70 |
71 | @Test
72 | public void testSerializeSetUtf8() {
73 | Set set = ImmutableSet.of(new Utf8("foo"), new Utf8("bar"), new Utf8("baz"));
74 | Set expectedSet = ImmutableSet.of("foo", "bar", "baz");
75 | ByteBuffer expectedBytes = SetSerializer.getInstance(UTF8Serializer.instance).serialize(expectedSet);
76 | assertEquals(expectedBytes, CassandraRecordUtils.toByteBuffer(set));
77 | }
78 |
79 | }
80 |
--------------------------------------------------------------------------------