This repository has been archived by the owner on Oct 30, 2023. It is now read-only.
/
GiraphConstants.java
1338 lines (1183 loc) · 60 KB
/
GiraphConstants.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.giraph.conf;
import org.apache.giraph.aggregators.AggregatorWriter;
import org.apache.giraph.aggregators.TextAggregatorWriter;
import org.apache.giraph.bsp.BspOutputFormat;
import org.apache.giraph.bsp.checkpoints.CheckpointSupportedChecker;
import org.apache.giraph.bsp.checkpoints.DefaultCheckpointSupportedChecker;
import org.apache.giraph.combiner.MessageCombiner;
import org.apache.giraph.comm.messages.InMemoryMessageStoreFactory;
import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
import org.apache.giraph.comm.messages.MessageStoreFactory;
import org.apache.giraph.edge.ByteArrayEdges;
import org.apache.giraph.edge.DefaultCreateSourceVertexCallback;
import org.apache.giraph.edge.CreateSourceVertexCallback;
import org.apache.giraph.edge.EdgeStoreFactory;
import org.apache.giraph.edge.InMemoryEdgeStoreFactory;
import org.apache.giraph.edge.OutEdges;
import org.apache.giraph.factories.ComputationFactory;
import org.apache.giraph.factories.DefaultComputationFactory;
import org.apache.giraph.factories.DefaultEdgeValueFactory;
import org.apache.giraph.factories.DefaultInputOutEdgesFactory;
import org.apache.giraph.factories.DefaultMessageValueFactory;
import org.apache.giraph.factories.DefaultOutEdgesFactory;
import org.apache.giraph.factories.DefaultVertexIdFactory;
import org.apache.giraph.factories.DefaultVertexValueFactory;
import org.apache.giraph.factories.EdgeValueFactory;
import org.apache.giraph.factories.MessageValueFactory;
import org.apache.giraph.factories.OutEdgesFactory;
import org.apache.giraph.factories.VertexIdFactory;
import org.apache.giraph.factories.VertexValueFactory;
import org.apache.giraph.graph.Computation;
import org.apache.giraph.graph.DefaultVertex;
import org.apache.giraph.graph.DefaultVertexResolver;
import org.apache.giraph.graph.DefaultVertexValueCombiner;
import org.apache.giraph.graph.JobProgressTrackerClient;
import org.apache.giraph.graph.Language;
import org.apache.giraph.graph.MapperObserver;
import org.apache.giraph.graph.RetryableJobProgressTrackerClient;
import org.apache.giraph.graph.Vertex;
import org.apache.giraph.graph.VertexResolver;
import org.apache.giraph.graph.VertexValueCombiner;
import org.apache.giraph.io.EdgeInputFormat;
import org.apache.giraph.io.EdgeOutputFormat;
import org.apache.giraph.io.MappingInputFormat;
import org.apache.giraph.io.VertexInputFormat;
import org.apache.giraph.io.VertexOutputFormat;
import org.apache.giraph.io.filters.DefaultEdgeInputFilter;
import org.apache.giraph.io.filters.DefaultVertexInputFilter;
import org.apache.giraph.io.filters.EdgeInputFilter;
import org.apache.giraph.io.filters.VertexInputFilter;
import org.apache.giraph.job.DefaultGiraphJobRetryChecker;
import org.apache.giraph.job.DefaultJobObserver;
import org.apache.giraph.job.DefaultJobProgressTrackerService;
import org.apache.giraph.job.GiraphJobObserver;
import org.apache.giraph.job.GiraphJobRetryChecker;
import org.apache.giraph.job.HaltApplicationUtils;
import org.apache.giraph.job.JobProgressTrackerService;
import org.apache.giraph.mapping.MappingStore;
import org.apache.giraph.mapping.MappingStoreOps;
import org.apache.giraph.mapping.translate.TranslateEdge;
import org.apache.giraph.master.DefaultMasterCompute;
import org.apache.giraph.master.MasterCompute;
import org.apache.giraph.master.MasterObserver;
import org.apache.giraph.ooc.persistence.OutOfCoreDataAccessor;
import org.apache.giraph.ooc.persistence.LocalDiskDataAccessor;
import org.apache.giraph.ooc.policy.MemoryEstimatorOracle;
import org.apache.giraph.ooc.policy.OutOfCoreOracle;
import org.apache.giraph.partition.GraphPartitionerFactory;
import org.apache.giraph.partition.HashPartitionerFactory;
import org.apache.giraph.partition.Partition;
import org.apache.giraph.partition.SimplePartition;
import org.apache.giraph.utils.GcObserver;
import org.apache.giraph.worker.DefaultWorkerContext;
import org.apache.giraph.worker.WorkerContext;
import org.apache.giraph.worker.WorkerObserver;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapreduce.OutputFormat;
import static java.util.concurrent.TimeUnit.HOURS;
import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.concurrent.TimeUnit.SECONDS;
/**
* Constants used all over Giraph for configuration.
*/
// CHECKSTYLE: stop InterfaceIsTypeCheck
public interface GiraphConstants {
/** 1KB in bytes */
int ONE_KB = 1024;
/** 1MB in bytes */
int ONE_MB = 1024 * 1024;
/** Mapping related information */
ClassConfOption<MappingStore> MAPPING_STORE_CLASS =
ClassConfOption.create("giraph.mappingStoreClass", null,
MappingStore.class, "MappingStore Class");
/** Class to use for performing read operations on mapping store */
ClassConfOption<MappingStoreOps> MAPPING_STORE_OPS_CLASS =
ClassConfOption.create("giraph.mappingStoreOpsClass", null,
MappingStoreOps.class, "MappingStoreOps class");
/** Upper value of LongByteMappingStore */
IntConfOption LB_MAPPINGSTORE_UPPER =
new IntConfOption("giraph.lbMappingStoreUpper", -1,
"'upper' value used by lbmappingstore");
/** Lower value of LongByteMappingStore */
IntConfOption LB_MAPPINGSTORE_LOWER =
new IntConfOption("giraph.lbMappingStoreLower", -1,
"'lower' value used by lbMappingstore");
/** Class used to conduct expensive edge translation during vertex input */
ClassConfOption EDGE_TRANSLATION_CLASS =
ClassConfOption.create("giraph.edgeTranslationClass", null,
TranslateEdge.class, "Class used to conduct expensive edge " +
"translation during vertex input phase");
/** Computation class - required */
ClassConfOption<Computation> COMPUTATION_CLASS =
ClassConfOption.create("giraph.computationClass", null,
Computation.class, "Computation class - required");
/** Computation factory class - optional */
ClassConfOption<ComputationFactory> COMPUTATION_FACTORY_CLASS =
ClassConfOption.create("giraph.computation.factory.class",
DefaultComputationFactory.class, ComputationFactory.class,
"Computation factory class - optional");
/** TypesHolder, used if Computation not set - optional */
ClassConfOption<TypesHolder> TYPES_HOLDER_CLASS =
ClassConfOption.create("giraph.typesHolder", null,
TypesHolder.class,
"TypesHolder, used if Computation not set - optional");
/** Edge Store Factory */
ClassConfOption<EdgeStoreFactory> EDGE_STORE_FACTORY_CLASS =
ClassConfOption.create("giraph.edgeStoreFactoryClass",
InMemoryEdgeStoreFactory.class,
EdgeStoreFactory.class,
"Edge Store Factory class to use for creating edgeStore");
/** Message Store Factory */
ClassConfOption<MessageStoreFactory> MESSAGE_STORE_FACTORY_CLASS =
ClassConfOption.create("giraph.messageStoreFactoryClass",
InMemoryMessageStoreFactory.class,
MessageStoreFactory.class,
"Message Store Factory Class that is to be used");
/** Language user's graph types are implemented in */
PerGraphTypeEnumConfOption<Language> GRAPH_TYPE_LANGUAGES =
PerGraphTypeEnumConfOption.create("giraph.types.language",
Language.class, Language.JAVA,
"Language user graph types (IVEMM) are implemented in");
/** Whether user graph types need wrappers */
PerGraphTypeBooleanConfOption GRAPH_TYPES_NEEDS_WRAPPERS =
new PerGraphTypeBooleanConfOption("giraph.jython.type.wrappers",
false, "Whether user graph types (IVEMM) need Jython wrappers");
/** Vertex id factory class - optional */
ClassConfOption<VertexIdFactory> VERTEX_ID_FACTORY_CLASS =
ClassConfOption.create("giraph.vertexIdFactoryClass",
DefaultVertexIdFactory.class, VertexIdFactory.class,
"Vertex ID factory class - optional");
/** Vertex value factory class - optional */
ClassConfOption<VertexValueFactory> VERTEX_VALUE_FACTORY_CLASS =
ClassConfOption.create("giraph.vertexValueFactoryClass",
DefaultVertexValueFactory.class, VertexValueFactory.class,
"Vertex value factory class - optional");
/** Edge value factory class - optional */
ClassConfOption<EdgeValueFactory> EDGE_VALUE_FACTORY_CLASS =
ClassConfOption.create("giraph.edgeValueFactoryClass",
DefaultEdgeValueFactory.class, EdgeValueFactory.class,
"Edge value factory class - optional");
/** Outgoing message value factory class - optional */
ClassConfOption<MessageValueFactory>
OUTGOING_MESSAGE_VALUE_FACTORY_CLASS =
ClassConfOption.create("giraph.outgoingMessageValueFactoryClass",
DefaultMessageValueFactory.class, MessageValueFactory.class,
"Outgoing message value factory class - optional");
/** Vertex edges class - optional */
ClassConfOption<OutEdges> VERTEX_EDGES_CLASS =
ClassConfOption.create("giraph.outEdgesClass", ByteArrayEdges.class,
OutEdges.class, "Vertex edges class - optional");
/** Vertex edges class to be used during edge input only - optional */
ClassConfOption<OutEdges> INPUT_VERTEX_EDGES_CLASS =
ClassConfOption.create("giraph.inputOutEdgesClass",
ByteArrayEdges.class, OutEdges.class,
"Vertex edges class to be used during edge input only - optional");
/** OutEdges factory class - optional */
ClassConfOption<OutEdgesFactory> VERTEX_EDGES_FACTORY_CLASS =
ClassConfOption.create("giraph.outEdgesFactoryClass",
DefaultOutEdgesFactory.class, OutEdgesFactory.class,
"OutEdges factory class - optional");
/** OutEdges for input factory class - optional */
ClassConfOption<OutEdgesFactory> INPUT_VERTEX_EDGES_FACTORY_CLASS =
ClassConfOption.create("giraph.inputOutEdgesFactoryClass",
DefaultInputOutEdgesFactory.class, OutEdgesFactory.class,
"OutEdges for input factory class - optional");
/** Class for Master - optional */
ClassConfOption<MasterCompute> MASTER_COMPUTE_CLASS =
ClassConfOption.create("giraph.masterComputeClass",
DefaultMasterCompute.class, MasterCompute.class,
"Class for Master - optional");
/** Classes for Master Observer - optional */
ClassConfOption<MasterObserver> MASTER_OBSERVER_CLASSES =
ClassConfOption.create("giraph.master.observers",
null, MasterObserver.class, "Classes for Master Observer - optional");
/** Classes for Worker Observer - optional */
ClassConfOption<WorkerObserver> WORKER_OBSERVER_CLASSES =
ClassConfOption.create("giraph.worker.observers", null,
WorkerObserver.class, "Classes for Worker Observer - optional");
/** Classes for Mapper Observer - optional */
ClassConfOption<MapperObserver> MAPPER_OBSERVER_CLASSES =
ClassConfOption.create("giraph.mapper.observers", null,
MapperObserver.class, "Classes for Mapper Observer - optional");
/** Classes for GC Observer - optional */
ClassConfOption<GcObserver> GC_OBSERVER_CLASSES =
ClassConfOption.create("giraph.gc.observers", null,
GcObserver.class, "Classes for GC oObserver - optional");
/** Message combiner class - optional */
ClassConfOption<MessageCombiner> MESSAGE_COMBINER_CLASS =
ClassConfOption.create("giraph.messageCombinerClass", null,
MessageCombiner.class, "Message combiner class - optional");
/** Vertex resolver class - optional */
ClassConfOption<VertexResolver> VERTEX_RESOLVER_CLASS =
ClassConfOption.create("giraph.vertexResolverClass",
DefaultVertexResolver.class, VertexResolver.class,
"Vertex resolver class - optional");
/** Vertex value combiner class - optional */
ClassConfOption<VertexValueCombiner> VERTEX_VALUE_COMBINER_CLASS =
ClassConfOption.create("giraph.vertexValueCombinerClass",
DefaultVertexValueCombiner.class, VertexValueCombiner.class,
"Vertex value combiner class - optional");
/** Which language computation is implemented in */
EnumConfOption<Language> COMPUTATION_LANGUAGE =
EnumConfOption.create("giraph.computation.language",
Language.class, Language.JAVA,
"Which language computation is implemented in");
/**
* Option of whether to create vertexes that were not existent before but
* received messages
*/
BooleanConfOption RESOLVER_CREATE_VERTEX_ON_MSGS =
new BooleanConfOption("giraph.vertex.resolver.create.on.msgs", true,
"Option of whether to create vertexes that were not existent " +
"before but received messages");
/** Graph partitioner factory class - optional */
ClassConfOption<GraphPartitionerFactory> GRAPH_PARTITIONER_FACTORY_CLASS =
ClassConfOption.create("giraph.graphPartitionerFactoryClass",
HashPartitionerFactory.class, GraphPartitionerFactory.class,
"Graph partitioner factory class - optional");
/** Observer class to watch over job status - optional */
ClassConfOption<GiraphJobObserver> JOB_OBSERVER_CLASS =
ClassConfOption.create("giraph.jobObserverClass",
DefaultJobObserver.class, GiraphJobObserver.class,
"Observer class to watch over job status - optional");
/** Observer class to watch over job status - optional */
ClassConfOption<GiraphJobRetryChecker> JOB_RETRY_CHECKER_CLASS =
ClassConfOption.create("giraph.jobRetryCheckerClass",
DefaultGiraphJobRetryChecker.class, GiraphJobRetryChecker.class,
"Class which decides whether a failed job should be retried - " +
"optional");
/**
* Maximum allowed time for job to run after getting all resources before it
* will be killed, in milliseconds (-1 if it has no limit)
*/
LongConfOption MAX_ALLOWED_JOB_TIME_MS =
new LongConfOption("giraph.maxAllowedJobTimeMilliseconds", -1,
"Maximum allowed time for job to run after getting all resources " +
"before it will be killed, in milliseconds " +
"(-1 if it has no limit)");
// At least one of the input format classes is required.
/** VertexInputFormat class */
ClassConfOption<VertexInputFormat> VERTEX_INPUT_FORMAT_CLASS =
ClassConfOption.create("giraph.vertexInputFormatClass", null,
VertexInputFormat.class, "VertexInputFormat class (at least " +
"one of the input format classes is required)");
/** EdgeInputFormat class */
ClassConfOption<EdgeInputFormat> EDGE_INPUT_FORMAT_CLASS =
ClassConfOption.create("giraph.edgeInputFormatClass", null,
EdgeInputFormat.class, "EdgeInputFormat class");
/** MappingInputFormat class */
ClassConfOption<MappingInputFormat> MAPPING_INPUT_FORMAT_CLASS =
ClassConfOption.create("giraph.mappingInputFormatClass", null,
MappingInputFormat.class, "MappingInputFormat class");
/** EdgeInputFilter class */
ClassConfOption<EdgeInputFilter> EDGE_INPUT_FILTER_CLASS =
ClassConfOption.create("giraph.edgeInputFilterClass",
DefaultEdgeInputFilter.class, EdgeInputFilter.class,
"EdgeInputFilter class");
/** VertexInputFilter class */
ClassConfOption<VertexInputFilter> VERTEX_INPUT_FILTER_CLASS =
ClassConfOption.create("giraph.vertexInputFilterClass",
DefaultVertexInputFilter.class, VertexInputFilter.class,
"VertexInputFilter class");
/** Vertex class */
ClassConfOption<Vertex> VERTEX_CLASS =
ClassConfOption.create("giraph.vertexClass",
DefaultVertex.class, Vertex.class,
"Vertex class");
/** VertexOutputFormat class */
ClassConfOption<VertexOutputFormat> VERTEX_OUTPUT_FORMAT_CLASS =
ClassConfOption.create("giraph.vertexOutputFormatClass", null,
VertexOutputFormat.class, "VertexOutputFormat class");
/** EdgeOutputFormat sub-directory */
StrConfOption VERTEX_OUTPUT_FORMAT_SUBDIR =
new StrConfOption("giraph.vertex.output.subdir", "",
"VertexOutputFormat sub-directory");
/** EdgeOutputFormat class */
ClassConfOption<EdgeOutputFormat> EDGE_OUTPUT_FORMAT_CLASS =
ClassConfOption.create("giraph.edgeOutputFormatClass", null,
EdgeOutputFormat.class, "EdgeOutputFormat class");
/** EdgeOutputFormat sub-directory */
StrConfOption EDGE_OUTPUT_FORMAT_SUBDIR =
new StrConfOption("giraph.edge.output.subdir", "",
"EdgeOutputFormat sub-directory");
/** GiraphTextOuputFormat Separator */
StrConfOption GIRAPH_TEXT_OUTPUT_FORMAT_SEPARATOR =
new StrConfOption("giraph.textoutputformat.separator", "\t",
"GiraphTextOuputFormat Separator");
/** Reverse values in the output */
BooleanConfOption GIRAPH_TEXT_OUTPUT_FORMAT_REVERSE =
new BooleanConfOption("giraph.textoutputformat.reverse", false,
"Reverse values in the output");
/**
* If you use this option, instead of having saving vertices in the end of
* application, saveVertex will be called right after each vertex.compute()
* is called.
* NOTE: This feature doesn't work well with checkpointing - if you restart
* from a checkpoint you won't have any output from previous supersteps.
*/
BooleanConfOption DO_OUTPUT_DURING_COMPUTATION =
new BooleanConfOption("giraph.doOutputDuringComputation", false,
"If you use this option, instead of having saving vertices in the " +
"end of application, saveVertex will be called right after each " +
"vertex.compute() is called." +
"NOTE: This feature doesn't work well with checkpointing - if you " +
"restart from a checkpoint you won't have any ouptut from previous " +
"supresteps.");
/**
* Vertex output format thread-safe - if your VertexOutputFormat allows
* several vertexWriters to be created and written to in parallel,
* you should set this to true.
*/
BooleanConfOption VERTEX_OUTPUT_FORMAT_THREAD_SAFE =
new BooleanConfOption("giraph.vertexOutputFormatThreadSafe", false,
"Vertex output format thread-safe - if your VertexOutputFormat " +
"allows several vertexWriters to be created and written to in " +
"parallel, you should set this to true.");
/** Number of threads for writing output in the end of the application */
IntConfOption NUM_OUTPUT_THREADS =
new IntConfOption("giraph.numOutputThreads", 1,
"Number of threads for writing output in the end of the application");
/** conf key for comma-separated list of jars to export to YARN workers */
StrConfOption GIRAPH_YARN_LIBJARS =
new StrConfOption("giraph.yarn.libjars", "",
"conf key for comma-separated list of jars to export to YARN workers");
/** Name of the XML file that will export our Configuration to YARN workers */
String GIRAPH_YARN_CONF_FILE = "giraph-conf.xml";
/** Giraph default heap size for all tasks when running on YARN profile */
int GIRAPH_YARN_TASK_HEAP_MB_DEFAULT = 1024;
/** Name of Giraph property for user-configurable heap memory per worker */
IntConfOption GIRAPH_YARN_TASK_HEAP_MB = new IntConfOption(
"giraph.yarn.task.heap.mb", GIRAPH_YARN_TASK_HEAP_MB_DEFAULT,
"Name of Giraph property for user-configurable heap memory per worker");
/** Default priority level in YARN for our task containers */
int GIRAPH_YARN_PRIORITY = 10;
/** Is this a pure YARN job (i.e. no MapReduce layer managing Giraph tasks) */
BooleanConfOption IS_PURE_YARN_JOB =
new BooleanConfOption("giraph.pure.yarn.job", false,
"Is this a pure YARN job (i.e. no MapReduce layer managing Giraph " +
"tasks)");
/** Vertex index class */
ClassConfOption<WritableComparable> VERTEX_ID_CLASS =
ClassConfOption.create("giraph.vertexIdClass", null,
WritableComparable.class, "Vertex index class");
/** Vertex value class */
ClassConfOption<Writable> VERTEX_VALUE_CLASS =
ClassConfOption.create("giraph.vertexValueClass", null, Writable.class,
"Vertex value class");
/** Edge value class */
ClassConfOption<Writable> EDGE_VALUE_CLASS =
ClassConfOption.create("giraph.edgeValueClass", null, Writable.class,
"Edge value class");
/** Outgoing message value class */
ClassConfOption<Writable> OUTGOING_MESSAGE_VALUE_CLASS =
ClassConfOption.create("giraph.outgoingMessageValueClass", null,
Writable.class, "Outgoing message value class");
/** Worker context class */
ClassConfOption<WorkerContext> WORKER_CONTEXT_CLASS =
ClassConfOption.create("giraph.workerContextClass",
DefaultWorkerContext.class, WorkerContext.class,
"Worker contextclass");
/** AggregatorWriter class - optional */
ClassConfOption<AggregatorWriter> AGGREGATOR_WRITER_CLASS =
ClassConfOption.create("giraph.aggregatorWriterClass",
TextAggregatorWriter.class, AggregatorWriter.class,
"AggregatorWriter class - optional");
/** Partition class - optional */
ClassConfOption<Partition> PARTITION_CLASS =
ClassConfOption.create("giraph.partitionClass", SimplePartition.class,
Partition.class, "Partition class - optional");
/**
* Minimum number of simultaneous workers before this job can run (int)
*/
String MIN_WORKERS = "giraph.minWorkers";
/**
* Maximum number of simultaneous worker tasks started by this job (int).
*/
String MAX_WORKERS = "giraph.maxWorkers";
/**
* Separate the workers and the master tasks. This is required
* to support dynamic recovery. (boolean)
*/
BooleanConfOption SPLIT_MASTER_WORKER =
new BooleanConfOption("giraph.SplitMasterWorker", true,
"Separate the workers and the master tasks. This is required to " +
"support dynamic recovery. (boolean)");
/** Indicates whether this job is run in an internal unit test */
BooleanConfOption LOCAL_TEST_MODE =
new BooleanConfOption("giraph.localTestMode", false,
"Indicates whether this job is run in an internal unit test");
/** Override the Hadoop log level and set the desired log level. */
StrConfOption LOG_LEVEL = new StrConfOption("giraph.logLevel", "info",
"Override the Hadoop log level and set the desired log level.");
/** Use thread level debugging? */
BooleanConfOption LOG_THREAD_LAYOUT =
new BooleanConfOption("giraph.logThreadLayout", false,
"Use thread level debugging?");
/** Configuration key to enable jmap printing */
BooleanConfOption JMAP_ENABLE =
new BooleanConfOption("giraph.jmap.histo.enable", false,
"Configuration key to enable jmap printing");
/** Configuration key for msec to sleep between calls */
IntConfOption JMAP_SLEEP_MILLIS =
new IntConfOption("giraph.jmap.histo.msec", SECONDS.toMillis(30),
"Configuration key for msec to sleep between calls");
/** Configuration key for how many lines to print */
IntConfOption JMAP_PRINT_LINES =
new IntConfOption("giraph.jmap.histo.print_lines", 30,
"Configuration key for how many lines to print");
/**
* Configuration key for printing live objects only
* This option will trigger Full GC for every jmap dump
* and so can significantly hinder performance.
*/
BooleanConfOption JMAP_LIVE_ONLY =
new BooleanConfOption("giraph.jmap.histo.live", false,
"Only print live objects in jmap?");
/**
* Option used by ReactiveJMapHistoDumper to check for an imminent
* OOM in worker or master process
*/
IntConfOption MIN_FREE_MBS_ON_HEAP =
new IntConfOption("giraph.heap.minFreeMb", 128, "Option used by " +
"worker and master observers to check for imminent OOM exception");
/**
* Option can be used to enable reactively dumping jmap histo when
* OOM is imminent
*/
BooleanConfOption REACTIVE_JMAP_ENABLE =
new BooleanConfOption("giraph.heap.enableReactiveJmapDumping", false,
"Option to enable dumping jmap histogram reactively based on " +
"free memory on heap");
/**
* Minimum percent of the maximum number of workers that have responded
* in order to continue progressing. (float)
*/
FloatConfOption MIN_PERCENT_RESPONDED =
new FloatConfOption("giraph.minPercentResponded", 100.0f,
"Minimum percent of the maximum number of workers that have " +
"responded in order to continue progressing. (float)");
/** Enable the Metrics system */
BooleanConfOption METRICS_ENABLE =
new BooleanConfOption("giraph.metrics.enable", false,
"Enable the Metrics system");
/** Directory in HDFS to write master metrics to, instead of stderr */
StrConfOption METRICS_DIRECTORY =
new StrConfOption("giraph.metrics.directory", "",
"Directory in HDFS to write master metrics to, instead of stderr");
/**
* ZooKeeper comma-separated list (if not set,
* will start up ZooKeeper locally). Consider that after locally-starting
* zookeeper, this parameter will updated the configuration with the corrent
* configuration value.
*/
StrConfOption ZOOKEEPER_LIST =
new StrConfOption("giraph.zkList", "",
"ZooKeeper comma-separated list (if not set, will start up " +
"ZooKeeper locally). Consider that after locally-starting " +
"zookeeper, this parameter will updated the configuration with " +
"the corrent configuration value.");
/**
* Zookeeper List will always hold a value during the computation while
* this option provides information regarding whether the zookeeper was
* internally started or externally provided.
*/
BooleanConfOption ZOOKEEPER_IS_EXTERNAL =
new BooleanConfOption("giraph.zkIsExternal", true,
"Zookeeper List will always hold a value during " +
"the computation while this option provides " +
"information regarding whether the zookeeper was " +
"internally started or externally provided.");
/** ZooKeeper session millisecond timeout */
IntConfOption ZOOKEEPER_SESSION_TIMEOUT =
new IntConfOption("giraph.zkSessionMsecTimeout", MINUTES.toMillis(1),
"ZooKeeper session millisecond timeout");
/** Polling interval to check for the ZooKeeper server data */
IntConfOption ZOOKEEPER_SERVERLIST_POLL_MSECS =
new IntConfOption("giraph.zkServerlistPollMsecs", SECONDS.toMillis(3),
"Polling interval to check for the ZooKeeper server data");
/** ZooKeeper port to use */
IntConfOption ZOOKEEPER_SERVER_PORT =
new IntConfOption("giraph.zkServerPort", 22181, "ZooKeeper port to use");
/** Local ZooKeeper directory to use */
String ZOOKEEPER_DIR = "giraph.zkDir";
/** Max attempts for handling ZooKeeper connection loss */
IntConfOption ZOOKEEPER_OPS_MAX_ATTEMPTS =
new IntConfOption("giraph.zkOpsMaxAttempts", 3,
"Max attempts for handling ZooKeeper connection loss");
/**
* Msecs to wait before retrying a failed ZooKeeper op due to connection loss.
*/
IntConfOption ZOOKEEPER_OPS_RETRY_WAIT_MSECS =
new IntConfOption("giraph.zkOpsRetryWaitMsecs", SECONDS.toMillis(5),
"Msecs to wait before retrying a failed ZooKeeper op due to " +
"connection loss.");
/** TCP backlog (defaults to number of workers) */
IntConfOption TCP_BACKLOG = new IntConfOption("giraph.tcpBacklog", 1,
"TCP backlog (defaults to number of workers)");
/** Use netty pooled memory buffer allocator */
BooleanConfOption NETTY_USE_POOLED_ALLOCATOR = new BooleanConfOption(
"giraph.useNettyPooledAllocator", false, "Should netty use pooled " +
"memory allocator?");
/** Use direct memory buffers in netty */
BooleanConfOption NETTY_USE_DIRECT_MEMORY = new BooleanConfOption(
"giraph.useNettyDirectMemory", false, "Should netty use direct " +
"memory buffers");
/** How big to make the encoder buffer? */
IntConfOption NETTY_REQUEST_ENCODER_BUFFER_SIZE =
new IntConfOption("giraph.nettyRequestEncoderBufferSize", 32 * ONE_KB,
"How big to make the encoder buffer?");
/** Netty client threads */
IntConfOption NETTY_CLIENT_THREADS =
new IntConfOption("giraph.nettyClientThreads", 4, "Netty client threads");
/** Netty server threads */
IntConfOption NETTY_SERVER_THREADS =
new IntConfOption("giraph.nettyServerThreads", 16,
"Netty server threads");
/** Use the execution handler in netty on the client? */
BooleanConfOption NETTY_CLIENT_USE_EXECUTION_HANDLER =
new BooleanConfOption("giraph.nettyClientUseExecutionHandler", true,
"Use the execution handler in netty on the client?");
/** Netty client execution threads (execution handler) */
IntConfOption NETTY_CLIENT_EXECUTION_THREADS =
new IntConfOption("giraph.nettyClientExecutionThreads", 8,
"Netty client execution threads (execution handler)");
/** Where to place the netty client execution handle? */
StrConfOption NETTY_CLIENT_EXECUTION_AFTER_HANDLER =
new StrConfOption("giraph.nettyClientExecutionAfterHandler",
"request-encoder",
"Where to place the netty client execution handle?");
/** Use the execution handler in netty on the server? */
BooleanConfOption NETTY_SERVER_USE_EXECUTION_HANDLER =
new BooleanConfOption("giraph.nettyServerUseExecutionHandler", true,
"Use the execution handler in netty on the server?");
/** Netty server execution threads (execution handler) */
IntConfOption NETTY_SERVER_EXECUTION_THREADS =
new IntConfOption("giraph.nettyServerExecutionThreads", 8,
"Netty server execution threads (execution handler)");
/** Where to place the netty server execution handle? */
StrConfOption NETTY_SERVER_EXECUTION_AFTER_HANDLER =
new StrConfOption("giraph.nettyServerExecutionAfterHandler",
"requestFrameDecoder",
"Where to place the netty server execution handle?");
/** Netty simulate a first request closed */
BooleanConfOption NETTY_SIMULATE_FIRST_REQUEST_CLOSED =
new BooleanConfOption("giraph.nettySimulateFirstRequestClosed", false,
"Netty simulate a first request closed");
/** Netty simulate a first response failed */
BooleanConfOption NETTY_SIMULATE_FIRST_RESPONSE_FAILED =
new BooleanConfOption("giraph.nettySimulateFirstResponseFailed", false,
"Netty simulate a first response failed");
/** Netty - set which compression to use */
StrConfOption NETTY_COMPRESSION_ALGORITHM =
new StrConfOption("giraph.nettyCompressionAlgorithm", "",
"Which compression algorithm to use in netty");
/**
* Whether netty should pro-actively read requests and feed them to its
* processing pipeline
*/
BooleanConfOption NETTY_AUTO_READ =
new BooleanConfOption("giraph.nettyAutoRead", true,
"Whether netty should pro-actively read requests and feed them to " +
"its processing pipeline");
/** Max resolve address attempts */
IntConfOption MAX_RESOLVE_ADDRESS_ATTEMPTS =
new IntConfOption("giraph.maxResolveAddressAttempts", 5,
"Max resolve address attempts");
/** Msecs to wait between waiting for all requests to finish */
IntConfOption WAITING_REQUEST_MSECS =
new IntConfOption("giraph.waitingRequestMsecs", SECONDS.toMillis(15),
"Msecs to wait between waiting for all requests to finish");
/** Millseconds to wait for an event before continuing */
IntConfOption EVENT_WAIT_MSECS =
new IntConfOption("giraph.eventWaitMsecs", SECONDS.toMillis(30),
"Millseconds to wait for an event before continuing");
/**
* Maximum milliseconds to wait before giving up trying to get the minimum
* number of workers before a superstep (int).
*/
IntConfOption MAX_MASTER_SUPERSTEP_WAIT_MSECS =
new IntConfOption("giraph.maxMasterSuperstepWaitMsecs",
MINUTES.toMillis(10),
"Maximum milliseconds to wait before giving up trying to get the " +
"minimum number of workers before a superstep (int).");
/** Milliseconds for a request to complete (or else resend) */
IntConfOption MAX_REQUEST_MILLISECONDS =
new IntConfOption("giraph.maxRequestMilliseconds", MINUTES.toMillis(10),
"Milliseconds for a request to complete (or else resend)");
/**
* Whether to resend request which timed out or fail the job if timeout
* happens
*/
BooleanConfOption RESEND_TIMED_OUT_REQUESTS =
new BooleanConfOption("giraph.resendTimedOutRequests", true,
"Whether to resend request which timed out or fail the job if " +
"timeout happens");
/** Netty max connection failures */
IntConfOption NETTY_MAX_CONNECTION_FAILURES =
new IntConfOption("giraph.nettyMaxConnectionFailures", 1000,
"Netty max connection failures");
/** How long to wait before trying to reconnect failed connections */
IntConfOption WAIT_TIME_BETWEEN_CONNECTION_RETRIES_MS =
new IntConfOption("giraph.waitTimeBetweenConnectionRetriesMs", 500,
"");
/** Initial port to start using for the IPC communication */
IntConfOption IPC_INITIAL_PORT =
new IntConfOption("giraph.ipcInitialPort", 30000,
"Initial port to start using for the IPC communication");
/** Maximum bind attempts for different IPC ports */
IntConfOption MAX_IPC_PORT_BIND_ATTEMPTS =
new IntConfOption("giraph.maxIpcPortBindAttempts", 20,
"Maximum bind attempts for different IPC ports");
/**
* Fail first IPC port binding attempt, simulate binding failure
* on real grid testing
*/
BooleanConfOption FAIL_FIRST_IPC_PORT_BIND_ATTEMPT =
new BooleanConfOption("giraph.failFirstIpcPortBindAttempt", false,
"Fail first IPC port binding attempt, simulate binding failure " +
"on real grid testing");
/** Client send buffer size */
IntConfOption CLIENT_SEND_BUFFER_SIZE =
new IntConfOption("giraph.clientSendBufferSize", 512 * ONE_KB,
"Client send buffer size");
/** Client receive buffer size */
IntConfOption CLIENT_RECEIVE_BUFFER_SIZE =
new IntConfOption("giraph.clientReceiveBufferSize", 32 * ONE_KB,
"Client receive buffer size");
/** Server send buffer size */
IntConfOption SERVER_SEND_BUFFER_SIZE =
new IntConfOption("giraph.serverSendBufferSize", 32 * ONE_KB,
"Server send buffer size");
/** Server receive buffer size */
IntConfOption SERVER_RECEIVE_BUFFER_SIZE =
new IntConfOption("giraph.serverReceiveBufferSize", 512 * ONE_KB,
"Server receive buffer size");
/** Maximum size of messages (in bytes) per peer before flush */
IntConfOption MAX_MSG_REQUEST_SIZE =
new IntConfOption("giraph.msgRequestSize", 512 * ONE_KB,
"Maximum size of messages (in bytes) per peer before flush");
/**
* How much bigger than the average per partition size to make initial per
* partition buffers.
* If this value is A, message request size is M,
* and a worker has P partitions, than its initial partition buffer size
* will be (M / P) * (1 + A).
*/
FloatConfOption ADDITIONAL_MSG_REQUEST_SIZE =
new FloatConfOption("giraph.additionalMsgRequestSize", 0.2f,
"How much bigger than the average per partition size to make " +
"initial per partition buffers. If this value is A, message " +
"request size is M, and a worker has P partitions, than its " +
"initial partition buffer size will be (M / P) * (1 + A).");
/** Warn if msg request size exceeds default size by this factor */
FloatConfOption REQUEST_SIZE_WARNING_THRESHOLD = new FloatConfOption(
"giraph.msgRequestWarningThreshold", 2.0f,
"If request sizes are bigger than the buffer size by this factor " +
"warnings are printed to the log and to the command line");
/** Maximum size of vertices (in bytes) per peer before flush */
IntConfOption MAX_VERTEX_REQUEST_SIZE =
new IntConfOption("giraph.vertexRequestSize", 512 * ONE_KB,
"Maximum size of vertices (in bytes) per peer before flush");
/**
* Additional size (expressed as a ratio) of each per-partition buffer on
* top of the average size for vertices.
*/
FloatConfOption ADDITIONAL_VERTEX_REQUEST_SIZE =
new FloatConfOption("giraph.additionalVertexRequestSize", 0.2f,
"Additional size (expressed as a ratio) of each per-partition " +
"buffer on top of the average size.");
/** Maximum size of edges (in bytes) per peer before flush */
IntConfOption MAX_EDGE_REQUEST_SIZE =
new IntConfOption("giraph.edgeRequestSize", 512 * ONE_KB,
"Maximum size of edges (in bytes) per peer before flush");
/**
* Additional size (expressed as a ratio) of each per-partition buffer on
* top of the average size for edges.
*/
FloatConfOption ADDITIONAL_EDGE_REQUEST_SIZE =
new FloatConfOption("giraph.additionalEdgeRequestSize", 0.2f,
"Additional size (expressed as a ratio) of each per-partition " +
"buffer on top of the average size.");
/** Maximum number of mutations per partition before flush */
IntConfOption MAX_MUTATIONS_PER_REQUEST =
new IntConfOption("giraph.maxMutationsPerRequest", 100,
"Maximum number of mutations per partition before flush");
/**
* Use message size encoding (typically better for complex objects,
* not meant for primitive wrapped messages)
*/
BooleanConfOption USE_MESSAGE_SIZE_ENCODING =
new BooleanConfOption("giraph.useMessageSizeEncoding", false,
"Use message size encoding (typically better for complex objects, " +
"not meant for primitive wrapped messages)");
/** Number of channels used per server */
IntConfOption CHANNELS_PER_SERVER =
new IntConfOption("giraph.channelsPerServer", 1,
"Number of channels used per server");
/** Number of flush threads per peer */
String MSG_NUM_FLUSH_THREADS = "giraph.msgNumFlushThreads";
/** Number of threads for vertex computation */
IntConfOption NUM_COMPUTE_THREADS =
new IntConfOption("giraph.numComputeThreads", 1,
"Number of threads for vertex computation");
/** Number of threads for input split loading */
IntConfOption NUM_INPUT_THREADS =
new IntConfOption("giraph.numInputThreads", 1,
"Number of threads for input split loading");
/** Minimum stragglers of the superstep before printing them out */
IntConfOption PARTITION_LONG_TAIL_MIN_PRINT =
new IntConfOption("giraph.partitionLongTailMinPrint", 1,
"Minimum stragglers of the superstep before printing them out");
/** Use superstep counters? (boolean) */
BooleanConfOption USE_SUPERSTEP_COUNTERS =
new BooleanConfOption("giraph.useSuperstepCounters", true,
"Use superstep counters? (boolean)");
/**
* Input split sample percent - Used only for sampling and testing, rather
* than an actual job. The idea is that to test, you might only want a
* fraction of the actual input splits from your VertexInputFormat to
* load (values should be [0, 100]).
*/
FloatConfOption INPUT_SPLIT_SAMPLE_PERCENT =
new FloatConfOption("giraph.inputSplitSamplePercent", 100f,
"Input split sample percent - Used only for sampling and testing, " +
"rather than an actual job. The idea is that to test, you might " +
"only want a fraction of the actual input splits from your " +
"VertexInputFormat to load (values should be [0, 100]).");
/**
* To limit outlier vertex input splits from producing too many vertices or
* to help with testing, the number of vertices loaded from an input split
* can be limited. By default, everything is loaded.
*/
LongConfOption INPUT_SPLIT_MAX_VERTICES =
new LongConfOption("giraph.InputSplitMaxVertices", -1,
"To limit outlier vertex input splits from producing too many " +
"vertices or to help with testing, the number of vertices " +
"loaded from an input split can be limited. By default, " +
"everything is loaded.");
/**
* To limit outlier vertex input splits from producing too many vertices or
* to help with testing, the number of edges loaded from an input split
* can be limited. By default, everything is loaded.
*/
LongConfOption INPUT_SPLIT_MAX_EDGES =
new LongConfOption("giraph.InputSplitMaxEdges", -1,
"To limit outlier vertex input splits from producing too many " +
"vertices or to help with testing, the number of edges loaded " +
"from an input split can be limited. By default, everything is " +
"loaded.");
/**
* To minimize network usage when reading input splits,
* each worker can prioritize splits that reside on its host.
* This, however, comes at the cost of increased load on ZooKeeper.
* Hence, users with a lot of splits and input threads (or with
* configurations that can't exploit locality) may want to disable it.
*/
BooleanConfOption USE_INPUT_SPLIT_LOCALITY =
new BooleanConfOption("giraph.useInputSplitLocality", true,
"To minimize network usage when reading input splits, each worker " +
"can prioritize splits that reside on its host. " +
"This, however, comes at the cost of increased load on ZooKeeper. " +
"Hence, users with a lot of splits and input threads (or with " +
"configurations that can't exploit locality) may want to disable " +
"it.");
/** Multiplier for the current workers squared */
FloatConfOption PARTITION_COUNT_MULTIPLIER =
new FloatConfOption("giraph.masterPartitionCountMultiplier", 1.0f,
"Multiplier for the current workers squared");
/** Minimum number of partitions to have per compute thread */
IntConfOption MIN_PARTITIONS_PER_COMPUTE_THREAD =
new IntConfOption("giraph.minPartitionsPerComputeThread", 1,
"Minimum number of partitions to have per compute thread");
/** Overrides default partition count calculation if not -1 */
IntConfOption USER_PARTITION_COUNT =
new IntConfOption("giraph.userPartitionCount", -1,
"Overrides default partition count calculation if not -1");
/** Vertex key space size for
* {@link org.apache.giraph.partition.WorkerGraphPartitionerImpl}
*/
String PARTITION_VERTEX_KEY_SPACE_SIZE = "giraph.vertexKeySpaceSize";
/**
* How often to checkpoint (i.e. 0, means no checkpoint,
* 1 means every superstep, 2 is every two supersteps, etc.).
*/
IntConfOption CHECKPOINT_FREQUENCY =
new IntConfOption("giraph.checkpointFrequency", 0,
"How often to checkpoint (i.e. 0, means no checkpoint, 1 means " +
"every superstep, 2 is every two supersteps, etc.).");
/**
* Delete checkpoints after a successful job run?
*/
BooleanConfOption CLEANUP_CHECKPOINTS_AFTER_SUCCESS =
new BooleanConfOption("giraph.cleanupCheckpointsAfterSuccess", true,
"Delete checkpoints after a successful job run?");
/**
* An application can be restarted manually by selecting a superstep. The
* corresponding checkpoint must exist for this to work. The user should
* set a long value. Default is start from scratch.
*/
String RESTART_SUPERSTEP = "giraph.restartSuperstep";
/**
* If application is restarted manually we need to specify job ID
* to restart from.
*/
StrConfOption RESTART_JOB_ID = new StrConfOption("giraph.restart.jobId",
null, "Which job ID should I try to restart?");
/**
* Base ZNode for Giraph's state in the ZooKeeper cluster. Must be a root
* znode on the cluster beginning with "/"
*/
String BASE_ZNODE_KEY = "giraph.zkBaseZNode";
/**
* If ZOOKEEPER_LIST is not set, then use this directory to manage
* ZooKeeper
*/
StrConfOption ZOOKEEPER_MANAGER_DIRECTORY =
new StrConfOption("giraph.zkManagerDirectory",
"_bsp/_defaultZkManagerDir",
"If ZOOKEEPER_LIST is not set, then use this directory to manage " +
"ZooKeeper");
/** Number of ZooKeeper client connection attempts before giving up. */
IntConfOption ZOOKEEPER_CONNECTION_ATTEMPTS =
new IntConfOption("giraph.zkConnectionAttempts", 10,
"Number of ZooKeeper client connection attempts before giving up.");
/** This directory has/stores the available checkpoint files in HDFS. */
StrConfOption CHECKPOINT_DIRECTORY =
new StrConfOption("giraph.checkpointDirectory", "_bsp/_checkpoints/",
"This directory has/stores the available checkpoint files in HDFS.");
/**
* Comma-separated list of directories in the local filesystem for
* out-of-core partitions.
*/
StrConfOption PARTITIONS_DIRECTORY =
new StrConfOption("giraph.partitionsDirectory", "_bsp/_partitions",
"Comma-separated list of directories in the local filesystem for " +
"out-of-core partitions.");
/**
* Number of IO threads used in out-of-core mechanism. If local disk is used
* for spilling data to and reading data from, this number should be equal to
* the number of available disks on each machine. In such case, one should
* use giraph.partitionsDirectory to specify directories mounted on different
* disks.
*/
IntConfOption NUM_OUT_OF_CORE_THREADS =
new IntConfOption("giraph.numOutOfCoreThreads", 1, "Number of IO " +
"threads used in out-of-core mechanism. If using local disk to " +
"spill data, this should be equal to the number of available " +
"disks. In such case, use giraph.partitionsDirectory to specify " +
"mount points on different disks.");
/** Enable out-of-core graph. */