forked from apache/cassandra
/
CassandraStorage.java
951 lines (865 loc) · 36.6 KB
/
CassandraStorage.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
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
/**
* 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.
*/
package org.apache.cassandra.hadoop.pig;
import java.io.IOException;
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.util.*;
import org.apache.cassandra.config.ConfigurationException;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.BytesType;
import org.apache.cassandra.db.marshal.IntegerType;
import org.apache.cassandra.db.marshal.TypeParser;
import org.apache.cassandra.thrift.*;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Hex;
import org.apache.cassandra.utils.UUIDGen;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.cassandra.db.Column;
import org.apache.cassandra.db.IColumn;
import org.apache.cassandra.db.marshal.*;
import org.apache.cassandra.db.marshal.AbstractCompositeType.CompositeComponent;
import org.apache.cassandra.hadoop.*;
import org.apache.cassandra.thrift.Mutation;
import org.apache.cassandra.thrift.Deletion;
import org.apache.cassandra.thrift.ColumnOrSuperColumn;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.*;
import org.apache.pig.*;
import org.apache.pig.backend.executionengine.ExecException;
import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
import org.apache.pig.data.*;
import org.apache.pig.ResourceSchema.ResourceFieldSchema;
import org.apache.pig.impl.util.UDFContext;
import org.apache.thrift.TDeserializer;
import org.apache.thrift.TException;
import org.apache.thrift.TSerializer;
/**
* A LoadStoreFunc for retrieving data from and storing data to Cassandra
*
* A row from a standard CF will be returned as nested tuples: (key, ((name1, val1), (name2, val2))).
*/
public class CassandraStorage extends LoadFunc implements StoreFuncInterface, LoadMetadata
{
// system environment variables that can be set to configure connection info:
// alternatively, Hadoop JobConf variables can be set using keys from ConfigHelper
public final static String PIG_INPUT_RPC_PORT = "PIG_INPUT_RPC_PORT";
public final static String PIG_INPUT_INITIAL_ADDRESS = "PIG_INPUT_INITIAL_ADDRESS";
public final static String PIG_INPUT_PARTITIONER = "PIG_INPUT_PARTITIONER";
public final static String PIG_OUTPUT_RPC_PORT = "PIG_OUTPUT_RPC_PORT";
public final static String PIG_OUTPUT_INITIAL_ADDRESS = "PIG_OUTPUT_INITIAL_ADDRESS";
public final static String PIG_OUTPUT_PARTITIONER = "PIG_OUTPUT_PARTITIONER";
public final static String PIG_RPC_PORT = "PIG_RPC_PORT";
public final static String PIG_INITIAL_ADDRESS = "PIG_INITIAL_ADDRESS";
public final static String PIG_PARTITIONER = "PIG_PARTITIONER";
public final static String PIG_INPUT_FORMAT = "PIG_INPUT_FORMAT";
public final static String PIG_OUTPUT_FORMAT = "PIG_OUTPUT_FORMAT";
public final static String PIG_ALLOW_DELETES = "PIG_ALLOW_DELETES";
public final static String PIG_WIDEROW_INPUT = "PIG_WIDEROW_INPUT";
private final static String DEFAULT_INPUT_FORMAT = "org.apache.cassandra.hadoop.ColumnFamilyInputFormat";
private final static String DEFAULT_OUTPUT_FORMAT = "org.apache.cassandra.hadoop.ColumnFamilyOutputFormat";
private final static boolean DEFAULT_WIDEROW_INPUT = false;
private final static ByteBuffer BOUND = ByteBufferUtil.EMPTY_BYTE_BUFFER;
private static final Log logger = LogFactory.getLog(CassandraStorage.class);
private ByteBuffer slice_start = BOUND;
private ByteBuffer slice_end = BOUND;
private boolean slice_reverse = false;
private boolean allow_deletes = false;
private String keyspace;
private String column_family;
private String loadSignature;
private String storeSignature;
private Configuration conf;
private RecordReader<ByteBuffer, Map<ByteBuffer, IColumn>> reader;
private RecordWriter<ByteBuffer, List<Mutation>> writer;
private String inputFormatClass;
private String outputFormatClass;
private int limit;
private boolean widerows;
// wide row hacks
private Map<ByteBuffer,IColumn> lastRow;
private boolean hasNext = true;
public CassandraStorage()
{
this(1024);
}
/**
* @param limit number of columns to fetch in a slice
*/
public CassandraStorage(int limit)
{
super();
this.limit = limit;
}
public int getLimit()
{
return limit;
}
public Tuple getNextWide() throws IOException
{
CfDef cfDef = getCfDef(loadSignature);
ByteBuffer key = null;
Tuple tuple = TupleFactory.getInstance().newTuple();
DefaultDataBag bag = new DefaultDataBag();
try
{
while(true)
{
hasNext = reader.nextKeyValue();
if (!hasNext)
{
if (lastRow != null)
{
if (tuple.size() == 0) // lastRow is a new one
{
key = (ByteBuffer)reader.getCurrentKey();
tuple.append(new DataByteArray(key.array(), key.position()+key.arrayOffset(), key.limit()+key.arrayOffset()));
}
for (Map.Entry<ByteBuffer, IColumn> entry : lastRow.entrySet())
{
bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
}
lastRow = null;
tuple.append(bag);
return tuple;
}
else
{
if (tuple.size() == 1) // rare case of just one wide row, key already set
{
tuple.append(bag);
return tuple;
}
else
return null;
}
}
if (key != null && !((ByteBuffer)reader.getCurrentKey()).equals(key)) // key changed
{
// read too much, hold on to it for next time
lastRow = (SortedMap<ByteBuffer,IColumn>)reader.getCurrentValue();
// but return what we have so far
tuple.append(bag);
return tuple;
}
if (key == null) // only set the key on the first iteration
{
key = (ByteBuffer)reader.getCurrentKey();
tuple.append(new DataByteArray(key.array(), key.position()+key.arrayOffset(), key.limit()+key.arrayOffset()));
}
SortedMap<ByteBuffer,IColumn> row = (SortedMap<ByteBuffer,IColumn>)reader.getCurrentValue();
if (lastRow != null) // prepend what was read last time
{
for (Map.Entry<ByteBuffer, IColumn> entry : lastRow.entrySet())
{
bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
}
lastRow = null;
}
for (Map.Entry<ByteBuffer, IColumn> entry : row.entrySet())
{
bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
}
}
}
catch (InterruptedException e)
{
throw new IOException(e.getMessage());
}
}
@Override
public Tuple getNext() throws IOException
{
if (widerows)
return getNextWide();
try
{
// load the next pair
if (!reader.nextKeyValue())
return null;
CfDef cfDef = getCfDef(loadSignature);
ByteBuffer key = reader.getCurrentKey();
Map<ByteBuffer, IColumn> cf = reader.getCurrentValue();
assert key != null && cf != null;
// output tuple, will hold the key, each indexed column in a tuple, then a bag of the rest
// NOTE: we're setting the tuple size here only for the key so we can use setTupleValue on it
Tuple tuple = TupleFactory.getInstance().newTuple(1);
DefaultDataBag bag = new DefaultDataBag();
// set the key
setTupleValue(tuple, 0, getDefaultMarshallers(cfDef).get(2).compose(key));
// we must add all the indexed columns first to match the schema
Map<ByteBuffer, Boolean> added = new HashMap<ByteBuffer, Boolean>();
// take care to iterate these in the same order as the schema does
for (ColumnDef cdef : cfDef.column_metadata)
{
if (cf.containsKey(cdef.name))
{
tuple.append(columnToTuple(cf.get(cdef.name), cfDef, parseType(cfDef.getComparator_type())));
}
else
{ // otherwise, we need to add an empty tuple to take its place
tuple.append(TupleFactory.getInstance().newTuple());
}
added.put(cdef.name, true);
}
// now add all the other columns
for (Map.Entry<ByteBuffer, IColumn> entry : cf.entrySet())
{
if (!added.containsKey(entry.getKey()))
bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
}
tuple.append(bag);
return tuple;
}
catch (InterruptedException e)
{
throw new IOException(e.getMessage());
}
}
/**
* Deconstructs a composite type to a Tuple.
*/
private Tuple composeComposite( AbstractCompositeType comparator, ByteBuffer name ) throws IOException
{
List<CompositeComponent> result = comparator.deconstruct( name );
Tuple t = TupleFactory.getInstance().newTuple( result.size() );
for( int i = 0; i < result.size(); i++ )
{
setTupleValue( t, i, result.get(i).comparator.compose( result.get(i).value ) );
}
return t;
}
private Tuple columnToTuple(IColumn col, CfDef cfDef, AbstractType comparator) throws IOException
{
Tuple pair = TupleFactory.getInstance().newTuple(2);
if( comparator instanceof AbstractCompositeType )
{
setTupleValue(pair, 0, composeComposite((AbstractCompositeType)comparator,col.name()));
}
else
{
setTupleValue(pair, 0, comparator.compose(col.name()));
}
if (col instanceof Column)
{
// standard
List<AbstractType> marshallers = getDefaultMarshallers(cfDef);
Map<ByteBuffer,AbstractType> validators = getValidatorMap(cfDef);
if (validators.get(col.name()) == null)
setTupleValue(pair, 1, marshallers.get(1).compose(col.value()));
else
setTupleValue(pair, 1, validators.get(col.name()).compose(col.value()));
return pair;
}
else
{
// super
ArrayList<Tuple> subcols = new ArrayList<Tuple>();
for (IColumn subcol : col.getSubColumns())
subcols.add(columnToTuple(subcol, cfDef, parseType(cfDef.getSubcomparator_type())));
pair.set(1, new DefaultDataBag(subcols));
}
return pair;
}
private void setTupleValue(Tuple pair, int position, Object value) throws ExecException
{
if (value instanceof BigInteger)
pair.set(position, ((BigInteger) value).intValue());
else if (value instanceof ByteBuffer)
pair.set(position, new DataByteArray(ByteBufferUtil.getArray((ByteBuffer) value)));
else if (value instanceof UUID)
pair.set(position, new DataByteArray(UUIDGen.decompose((java.util.UUID) value)));
else if (value instanceof Date)
pair.set(position, new DataByteArray(ByteBufferUtil.getArray(DateType.instance.decompose((Date) value))));
else
pair.set(position, value);
}
private CfDef getCfDef(String signature)
{
UDFContext context = UDFContext.getUDFContext();
Properties property = context.getUDFProperties(CassandraStorage.class);
return cfdefFromString(property.getProperty(signature));
}
private List<AbstractType> getDefaultMarshallers(CfDef cfDef) throws IOException
{
ArrayList<AbstractType> marshallers = new ArrayList<AbstractType>();
AbstractType comparator;
AbstractType subcomparator;
AbstractType default_validator;
AbstractType key_validator;
comparator = parseType(cfDef.getComparator_type());
subcomparator = parseType(cfDef.getSubcomparator_type());
default_validator = parseType(cfDef.getDefault_validation_class());
key_validator = parseType(cfDef.getKey_validation_class());
marshallers.add(comparator);
marshallers.add(default_validator);
marshallers.add(key_validator);
marshallers.add(subcomparator);
return marshallers;
}
private Map<ByteBuffer, AbstractType> getValidatorMap(CfDef cfDef) throws IOException
{
Map<ByteBuffer, AbstractType> validators = new HashMap<ByteBuffer, AbstractType>();
for (ColumnDef cd : cfDef.getColumn_metadata())
{
if (cd.getValidation_class() != null && !cd.getValidation_class().isEmpty())
{
AbstractType validator = null;
try
{
validator = TypeParser.parse(cd.getValidation_class());
validators.put(cd.name, validator);
}
catch (ConfigurationException e)
{
throw new IOException(e);
}
}
}
return validators;
}
private AbstractType parseType(String type) throws IOException
{
try
{
// always treat counters like longs, specifically CCT.compose is not what we need
if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
return LongType.instance;
return TypeParser.parse(type);
}
catch (ConfigurationException e)
{
throw new IOException(e);
}
}
@Override
public InputFormat getInputFormat()
{
try
{
return FBUtilities.construct(inputFormatClass, "inputformat");
}
catch (ConfigurationException e)
{
throw new RuntimeException(e);
}
}
@Override
public void prepareToRead(RecordReader reader, PigSplit split)
{
this.reader = reader;
}
public static Map<String, String> getQueryMap(String query)
{
String[] params = query.split("&");
Map<String, String> map = new HashMap<String, String>();
for (String param : params)
{
String[] keyValue = param.split("=");
map.put(keyValue[0], keyValue[1]);
}
return map;
}
private void setLocationFromUri(String location) throws IOException
{
// parse uri into keyspace and columnfamily
String names[];
try
{
if (!location.startsWith("cassandra://"))
throw new Exception("Bad scheme.");
String[] urlParts = location.split("\\?");
if (urlParts.length > 1)
{
Map<String, String> urlQuery = getQueryMap(urlParts[1]);
AbstractType comparator = BytesType.instance;
if (urlQuery.containsKey("comparator"))
comparator = TypeParser.parse(urlQuery.get("comparator"));
if (urlQuery.containsKey("slice_start"))
slice_start = comparator.fromString(urlQuery.get("slice_start"));
if (urlQuery.containsKey("slice_end"))
slice_end = comparator.fromString(urlQuery.get("slice_end"));
if (urlQuery.containsKey("reversed"))
slice_reverse = Boolean.parseBoolean(urlQuery.get("reversed"));
if (urlQuery.containsKey("limit"))
limit = Integer.parseInt(urlQuery.get("limit"));
}
String[] parts = urlParts[0].split("/+");
keyspace = parts[1];
column_family = parts[2];
}
catch (Exception e)
{
throw new IOException("Expected 'cassandra://<keyspace>/<columnfamily>[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]]': " + e.getMessage());
}
}
private void setConnectionInformation() throws IOException
{
if (System.getenv(PIG_RPC_PORT) != null)
{
ConfigHelper.setInputRpcPort(conf, System.getenv(PIG_RPC_PORT));
ConfigHelper.setOutputRpcPort(conf, System.getenv(PIG_RPC_PORT));
}
if (System.getenv(PIG_INPUT_RPC_PORT) != null)
ConfigHelper.setInputRpcPort(conf, System.getenv(PIG_INPUT_RPC_PORT));
if (System.getenv(PIG_OUTPUT_RPC_PORT) != null)
ConfigHelper.setOutputRpcPort(conf, System.getenv(PIG_OUTPUT_RPC_PORT));
if (System.getenv(PIG_INITIAL_ADDRESS) != null)
{
ConfigHelper.setInputInitialAddress(conf, System.getenv(PIG_INITIAL_ADDRESS));
ConfigHelper.setOutputInitialAddress(conf, System.getenv(PIG_INITIAL_ADDRESS));
}
if (System.getenv(PIG_INPUT_INITIAL_ADDRESS) != null)
ConfigHelper.setInputInitialAddress(conf, System.getenv(PIG_INPUT_INITIAL_ADDRESS));
if (System.getenv(PIG_OUTPUT_INITIAL_ADDRESS) != null)
ConfigHelper.setOutputInitialAddress(conf, System.getenv(PIG_OUTPUT_INITIAL_ADDRESS));
if (System.getenv(PIG_PARTITIONER) != null)
{
ConfigHelper.setInputPartitioner(conf, System.getenv(PIG_PARTITIONER));
ConfigHelper.setOutputPartitioner(conf, System.getenv(PIG_PARTITIONER));
}
if(System.getenv(PIG_INPUT_PARTITIONER) != null)
ConfigHelper.setInputPartitioner(conf, System.getenv(PIG_INPUT_PARTITIONER));
if(System.getenv(PIG_OUTPUT_PARTITIONER) != null)
ConfigHelper.setOutputPartitioner(conf, System.getenv(PIG_OUTPUT_PARTITIONER));
if (System.getenv(PIG_INPUT_FORMAT) != null)
inputFormatClass = getFullyQualifiedClassName(System.getenv(PIG_INPUT_FORMAT));
else
inputFormatClass = DEFAULT_INPUT_FORMAT;
if (System.getenv(PIG_OUTPUT_FORMAT) != null)
outputFormatClass = getFullyQualifiedClassName(System.getenv(PIG_OUTPUT_FORMAT));
else
outputFormatClass = DEFAULT_OUTPUT_FORMAT;
if (System.getenv(PIG_ALLOW_DELETES) != null)
allow_deletes = Boolean.valueOf(System.getenv(PIG_ALLOW_DELETES));
}
private String getFullyQualifiedClassName(String classname)
{
return classname.contains(".") ? classname : "org.apache.cassandra.hadoop." + classname;
}
@Override
public void setLocation(String location, Job job) throws IOException
{
conf = job.getConfiguration();
setLocationFromUri(location);
if (ConfigHelper.getInputSlicePredicate(conf) == null)
{
SliceRange range = new SliceRange(slice_start, slice_end, slice_reverse, limit);
SlicePredicate predicate = new SlicePredicate().setSlice_range(range);
ConfigHelper.setInputSlicePredicate(conf, predicate);
}
widerows = DEFAULT_WIDEROW_INPUT;
if (System.getenv(PIG_WIDEROW_INPUT) != null)
widerows = Boolean.valueOf(System.getProperty(PIG_WIDEROW_INPUT));
ConfigHelper.setInputColumnFamily(conf, keyspace, column_family, widerows);
setConnectionInformation();
if (ConfigHelper.getInputRpcPort(conf) == 0)
throw new IOException("PIG_INPUT_RPC_PORT or PIG_RPC_PORT environment variable not set");
if (ConfigHelper.getInputInitialAddress(conf) == null)
throw new IOException("PIG_INPUT_INITIAL_ADDRESS or PIG_INITIAL_ADDRESS environment variable not set");
if (ConfigHelper.getInputPartitioner(conf) == null)
throw new IOException("PIG_INPUT_PARTITIONER or PIG_PARTITIONER environment variable not set");
if (loadSignature == null)
loadSignature = location;
initSchema(loadSignature);
}
public ResourceSchema getSchema(String location, Job job) throws IOException
{
setLocation(location, job);
CfDef cfDef = getCfDef(loadSignature);
if (cfDef.column_type.equals("Super"))
return null;
/*
Our returned schema should look like this:
(key, index1:(name, value), index2:(name, value), columns:{(name, value)})
Which is to say, columns that have metadata will be returned as named tuples, but unknown columns will go into a bag.
This way, wide rows can still be handled by the bag, but known columns can easily be referenced.
*/
// top-level schema, no type
ResourceSchema schema = new ResourceSchema();
// get default marshallers and validators
List<AbstractType> marshallers = getDefaultMarshallers(cfDef);
Map<ByteBuffer,AbstractType> validators = getValidatorMap(cfDef);
// add key
ResourceFieldSchema keyFieldSchema = new ResourceFieldSchema();
keyFieldSchema.setName("key");
keyFieldSchema.setType(getPigType(marshallers.get(2)));
ResourceSchema bagSchema = new ResourceSchema();
ResourceFieldSchema bagField = new ResourceFieldSchema();
bagField.setType(DataType.BAG);
bagField.setName("columns");
// inside the bag, place one tuple with the default comparator/validator schema
ResourceSchema bagTupleSchema = new ResourceSchema();
ResourceFieldSchema bagTupleField = new ResourceFieldSchema();
bagTupleField.setType(DataType.TUPLE);
ResourceFieldSchema bagcolSchema = new ResourceFieldSchema();
ResourceFieldSchema bagvalSchema = new ResourceFieldSchema();
bagcolSchema.setName("name");
bagvalSchema.setName("value");
bagcolSchema.setType(getPigType(marshallers.get(0)));
bagvalSchema.setType(getPigType(marshallers.get(1)));
bagTupleSchema.setFields(new ResourceFieldSchema[] { bagcolSchema, bagvalSchema });
bagTupleField.setSchema(bagTupleSchema);
bagSchema.setFields(new ResourceFieldSchema[] { bagTupleField });
bagField.setSchema(bagSchema);
// will contain all fields for this schema
List<ResourceFieldSchema> allSchemaFields = new ArrayList<ResourceFieldSchema>();
// add the key first, then the indexed columns, and finally the bag
allSchemaFields.add(keyFieldSchema);
// defined validators/indexes
for (ColumnDef cdef : cfDef.column_metadata)
{
// make a new tuple for each col/val pair
ResourceSchema innerTupleSchema = new ResourceSchema();
ResourceFieldSchema innerTupleField = new ResourceFieldSchema();
innerTupleField.setType(DataType.TUPLE);
innerTupleField.setSchema(innerTupleSchema);
innerTupleField.setName(new String(cdef.getName()));
ResourceFieldSchema idxColSchema = new ResourceFieldSchema();
idxColSchema.setName("name");
idxColSchema.setType(getPigType(marshallers.get(0)));
ResourceFieldSchema valSchema = new ResourceFieldSchema();
AbstractType validator = validators.get(cdef.name);
if (validator == null)
validator = marshallers.get(1);
valSchema.setName("value");
valSchema.setType(getPigType(validator));
innerTupleSchema.setFields(new ResourceFieldSchema[] { idxColSchema, valSchema });
allSchemaFields.add(innerTupleField);
}
// bag at the end for unknown columns
allSchemaFields.add(bagField);
// top level schema contains everything
schema.setFields(allSchemaFields.toArray(new ResourceFieldSchema[allSchemaFields.size()]));
return schema;
}
private byte getPigType(AbstractType type)
{
if (type instanceof LongType || type instanceof DateType) // DateType is bad and it should feel bad
return DataType.LONG;
else if (type instanceof IntegerType)
return DataType.INTEGER;
else if (type instanceof AsciiType)
return DataType.CHARARRAY;
else if (type instanceof UTF8Type)
return DataType.CHARARRAY;
else if (type instanceof FloatType)
return DataType.FLOAT;
else if (type instanceof DoubleType)
return DataType.DOUBLE;
else if (type instanceof AbstractCompositeType )
return DataType.TUPLE;
return DataType.BYTEARRAY;
}
public ResourceStatistics getStatistics(String location, Job job)
{
return null;
}
public String[] getPartitionKeys(String location, Job job)
{
return null;
}
public void setPartitionFilter(Expression partitionFilter)
{
// no-op
}
@Override
public String relativeToAbsolutePath(String location, Path curDir) throws IOException
{
return location;
}
@Override
public void setUDFContextSignature(String signature)
{
this.loadSignature = signature;
}
/* StoreFunc methods */
public void setStoreFuncUDFContextSignature(String signature)
{
this.storeSignature = signature;
}
public String relToAbsPathForStoreLocation(String location, Path curDir) throws IOException
{
return relativeToAbsolutePath(location, curDir);
}
public void setStoreLocation(String location, Job job) throws IOException
{
conf = job.getConfiguration();
setLocationFromUri(location);
ConfigHelper.setOutputColumnFamily(conf, keyspace, column_family);
setConnectionInformation();
if (ConfigHelper.getOutputRpcPort(conf) == 0)
throw new IOException("PIG_OUTPUT_RPC_PORT or PIG_RPC_PORT environment variable not set");
if (ConfigHelper.getOutputInitialAddress(conf) == null)
throw new IOException("PIG_OUTPUT_INITIAL_ADDRESS or PIG_INITIAL_ADDRESS environment variable not set");
if (ConfigHelper.getOutputPartitioner(conf) == null)
throw new IOException("PIG_OUTPUT_PARTITIONER or PIG_PARTITIONER environment variable not set");
initSchema(storeSignature);
}
public OutputFormat getOutputFormat()
{
try
{
return FBUtilities.construct(outputFormatClass, "outputformat");
}
catch (ConfigurationException e)
{
throw new RuntimeException(e);
}
}
public void checkSchema(ResourceSchema schema) throws IOException
{
// we don't care about types, they all get casted to ByteBuffers
}
public void prepareToWrite(RecordWriter writer)
{
this.writer = writer;
}
private ByteBuffer objToBB(Object o)
{
if (o == null)
return (ByteBuffer)o;
if (o instanceof java.lang.String)
return ByteBuffer.wrap(new DataByteArray((String)o).get());
if (o instanceof Integer)
return Int32Type.instance.decompose((Integer)o);
if (o instanceof Long)
return LongType.instance.decompose((Long)o);
if (o instanceof Float)
return FloatType.instance.decompose((Float)o);
if (o instanceof Double)
return DoubleType.instance.decompose((Double)o);
if (o instanceof UUID)
return ByteBuffer.wrap(UUIDGen.decompose((UUID) o));
return ByteBuffer.wrap(((DataByteArray) o).get());
}
public void putNext(Tuple t) throws IOException
{
/*
We support two cases for output:
First, the original output:
(key, (name, value), (name,value), {(name,value)}) (tuples or bag is optional)
For supers, we only accept the original output.
*/
if (t.size() < 1)
{
// simply nothing here, we can't even delete without a key
logger.warn("Empty output skipped, filter empty tuples to suppress this warning");
return;
}
ByteBuffer key = objToBB(t.get(0));
if (t.getType(1) == DataType.TUPLE)
writeColumnsFromTuple(key, t, 1);
else if (t.getType(1) == DataType.BAG)
{
if (t.size() > 2)
throw new IOException("No arguments allowed after bag");
writeColumnsFromBag(key, (DefaultDataBag) t.get(1));
}
else
throw new IOException("Second argument in output must be a tuple or bag");
}
private void writeColumnsFromTuple(ByteBuffer key, Tuple t, int offset) throws IOException
{
ArrayList<Mutation> mutationList = new ArrayList<Mutation>();
for (int i = offset; i < t.size(); i++)
{
if (t.getType(i) == DataType.BAG)
writeColumnsFromBag(key, (DefaultDataBag) t.get(i));
else if (t.getType(i) == DataType.TUPLE)
{
Tuple inner = (Tuple) t.get(i);
if (inner.size() > 0) // may be empty, for an indexed column that wasn't present
mutationList.add(mutationFromTuple(inner));
}
else
throw new IOException("Output type was not a bag or a tuple");
}
if (mutationList.size() > 0)
writeMutations(key, mutationList);
}
private Mutation mutationFromTuple(Tuple t) throws IOException
{
Mutation mutation = new Mutation();
if (t.get(1) == null)
{
if (allow_deletes)
{
mutation.deletion = new Deletion();
mutation.deletion.predicate = new org.apache.cassandra.thrift.SlicePredicate();
mutation.deletion.predicate.column_names = Arrays.asList(objToBB(t.get(0)));
mutation.deletion.setTimestamp(FBUtilities.timestampMicros());
}
else
throw new IOException("null found but deletes are disabled, set " + PIG_ALLOW_DELETES + "=true to enable");
}
else
{
org.apache.cassandra.thrift.Column column = new org.apache.cassandra.thrift.Column();
column.setName(objToBB(t.get(0)));
column.setValue(objToBB(t.get(1)));
column.setTimestamp(FBUtilities.timestampMicros());
mutation.column_or_supercolumn = new ColumnOrSuperColumn();
mutation.column_or_supercolumn.column = column;
}
return mutation;
}
private void writeColumnsFromBag(ByteBuffer key, DefaultDataBag bag) throws IOException
{
List<Mutation> mutationList = new ArrayList<Mutation>();
for (Tuple pair : bag)
{
Mutation mutation = new Mutation();
if (DataType.findType(pair.get(1)) == DataType.BAG) // supercolumn
{
SuperColumn sc = new SuperColumn();
sc.setName(objToBB(pair.get(0)));
List<org.apache.cassandra.thrift.Column> columns = new ArrayList<org.apache.cassandra.thrift.Column>();
for (Tuple subcol : (DefaultDataBag) pair.get(1))
{
org.apache.cassandra.thrift.Column column = new org.apache.cassandra.thrift.Column();
column.setName(objToBB(subcol.get(0)));
column.setValue(objToBB(subcol.get(1)));
column.setTimestamp(FBUtilities.timestampMicros());
columns.add(column);
}
if (columns.isEmpty())
{
if (allow_deletes)
{
mutation.deletion = new Deletion();
mutation.deletion.super_column = objToBB(pair.get(0));
mutation.deletion.setTimestamp(FBUtilities.timestampMicros());
}
else
throw new IOException("SuperColumn deletion attempted with empty bag, but deletes are disabled, set " + PIG_ALLOW_DELETES + "=true to enable");
}
else
{
sc.columns = columns;
mutation.column_or_supercolumn = new ColumnOrSuperColumn();
mutation.column_or_supercolumn.super_column = sc;
}
}
else
mutation = mutationFromTuple(pair);
mutationList.add(mutation);
// for wide rows, we need to limit the amount of mutations we write at once
if (mutationList.size() >= 10) // arbitrary, CFOF will re-batch this up, and BOF won't care
{
writeMutations(key, mutationList);
mutationList.clear();
}
}
// write the last batch
if (mutationList.size() > 0)
writeMutations(key, mutationList);
}
private void writeMutations(ByteBuffer key, List<Mutation> mutations) throws IOException
{
try
{
writer.write(key, mutations);
}
catch (InterruptedException e)
{
throw new IOException(e);
}
}
public void cleanupOnFailure(String failure, Job job)
{
}
/* Methods to get the column family schema from Cassandra */
private void initSchema(String signature)
{
UDFContext context = UDFContext.getUDFContext();
Properties property = context.getUDFProperties(CassandraStorage.class);
// Only get the schema if we haven't already gotten it
if (!property.containsKey(signature))
{
Cassandra.Client client = null;
try
{
client = ConfigHelper.getClientFromInputAddressList(conf);
CfDef cfDef = null;
client.set_keyspace(keyspace);
KsDef ksDef = client.describe_keyspace(keyspace);
List<CfDef> defs = ksDef.getCf_defs();
for (CfDef def : defs)
{
if (column_family.equalsIgnoreCase(def.getName()))
{
cfDef = def;
break;
}
}
if (cfDef != null)
property.setProperty(signature, cfdefToString(cfDef));
else
throw new RuntimeException("Column family '" + column_family + "' not found in keyspace '" + keyspace + "'");
}
catch (TException e)
{
throw new RuntimeException(e);
}
catch (InvalidRequestException e)
{
throw new RuntimeException(e);
}
catch (NotFoundException e)
{
throw new RuntimeException(e);
}
catch (IOException e)
{
throw new RuntimeException(e);
}
}
}
private static String cfdefToString(CfDef cfDef)
{
assert cfDef != null;
// this is so awful it's kind of cool!
TSerializer serializer = new TSerializer(new TBinaryProtocol.Factory());
try
{
return Hex.bytesToHex(serializer.serialize(cfDef));
}
catch (TException e)
{
throw new RuntimeException(e);
}
}
private static CfDef cfdefFromString(String st)
{
assert st != null;
TDeserializer deserializer = new TDeserializer(new TBinaryProtocol.Factory());
CfDef cfDef = new CfDef();
try
{
deserializer.deserialize(cfDef, Hex.hexToBytes(st));
}
catch (TException e)
{
throw new RuntimeException(e);
}
return cfDef;
}
}