-
Notifications
You must be signed in to change notification settings - Fork 0
/
ColumnFamilyRecordReader.java
executable file
·270 lines (241 loc) · 9.04 KB
/
ColumnFamilyRecordReader.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
package org.apache.cassandra.hadoop;
/*
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*
*/
import java.io.IOException;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.List;
import java.util.SortedMap;
import java.util.TreeMap;
import com.google.common.collect.AbstractIterator;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.thrift.*;
import org.apache.cassandra.thrift.Column;
import org.apache.cassandra.thrift.SuperColumn;
import org.apache.cassandra.utils.Pair;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.transport.TSocket;
import org.apache.thrift.transport.TTransportException;
public class ColumnFamilyRecordReader extends RecordReader<String, SortedMap<byte[], IColumn>>
{
private ColumnFamilySplit split;
private RowIterator iter;
private Pair<String, SortedMap<byte[], IColumn>> currentRow;
private SlicePredicate predicate;
private int totalRowCount; // total number of rows to fetch
private int batchRowCount; // fetch this many per batch
private String cfName;
private String keyspace;
public void close()
{
if (iter != null)
iter.close();
}
public String getCurrentKey()
{
return currentRow.left;
}
public SortedMap<byte[], IColumn> getCurrentValue()
{
return currentRow.right;
}
public float getProgress()
{
// the progress is likely to be reported slightly off the actual but close enough
return ((float)iter.rowsRead()) / totalRowCount;
}
public void initialize(InputSplit split, TaskAttemptContext context) throws IOException
{
this.split = (ColumnFamilySplit) split;
Configuration conf = context.getConfiguration();
predicate = ConfigHelper.getSlicePredicate(conf);
totalRowCount = ConfigHelper.getInputSplitSize(conf);
batchRowCount = ConfigHelper.getRangeBatchSize(conf);
cfName = ConfigHelper.getColumnFamily(conf);
keyspace = ConfigHelper.getKeyspace(conf);
iter = new RowIterator();
}
public boolean nextKeyValue() throws IOException
{
if (!iter.hasNext())
return false;
currentRow = iter.next();
return true;
}
private class RowIterator extends AbstractIterator<Pair<String, SortedMap<byte[], IColumn>>>
{
private List<KeySlice> rows;
private String startToken;
private int totalRead = 0;
private int i = 0;
private AbstractType comparator = DatabaseDescriptor.getComparator(keyspace, cfName);
private TSocket socket;
private void maybeInit()
{
// check if we need another batch
if (rows != null && i >= rows.size())
rows = null;
if (rows != null)
return;
// close previous connection if one is open
close();
socket = new TSocket(getLocation(),
DatabaseDescriptor.getThriftPort());
TBinaryProtocol binaryProtocol = new TBinaryProtocol(socket, false, false);
Cassandra.Client client = new Cassandra.Client(binaryProtocol);
try
{
socket.open();
}
catch (TTransportException e)
{
throw new RuntimeException(e);
}
if (startToken == null)
{
startToken = split.getStartToken();
}
else if (startToken.equals(split.getEndToken()))
{
rows = null;
return;
}
KeyRange keyRange = new KeyRange(batchRowCount)
.setStart_token(startToken)
.setEnd_token(split.getEndToken());
try
{
rows = client.get_range_slices(keyspace,
new ColumnParent(cfName),
predicate,
keyRange,
ConsistencyLevel.ONE);
// nothing new? reached the end
if (rows.isEmpty())
{
rows = null;
return;
}
// reset to iterate through this new batch
i = 0;
// prepare for the next slice to be read
KeySlice lastRow = rows.get(rows.size() - 1);
IPartitioner p = DatabaseDescriptor.getPartitioner();
startToken = p.getTokenFactory().toString(p.getToken(lastRow.getKey()));
}
catch (Exception e)
{
throw new RuntimeException(e);
}
}
// we don't use endpointsnitch since we are trying to support hadoop nodes that are
// not necessarily on Cassandra machines, too. This should be adequate for single-DC clusters, at least.
private String getLocation()
{
InetAddress[] localAddresses = new InetAddress[0];
try
{
localAddresses = InetAddress.getAllByName(InetAddress.getLocalHost().getHostAddress());
}
catch (UnknownHostException e)
{
throw new AssertionError(e);
}
for (InetAddress address : localAddresses)
{
for (String location : split.getLocations())
{
InetAddress locationAddress = null;
try
{
locationAddress = InetAddress.getByName(location);
}
catch (UnknownHostException e)
{
throw new AssertionError(e);
}
if (address.equals(locationAddress))
{
return location;
}
}
}
return split.getLocations()[0];
}
/**
* @return total number of rows read by this record reader
*/
public int rowsRead()
{
return totalRead;
}
@Override
protected Pair<String, SortedMap<byte[], IColumn>> computeNext()
{
maybeInit();
if (rows == null)
return endOfData();
totalRead++;
KeySlice ks = rows.get(i++);
SortedMap<byte[], IColumn> map = new TreeMap<byte[], IColumn>(comparator);
for (ColumnOrSuperColumn cosc : ks.columns)
{
IColumn column = unthriftify(cosc);
map.put(column.name(), column);
}
return new Pair<String, SortedMap<byte[], IColumn>>(ks.key, map);
}
public void close()
{
if (socket != null && socket.isOpen())
{
socket.close();
}
}
}
private IColumn unthriftify(ColumnOrSuperColumn cosc)
{
if (cosc.column == null)
return unthriftifySuper(cosc.super_column);
return unthriftifySimple(cosc.column);
}
private IColumn unthriftifySuper(SuperColumn super_column)
{
AbstractType subComparator = DatabaseDescriptor.getSubComparator(keyspace, cfName);
org.apache.cassandra.db.SuperColumn sc = new org.apache.cassandra.db.SuperColumn(super_column.name, subComparator);
for (Column column : super_column.columns)
{
sc.addColumn(unthriftifySimple(column));
}
return sc;
}
private IColumn unthriftifySimple(Column column)
{
return new org.apache.cassandra.db.Column(column.name, column.value, column.timestamp);
}
}