/
CassandraServer.java
1142 lines (1002 loc) · 44.7 KB
/
CassandraServer.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
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* 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.thrift;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.nio.ByteBuffer;
import java.nio.charset.CharacterCodingException;
import java.util.*;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeoutException;
import java.util.zip.DataFormatException;
import java.util.zip.Inflater;
import com.google.common.base.Predicates;
import com.google.common.collect.Maps;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.antlr.runtime.RecognitionException;
import org.apache.cassandra.auth.Permission;
import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.concurrent.StageManager;
import org.apache.cassandra.config.*;
import org.apache.cassandra.cql.QueryProcessor;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.filter.QueryPath;
import org.apache.cassandra.db.marshal.MarshalException;
import org.apache.cassandra.db.migration.*;
import org.apache.cassandra.db.context.CounterContext;
import org.apache.cassandra.dht.*;
import org.apache.cassandra.locator.*;
import org.apache.cassandra.scheduler.IRequestScheduler;
import org.apache.cassandra.service.ClientState;
import org.apache.cassandra.service.StorageProxy;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.thrift.TException;
public class CassandraServer implements Cassandra.Iface
{
private static Logger logger = LoggerFactory.getLogger(CassandraServer.class);
private final static List<ColumnOrSuperColumn> EMPTY_COLUMNS = Collections.emptyList();
private final static List<Column> EMPTY_SUBCOLUMNS = Collections.emptyList();
private final static List<CounterColumn> EMPTY_COUNTER_SUBCOLUMNS = Collections.emptyList();
// thread local state containing session information
public final ThreadLocal<ClientState> clientState = new ThreadLocal<ClientState>()
{
@Override
public ClientState initialValue()
{
return new ClientState();
}
};
/*
* RequestScheduler to perform the scheduling of incoming requests
*/
private final IRequestScheduler requestScheduler;
public CassandraServer()
{
requestScheduler = DatabaseDescriptor.getRequestScheduler();
}
public ClientState state()
{
return clientState.get();
}
protected Map<DecoratedKey, ColumnFamily> readColumnFamily(List<ReadCommand> commands, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
// TODO - Support multiple column families per row, right now row only contains 1 column family
Map<DecoratedKey, ColumnFamily> columnFamilyKeyMap = new HashMap<DecoratedKey, ColumnFamily>();
if (consistency_level == ConsistencyLevel.ANY)
{
throw new InvalidRequestException("Consistency level any may not be applied to read operations");
}
List<Row> rows;
try
{
try
{
schedule();
rows = StorageProxy.read(commands, consistency_level);
}
finally
{
release();
}
}
catch (TimeoutException e)
{
throw new TimedOutException();
}
catch (IOException e)
{
throw new RuntimeException(e);
}
for (Row row: rows)
{
columnFamilyKeyMap.put(row.key, row.cf);
}
return columnFamilyKeyMap;
}
public List<Column> thriftifySubColumns(Collection<IColumn> columns)
{
if (columns == null || columns.isEmpty())
{
return EMPTY_SUBCOLUMNS;
}
ArrayList<Column> thriftColumns = new ArrayList<Column>(columns.size());
for (IColumn column : columns)
{
if (column.isMarkedForDelete())
{
continue;
}
Column thrift_column = new Column(column.name()).setValue(column.value()).setTimestamp(column.timestamp());
if (column instanceof ExpiringColumn)
{
thrift_column.setTtl(((ExpiringColumn) column).getTimeToLive());
}
thriftColumns.add(thrift_column);
}
return thriftColumns;
}
public List<CounterColumn> thriftifyCounterSubColumns(Collection<IColumn> columns)
{
if (columns == null || columns.isEmpty())
{
return EMPTY_COUNTER_SUBCOLUMNS;
}
ArrayList<CounterColumn> thriftColumns = new ArrayList<CounterColumn>(columns.size());
for (IColumn column : columns)
{
if (column.isMarkedForDelete())
{
continue;
}
assert column instanceof org.apache.cassandra.db.CounterColumn;
CounterColumn thrift_column = new CounterColumn(column.name(), CounterContext.instance().total(column.value()));
thriftColumns.add(thrift_column);
}
return thriftColumns;
}
public List<ColumnOrSuperColumn> thriftifyColumns(Collection<IColumn> columns, boolean reverseOrder)
{
ArrayList<ColumnOrSuperColumn> thriftColumns = new ArrayList<ColumnOrSuperColumn>(columns.size());
for (IColumn column : columns)
{
if (column.isMarkedForDelete())
{
continue;
}
if (column instanceof org.apache.cassandra.db.CounterColumn)
{
CounterColumn thrift_column = new CounterColumn(column.name(), CounterContext.instance().total(column.value()));
thriftColumns.add(new ColumnOrSuperColumn().setCounter_column(thrift_column));
}
else
{
Column thrift_column = new Column(column.name()).setValue(column.value()).setTimestamp(column.timestamp());
if (column instanceof ExpiringColumn)
{
thrift_column.setTtl(((ExpiringColumn) column).getTimeToLive());
}
thriftColumns.add(new ColumnOrSuperColumn().setColumn(thrift_column));
}
}
// we have to do the reversing here, since internally we pass results around in ColumnFamily
// objects, which always sort their columns in the "natural" order
// TODO this is inconvenient for direct users of StorageProxy
if (reverseOrder)
Collections.reverse(thriftColumns);
return thriftColumns;
}
private List<ColumnOrSuperColumn> thriftifySuperColumns(Collection<IColumn> columns, boolean reverseOrder, boolean isCounterCF)
{
if (isCounterCF)
return thriftifyCounterSuperColumns(columns, reverseOrder);
else
return thriftifySuperColumns(columns, reverseOrder);
}
private List<ColumnOrSuperColumn> thriftifySuperColumns(Collection<IColumn> columns, boolean reverseOrder)
{
ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>(columns.size());
for (IColumn column : columns)
{
List<Column> subcolumns = thriftifySubColumns(column.getSubColumns());
if (subcolumns.isEmpty())
{
continue;
}
SuperColumn superColumn = new SuperColumn(column.name(), subcolumns);
thriftSuperColumns.add(new ColumnOrSuperColumn().setSuper_column(superColumn));
}
if (reverseOrder)
Collections.reverse(thriftSuperColumns);
return thriftSuperColumns;
}
private List<ColumnOrSuperColumn> thriftifyCounterSuperColumns(Collection<IColumn> columns, boolean reverseOrder)
{
ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>(columns.size());
for (IColumn column : columns)
{
List<CounterColumn> subcolumns = thriftifyCounterSubColumns(column.getSubColumns());
if (subcolumns.isEmpty())
{
continue;
}
CounterSuperColumn superColumn = new CounterSuperColumn(column.name(), subcolumns);
thriftSuperColumns.add(new ColumnOrSuperColumn().setCounter_super_column(superColumn));
}
if (reverseOrder)
Collections.reverse(thriftSuperColumns);
return thriftSuperColumns;
}
private Map<ByteBuffer, List<ColumnOrSuperColumn>> getSlice(List<ReadCommand> commands, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
Map<DecoratedKey, ColumnFamily> columnFamilies = readColumnFamily(commands, consistency_level);
Map<ByteBuffer, List<ColumnOrSuperColumn>> columnFamiliesMap = new HashMap<ByteBuffer, List<ColumnOrSuperColumn>>();
for (ReadCommand command: commands)
{
ColumnFamily cf = columnFamilies.get(StorageService.getPartitioner().decorateKey(command.key));
boolean reverseOrder = command instanceof SliceFromReadCommand && ((SliceFromReadCommand)command).reversed;
List<ColumnOrSuperColumn> thriftifiedColumns = thriftifyColumnFamily(cf, command.queryPath.superColumnName != null, reverseOrder);
columnFamiliesMap.put(command.key, thriftifiedColumns);
}
return columnFamiliesMap;
}
private List<ColumnOrSuperColumn> thriftifyColumnFamily(ColumnFamily cf, boolean subcolumnsOnly, boolean reverseOrder)
{
if (cf == null || cf.getColumnsMap().size() == 0)
return EMPTY_COLUMNS;
if (subcolumnsOnly)
{
IColumn column = cf.getColumnsMap().values().iterator().next();
Collection<IColumn> subcolumns = column.getSubColumns();
if (subcolumns == null || subcolumns.isEmpty())
return EMPTY_COLUMNS;
else
return thriftifyColumns(subcolumns, reverseOrder);
}
if (cf.isSuper())
{
boolean isCounterCF = cf.metadata().getDefaultValidator().isCommutative();
return thriftifySuperColumns(cf.getSortedColumns(), reverseOrder, isCounterCF);
}
else
{
return thriftifyColumns(cf.getSortedColumns(), reverseOrder);
}
}
public List<ColumnOrSuperColumn> get_slice(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
logger.debug("get_slice");
state().hasColumnFamilyAccess(column_parent.column_family, Permission.READ);
return multigetSliceInternal(state().getKeyspace(), Collections.singletonList(key), column_parent, predicate, consistency_level).get(key);
}
public Map<ByteBuffer, List<ColumnOrSuperColumn>> multiget_slice(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
logger.debug("multiget_slice");
state().hasColumnFamilyAccess(column_parent.column_family, Permission.READ);
return multigetSliceInternal(state().getKeyspace(), keys, column_parent, predicate, consistency_level);
}
private Map<ByteBuffer, List<ColumnOrSuperColumn>> multigetSliceInternal(String keyspace, List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_parent.column_family);
ThriftValidation.validateColumnParent(metadata, column_parent);
ThriftValidation.validatePredicate(metadata, column_parent, predicate);
ThriftValidation.validateConsistencyLevel(keyspace, consistency_level);
List<ReadCommand> commands = new ArrayList<ReadCommand>();
if (predicate.column_names != null)
{
for (ByteBuffer key: keys)
{
ThriftValidation.validateKey(metadata, key);
commands.add(new SliceByNamesReadCommand(keyspace, key, column_parent, predicate.column_names));
}
}
else
{
SliceRange range = predicate.slice_range;
for (ByteBuffer key: keys)
{
ThriftValidation.validateKey(metadata, key);
commands.add(new SliceFromReadCommand(keyspace, key, column_parent, range.start, range.finish, range.reversed, range.count));
}
}
return getSlice(commands, consistency_level);
}
private ColumnOrSuperColumn internal_get(ByteBuffer key, ColumnPath column_path, ConsistencyLevel consistency_level)
throws InvalidRequestException, NotFoundException, UnavailableException, TimedOutException
{
state().hasColumnFamilyAccess(column_path.column_family, Permission.READ);
String keyspace = state().getKeyspace();
CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_path.column_family);
ThriftValidation.validateColumnPath(metadata, column_path);
ThriftValidation.validateConsistencyLevel(keyspace, consistency_level);
QueryPath path = new QueryPath(column_path.column_family, column_path.column == null ? null : column_path.super_column);
List<ByteBuffer> nameAsList = Arrays.asList(column_path.column == null ? column_path.super_column : column_path.column);
ThriftValidation.validateKey(metadata, key);
ReadCommand command = new SliceByNamesReadCommand(keyspace, key, path, nameAsList);
Map<DecoratedKey, ColumnFamily> cfamilies = readColumnFamily(Arrays.asList(command), consistency_level);
ColumnFamily cf = cfamilies.get(StorageService.getPartitioner().decorateKey(command.key));
if (cf == null)
throw new NotFoundException();
List<ColumnOrSuperColumn> tcolumns = thriftifyColumnFamily(cf, command.queryPath.superColumnName != null, false);
if (tcolumns.isEmpty())
throw new NotFoundException();
assert tcolumns.size() == 1;
return tcolumns.get(0);
}
public ColumnOrSuperColumn get(ByteBuffer key, ColumnPath column_path, ConsistencyLevel consistency_level)
throws InvalidRequestException, NotFoundException, UnavailableException, TimedOutException
{
logger.debug("get");
return internal_get(key, column_path, consistency_level);
}
public int get_count(ByteBuffer key, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
logger.debug("get_count");
state().hasColumnFamilyAccess(column_parent.column_family, Permission.READ);
return get_slice(key, column_parent, predicate, consistency_level).size();
}
public Map<ByteBuffer, Integer> multiget_count(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
logger.debug("multiget_count");
state().hasColumnFamilyAccess(column_parent.column_family, Permission.READ);
String keyspace = state().getKeyspace();
Map<ByteBuffer, Integer> counts = new HashMap<ByteBuffer, Integer>();
Map<ByteBuffer, List<ColumnOrSuperColumn>> columnFamiliesMap = multigetSliceInternal(keyspace, keys, column_parent, predicate, consistency_level);
for (Map.Entry<ByteBuffer, List<ColumnOrSuperColumn>> cf : columnFamiliesMap.entrySet()) {
counts.put(cf.getKey(), cf.getValue().size());
}
return counts;
}
private void internal_insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
state().hasColumnFamilyAccess(column_parent.column_family, Permission.WRITE);
CFMetaData metadata = ThriftValidation.validateColumnFamily(state().getKeyspace(), column_parent.column_family, false);
ThriftValidation.validateKey(metadata, key);
ThriftValidation.validateColumnParent(metadata, column_parent);
// SuperColumn field is usually optional, but not when we're inserting
if (metadata.cfType == ColumnFamilyType.Super && column_parent.super_column == null)
{
throw new InvalidRequestException("missing mandatory super column name for super CF " + column_parent.column_family);
}
ThriftValidation.validateColumnNames(metadata, column_parent, Arrays.asList(column.name));
ThriftValidation.validateColumnData(metadata, column);
RowMutation rm = new RowMutation(state().getKeyspace(), key);
try
{
rm.add(new QueryPath(column_parent.column_family, column_parent.super_column, column.name), column.value, column.timestamp, column.ttl);
}
catch (MarshalException e)
{
throw new InvalidRequestException(e.getMessage());
}
doInsert(consistency_level, Arrays.asList(rm));
}
public void insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
logger.debug("insert");
internal_insert(key, column_parent, column, consistency_level);
}
private void internal_batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
List<String> cfamsSeen = new ArrayList<String>();
List<IMutation> rowMutations = new ArrayList<IMutation>();
String keyspace = state().getKeyspace();
for (Map.Entry<ByteBuffer, Map<String, List<Mutation>>> mutationEntry: mutation_map.entrySet())
{
ByteBuffer key = mutationEntry.getKey();
// We need to separate row mutation for standard cf and counter cf (that will be encapsulated in a
// CounterMutation) because it doesn't follow the same code path
RowMutation rmStandard = null;
RowMutation rmCounter = null;
Map<String, List<Mutation>> columnFamilyToMutations = mutationEntry.getValue();
for (Map.Entry<String, List<Mutation>> columnFamilyMutations : columnFamilyToMutations.entrySet())
{
String cfName = columnFamilyMutations.getKey();
// Avoid unneeded authorizations
if (!(cfamsSeen.contains(cfName)))
{
state().hasColumnFamilyAccess(cfName, Permission.WRITE);
cfamsSeen.add(cfName);
}
CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, cfName);
ThriftValidation.validateKey(metadata, key);
RowMutation rm;
if (metadata.getDefaultValidator().isCommutative())
{
ThriftValidation.validateCommutativeForWrite(metadata, consistency_level);
rmCounter = rmCounter == null ? new RowMutation(keyspace, key) : rmCounter;
rm = rmCounter;
}
else
{
rmStandard = rmStandard == null ? new RowMutation(keyspace, key) : rmStandard;
rm = rmStandard;
}
for (Mutation mutation : columnFamilyMutations.getValue())
{
ThriftValidation.validateMutation(metadata, mutation);
if (mutation.deletion != null)
{
rm.deleteColumnOrSuperColumn(cfName, mutation.deletion);
}
if (mutation.column_or_supercolumn != null)
{
rm.addColumnOrSuperColumn(cfName, mutation.column_or_supercolumn);
}
}
}
if (rmStandard != null && !rmStandard.isEmpty())
rowMutations.add(rmStandard);
if (rmCounter != null && !rmCounter.isEmpty())
rowMutations.add(new org.apache.cassandra.db.CounterMutation(rmCounter, consistency_level));
}
doInsert(consistency_level, rowMutations);
}
public void batch_mutate(Map<ByteBuffer,Map<String,List<Mutation>>> mutation_map, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
logger.debug("batch_mutate");
internal_batch_mutate(mutation_map, consistency_level);
}
private void internal_remove(ByteBuffer key, ColumnPath column_path, long timestamp, ConsistencyLevel consistency_level, boolean isCommutativeOp)
throws InvalidRequestException, UnavailableException, TimedOutException
{
state().hasColumnFamilyAccess(column_path.column_family, Permission.WRITE);
CFMetaData metadata = ThriftValidation.validateColumnFamily(state().getKeyspace(), column_path.column_family, isCommutativeOp);
ThriftValidation.validateKey(metadata, key);
ThriftValidation.validateColumnPathOrParent(metadata, column_path);
if (isCommutativeOp)
ThriftValidation.validateCommutativeForWrite(metadata, consistency_level);
RowMutation rm = new RowMutation(state().getKeyspace(), key);
rm.delete(new QueryPath(column_path), timestamp);
if (isCommutativeOp)
doInsert(consistency_level, Arrays.asList(new CounterMutation(rm, consistency_level)));
else
doInsert(consistency_level, Arrays.asList(rm));
}
public void remove(ByteBuffer key, ColumnPath column_path, long timestamp, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TimedOutException
{
logger.debug("remove");
internal_remove(key, column_path, timestamp, consistency_level, false);
}
private void doInsert(ConsistencyLevel consistency_level, List<? extends IMutation> mutations) throws UnavailableException, TimedOutException, InvalidRequestException
{
ThriftValidation.validateConsistencyLevel(state().getKeyspace(), consistency_level);
try
{
schedule();
try
{
if (!mutations.isEmpty())
StorageProxy.mutate(mutations, consistency_level);
}
catch (TimeoutException e)
{
throw new TimedOutException();
}
}
finally
{
release();
}
}
public KsDef describe_keyspace(String table) throws NotFoundException, InvalidRequestException
{
state().hasKeyspaceListAccess(Permission.READ);
KSMetaData ksm = DatabaseDescriptor.getTableDefinition(table);
if (ksm == null)
throw new NotFoundException();
return KSMetaData.toThrift(ksm);
}
public List<KeySlice> get_range_slices(ColumnParent column_parent, SlicePredicate predicate, KeyRange range, ConsistencyLevel consistency_level)
throws InvalidRequestException, UnavailableException, TException, TimedOutException
{
logger.debug("range_slice");
String keyspace = state().getKeyspace();
state().hasColumnFamilyAccess(column_parent.column_family, Permission.READ);
CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_parent.column_family);
ThriftValidation.validateColumnParent(metadata, column_parent);
ThriftValidation.validatePredicate(metadata, column_parent, predicate);
ThriftValidation.validateKeyRange(range);
ThriftValidation.validateConsistencyLevel(keyspace, consistency_level);
List<Row> rows;
try
{
IPartitioner p = StorageService.getPartitioner();
AbstractBounds bounds;
if (range.start_key == null)
{
Token.TokenFactory tokenFactory = p.getTokenFactory();
Token left = tokenFactory.fromString(range.start_token);
Token right = tokenFactory.fromString(range.end_token);
bounds = new Range(left, right);
}
else
{
bounds = new Bounds(p.getToken(range.start_key), p.getToken(range.end_key));
}
try
{
schedule();
rows = StorageProxy.getRangeSlice(new RangeSliceCommand(keyspace, column_parent, predicate, bounds, range.count), consistency_level);
}
finally
{
release();
}
assert rows != null;
}
catch (TimeoutException e)
{
throw new TimedOutException();
}
catch (IOException e)
{
throw new RuntimeException(e);
}
return thriftifyKeySlices(rows, column_parent, predicate);
}
private List<KeySlice> thriftifyKeySlices(List<Row> rows, ColumnParent column_parent, SlicePredicate predicate)
{
List<KeySlice> keySlices = new ArrayList<KeySlice>(rows.size());
boolean reversed = predicate.slice_range != null && predicate.slice_range.reversed;
for (Row row : rows)
{
List<ColumnOrSuperColumn> thriftifiedColumns = thriftifyColumnFamily(row.cf, column_parent.super_column != null, reversed);
keySlices.add(new KeySlice(row.key.key, thriftifiedColumns));
}
return keySlices;
}
public List<KeySlice> get_indexed_slices(ColumnParent column_parent, IndexClause index_clause, SlicePredicate column_predicate, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException, TException
{
logger.debug("scan");
state().hasColumnFamilyAccess(column_parent.column_family, Permission.READ);
String keyspace = state().getKeyspace();
CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_parent.column_family, false);
ThriftValidation.validateColumnParent(metadata, column_parent);
ThriftValidation.validatePredicate(metadata, column_parent, column_predicate);
ThriftValidation.validateIndexClauses(metadata, index_clause);
ThriftValidation.validateConsistencyLevel(keyspace, consistency_level);
List<Row> rows;
try
{
rows = StorageProxy.scan(keyspace, column_parent.column_family, index_clause, column_predicate, consistency_level);
}
catch (IOException e)
{
throw new RuntimeException(e);
}
catch (TimeoutException e)
{
throw new TimedOutException();
}
return thriftifyKeySlices(rows, column_parent, column_predicate);
}
public List<KsDef> describe_keyspaces() throws TException, InvalidRequestException
{
state().hasKeyspaceListAccess(Permission.READ);
Set<String> keyspaces = DatabaseDescriptor.getTables();
List<KsDef> ksset = new ArrayList<KsDef>();
for (String ks : keyspaces)
{
try
{
ksset.add(describe_keyspace(ks));
}
catch (NotFoundException nfe)
{
logger.info("Failed to find metadata for keyspace '" + ks + "'. Continuing... ");
}
}
return ksset;
}
public String describe_cluster_name() throws TException
{
return DatabaseDescriptor.getClusterName();
}
public String describe_version() throws TException
{
return Constants.VERSION;
}
public List<TokenRange> describe_ring(String keyspace)throws InvalidRequestException
{
if (keyspace == null || !DatabaseDescriptor.getNonSystemTables().contains(keyspace))
throw new InvalidRequestException("There is no ring for the keyspace: " + keyspace);
List<TokenRange> ranges = new ArrayList<TokenRange>();
Token.TokenFactory tf = StorageService.getPartitioner().getTokenFactory();
for (Map.Entry<Range, List<String>> entry : StorageService.instance.getRangeToEndpointMap(keyspace).entrySet())
{
Range range = entry.getKey();
List<String> endpoints = entry.getValue();
ranges.add(new TokenRange(tf.toString(range.left), tf.toString(range.right), endpoints));
}
return ranges;
}
public String describe_partitioner() throws TException
{
return StorageService.getPartitioner().getClass().getName();
}
public String describe_snitch() throws TException
{
if (DatabaseDescriptor.getEndpointSnitch() instanceof DynamicEndpointSnitch)
return ((DynamicEndpointSnitch)DatabaseDescriptor.getEndpointSnitch()).subsnitch.getClass().getName();
return DatabaseDescriptor.getEndpointSnitch().getClass().getName();
}
public List<String> describe_splits(String cfName, String start_token, String end_token, int keys_per_split)
throws TException, InvalidRequestException
{
// TODO: add keyspace authorization call post CASSANDRA-1425
Token.TokenFactory tf = StorageService.getPartitioner().getTokenFactory();
List<Token> tokens = StorageService.instance.getSplits(state().getKeyspace(), cfName, new Range(tf.fromString(start_token), tf.fromString(end_token)), keys_per_split);
List<String> splits = new ArrayList<String>(tokens.size());
for (Token token : tokens)
{
splits.add(tf.toString(token));
}
return splits;
}
public void login(AuthenticationRequest auth_request) throws AuthenticationException, AuthorizationException, TException
{
state().login(auth_request.getCredentials());
}
/**
* Schedule the current thread for access to the required services
*/
private void schedule()
{
requestScheduler.queue(Thread.currentThread(), state().getSchedulingValue());
}
/**
* Release count for the used up resources
*/
private void release()
{
requestScheduler.release();
}
// helper method to apply migration on the migration stage. typical migration failures will throw an
// InvalidRequestException. atypical failures will throw a RuntimeException.
private static void applyMigrationOnStage(final Migration m)
{
Future f = StageManager.getStage(Stage.MIGRATION).submit(new Callable()
{
public Object call() throws Exception
{
m.apply();
m.announce();
return null;
}
});
try
{
f.get();
}
catch (InterruptedException e)
{
throw new AssertionError(e);
}
catch (ExecutionException e)
{
throw new RuntimeException(e);
}
}
public synchronized String system_add_column_family(CfDef cf_def)
throws InvalidRequestException, SchemaDisagreementException, TException
{
logger.debug("add_column_family");
state().hasColumnFamilyListAccess(Permission.WRITE);
ThriftValidation.validateCfDef(cf_def);
validateSchemaAgreement();
try
{
applyMigrationOnStage(new AddColumnFamily(CFMetaData.fromThrift(cf_def)));
return DatabaseDescriptor.getDefsVersion().toString();
}
catch (ConfigurationException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
catch (IOException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
}
public synchronized String system_drop_column_family(String column_family)
throws InvalidRequestException, SchemaDisagreementException, TException
{
logger.debug("drop_column_family");
state().hasColumnFamilyListAccess(Permission.WRITE);
validateSchemaAgreement();
try
{
applyMigrationOnStage(new DropColumnFamily(state().getKeyspace(), column_family));
return DatabaseDescriptor.getDefsVersion().toString();
}
catch (ConfigurationException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
catch (IOException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
}
public synchronized String system_add_keyspace(KsDef ks_def)
throws InvalidRequestException, SchemaDisagreementException, TException
{
logger.debug("add_keyspace");
state().hasKeyspaceListAccess(Permission.WRITE);
validateSchemaAgreement();
// generate a meaningful error if the user setup keyspace and/or column definition incorrectly
for (CfDef cf : ks_def.cf_defs)
{
if (!cf.getKeyspace().equals(ks_def.getName()))
{
throw new InvalidRequestException("CsDef (" + cf.getName() +") had a keyspace definition that did not match KsDef");
}
}
try
{
Collection<CFMetaData> cfDefs = new ArrayList<CFMetaData>(ks_def.cf_defs.size());
for (CfDef cfDef : ks_def.cf_defs)
{
ThriftValidation.validateCfDef(cfDef);
cfDefs.add(CFMetaData.fromThrift(cfDef));
}
ThriftValidation.validateKsDef(ks_def);
applyMigrationOnStage(new AddKeyspace(KSMetaData.fromThrift(ks_def, cfDefs.toArray(new CFMetaData[cfDefs.size()]))));
return DatabaseDescriptor.getDefsVersion().toString();
}
catch (ConfigurationException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
catch (IOException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
}
public synchronized String system_drop_keyspace(String keyspace)
throws InvalidRequestException, SchemaDisagreementException, TException
{
logger.debug("drop_keyspace");
state().hasKeyspaceListAccess(Permission.WRITE);
validateSchemaAgreement();
try
{
applyMigrationOnStage(new DropKeyspace(keyspace));
return DatabaseDescriptor.getDefsVersion().toString();
}
catch (ConfigurationException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
catch (IOException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
}
/** update an existing keyspace, but do not allow column family modifications.
* @throws SchemaDisagreementException */
public synchronized String system_update_keyspace(KsDef ks_def)
throws InvalidRequestException, SchemaDisagreementException, TException
{
logger.debug("update_keyspace");
state().hasKeyspaceListAccess(Permission.WRITE);
ThriftValidation.validateTable(ks_def.name);
if (ks_def.getCf_defs() != null && ks_def.getCf_defs().size() > 0)
throw new InvalidRequestException("Keyspace update must not contain any column family definitions.");
validateSchemaAgreement();
try
{
ThriftValidation.validateKsDef(ks_def);
applyMigrationOnStage(new UpdateKeyspace(KSMetaData.fromThrift(ks_def)));
return DatabaseDescriptor.getDefsVersion().toString();
}
catch (ConfigurationException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
catch (IOException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
}
public synchronized String system_update_column_family(CfDef cf_def)
throws InvalidRequestException, SchemaDisagreementException, TException
{
logger.debug("update_column_family");
state().hasColumnFamilyListAccess(Permission.WRITE);
ThriftValidation.validateCfDef(cf_def);
if (cf_def.keyspace == null || cf_def.name == null)
throw new InvalidRequestException("Keyspace and CF name must be set.");
CFMetaData oldCfm = DatabaseDescriptor.getCFMetaData(CFMetaData.getId(cf_def.keyspace, cf_def.name));
if (oldCfm == null)
throw new InvalidRequestException("Could not find column family definition to modify.");
validateSchemaAgreement();
try
{
// ideally, apply() would happen on the stage with the
CFMetaData.applyImplicitDefaults(cf_def);
UpdateColumnFamily update = new UpdateColumnFamily(CFMetaData.convertToAvro(cf_def));
applyMigrationOnStage(update);
return DatabaseDescriptor.getDefsVersion().toString();
}
catch (ConfigurationException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
catch (IOException e)
{
InvalidRequestException ex = new InvalidRequestException(e.getMessage());
ex.initCause(e);
throw ex;
}
}
private void validateSchemaAgreement() throws SchemaDisagreementException
{
// unreachable hosts don't count towards disagreement
Map<String, List<String>> versions = Maps.filterKeys(StorageProxy.describeSchemaVersions(),
Predicates.not(Predicates.equalTo(StorageProxy.UNREACHABLE)));
if (versions.size() > 1)
throw new SchemaDisagreementException();
}
public void truncate(String cfname) throws InvalidRequestException, UnavailableException, TException
{
logger.debug("truncating {} in {}", cfname, state().getKeyspace());
state().hasColumnFamilyAccess(cfname, Permission.WRITE);
try
{