-
Notifications
You must be signed in to change notification settings - Fork 4
/
Copy pathguineapig1_3.py
1386 lines (1152 loc) · 53.4 KB
/
guineapig1_3.py
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
# earlier version of guineapig.py
##############################################################################
# (C) Copyright 2014 William W. Cohen. All rights reserved.
##############################################################################
import sys
import logging
import copy
import subprocess
import collections
import os
import os.path
import urlparse
import getopt
import csv
###############################################################################
# helpers functions and data structures
###############################################################################
class GPig(object):
"""Collection of utilities for Guinea Pig."""
HADOOP_LOC = 'hadoop' #assume hadoop is on the path at planning time
MY_LOC = 'guineapig1_3.py'
#global options for Guinea Pig can be passed in with the --opts
#command-line option, and these are the default values
defaultJar = '/usr/lib/hadoop/contrib/streaming/hadoop-streaming-1.2.0.1.3.0.0-107.jar'
envjar = os.environ.get('GP_STREAMJAR', defaultJar)
DEFAULT_OPTS = {'streamJar': envjar,
'parallel':5,
'target':'shell',
'echo':0,
'viewdir':'gpig_views',
}
#there are the types of each option that has a non-string value
DEFAULT_OPT_TYPES = {'parallel':int,'echo':int}
#we need to pass non-default options in to mappers and reducers,
#but since the remote worker's environment can be different, we
#also need to pass in options computed from the environment
COMPUTED_OPTION_DEFAULTS = {'streamJar':defaultJar}
@staticmethod
def getCompiler(target):
if target=='shell': return ShellCompiler()
elif target=='hadoop': return HadoopCompiler()
else: assert 'illegal compilation target '+target
@staticmethod
def getArgvParams():
"""Return a dictionary holding the argument of the --params option in
sys.argv."""
return GPig.getArgvDict('--params')
@staticmethod
def getArgvOpts():
"""Return a dictionary holding the argument of the --opts option in
sys.argv."""
return GPig.getArgvDict('--opts')
@staticmethod
def getArgvDict(optname):
"""Return a dictionary of parameter values that were defined on the command line
view an option like '--params filename:foo.txt,basedir:/tmp/glob/'.
"""
assert optname.startswith('--')
for i,a in enumerate(sys.argv):
if a==optname:
paramString = sys.argv[i+1]
return dict(pair.split(":") for pair in paramString.split(","))
return {}
@staticmethod
def rowsOf(view):
"""Iterate over the rows in a view."""
for line in open(view.distributableFile()):
yield view.planner._serializer.fromString(line.strip())
@staticmethod
def onlyRowOf(view):
"""Return the first row in a side view, and raise an error if it
is not the only row of the view."""
result = None
logging.info('loading '+view.distributableFile())
for line in open(view.distributableFile()):
assert not result,'multiple rows in stored file for '+view.tag
result = view.planner._serializer.fromString(line.strip())
return result
@staticmethod
class SafeEvaluator(object):
"""Evaluates expressions that correzpond to serialized guinea pig rows."""
def __init__(self,restrictedBindings={}):
self.restrictedBindings = restrictedBindings
def eval(self,s):
code = compile(s,'<gpig row>','eval')
return eval(code,self.restrictedBindings)
class Jin(object):
""""Object to hold description of a single join input."""
def __init__(self,view,by=(lambda x:x),outer=False):
self.view = view
self.joinBy = by
self.outer = outer
self._padWithNulls = False
def __str__(self):
viewStr = View.asTag(self.view) if self.view else '_'
outerStr = ',outer=True' if self.outer else ''
padStr = ',_padWithNulls=True' if self._padWithNulls else ''
return "Jin(%s,by=%s%s%s)" % (viewStr,self.joinBy,outerStr,padStr)
class ReduceTo(object):
"""An object x that can be the argument of a reducingTo=x
parameter in a Group view."""
def __init__(self,baseType,by=lambda accum,val:accum+val):
self.baseType = baseType
self.reduceBy = by
class ReduceToCount(ReduceTo):
"""Produce the count of the number of objects that would be placed in a group."""
def __init__(self):
ReduceTo.__init__(self,int,by=lambda accum,val:accum+1)
class ReduceToSum(ReduceTo):
"""Produce the sum of the objects - which must be numbers - that would
be placed in a group."""
def __init__(self):
ReduceTo.__init__(self,int,by=lambda accum,val:accum+val)
class ReduceToList(ReduceTo):
"""Produce a list of the objects that would be placed in a group."""
def __init__(self):
ReduceTo.__init__(self,list,by=lambda accum,val:accum+[val])
###############################################################################
# abstract views
##############################################################################
class View(object):
"""A definition of a relation for Guinea Pig. A View object can be
produce a storagePlan(), which can then be executed to produce the
contents of the relation. Intutitively, a relation is and
unordered bag of rows, and a row an almost-arbitrary python data
structure. (It must be something that can be stored and retrieved
by the RowSerializer.)
Steps in the storagePlan are executed by delegation, thru the
planner, to methods of a View class named doFoo.
"""
def __init__(self):
"""The planner and tag must be set before this is used."""
self.planner = None #pointer to planner object
self.tag = None #for naming storedFile and checkpoints
self.storeMe = None #try and store this view if true
self.retainedPart = None #used in map-reduce views only
self.sideviews = [] #non-empty for Augment views only
self.inners = [] #always used
#self.inner is shortcut for inners[0]
def _getInner(self): return self.inners[0]
def _setInner(self,val): self.inners = [val]
inner = property(_getInner,_setInner)
#
# ways to modify a view
#
def opts(self,stored=None):
"""Return the same view with options set appropriately. Possible
options include:
- stored=True - Explicitly store this view on disk whenever
it is used in another view's definition. This might be set
by the user for debugging purposes, or by the planner,
to prevent incorrect optimizations. Generally "inner"
views are not explicitly stored.
- stored='distributedCache' - Store this view in the working
directory and/or the Hadoop distributed cache.
"""
self.storeMe = stored
return self
def showExtras(self):
"""Printable representation of the options for a view."""
result = ''
flagPairs = []
if self.storeMe: flagPairs += ['stored=%s' % repr(self.storeMe)]
if flagPairs: result += '.opts(' + ",".join(flagPairs) + ')'
return result
#
# how the view is saved on disk
#
def storedFile(self):
"""The file that will hold the materialized relation."""
return self.planner.opts['viewdir'] + '/' + self.tag + '.gp'
def distributableFile(self):
"""The file that will hold the materialized relation in the working directory
in preparation to be uploaded to the distributed cache."""
return self.tag + '.gp'
@staticmethod
def viewNameFor(fileName):
"""The view associated with the given file name"""
vname = os.path.basename(fileName)
if vname.endswith(".gp"): vname = vname[0:-len(".gp")]
return vname
#
# semantics of the view
#
def checkpoint(self):
"""A checkpoint is an intermediate computation for the view, which is
saved on disk. The rowGenerator() for the view will assume
that the checkpoint is available.
"""
assert False, 'abstract method called'
def checkpointPlan(self):
"""A plan to produce checkpoint()."""
assert False, 'abstract method called'
def rowGenerator(self):
"""A generator for the rows in this relation, which assumes existence
of the checkpoint."""
assert False, 'abstract method called'
def explanation(self):
"""Return an explanation of how rows are generated."""
assert False, 'abstract method called'
def storagePlan(self):
"""A plan to store the view."""
return self.planner.buildRecursiveStoragePlan(self)
def nonrecursiveStoragePlan(self):
"""Materialize the relation, assuming that there are no descendent
inner views that need to be materialized first."""
plan = Plan()
plan.includeStepsOf(self.checkpointPlan())
plan.append(TransformStep(view=self,whatToDo='doStoreRows',srcs=[self.checkpoint()],dst=self.storedFile(),why=self.explanation()))
return plan
def applyDict(self,mapping,innerviewsOnly=False):
"""Given a mapping from view tags to views, replace every inner view with
the appropriate value from the mapping, and return the result."""
if self.tag in mapping and not innerviewsOnly:
return mapping[self.tag]
elif not self.inners:
return self
else:
result = copy.copy(self)
result.inners = map(lambda v:v.applyDict(mapping), self.inners)
return result
def sideviewsNeeded(self):
"""Sideviews needed by this view."""
result = []
for sv in self.sideviews:
result += [sv]
for v in self.inners:
result += list(v._sideviewsOfDescendants())
return result
def _sideviewsOfDescendants(self):
if not self.storeMe:
for sv in self.sideviews:
yield sv
for v in self.inners:
for sv in v._sideviewsOfDescendants():
yield sv
def enforceStorageConstraints(self):
"""Subclass this, if there are constraints on when one must explicitly
store inner views."""
pass
def doStoreRows(self):
"""Called by planner at execution time to store the rows of the view."""
for row in self.rowGenerator():
print self.planner._serializer.toString(row)
#
# support the "pipe" syntax: view1 | view2
#
def __or__(self,otherView):
"""Overload the pipe operator x | y to return with y, with x as its inner view."""
otherView.acceptInnerView(self)
return otherView
def acceptInnerView(self,otherView):
"""Replace an appropriate input view with otherView. This is subclassed to
implement the the pipe operator."""
assert not self.inner,'An inner view is defined for '+self.tag+' so you cannot use it as RHS of a pipe'
self.inner = otherView #subclass if needed
#
# printing views
#
def pprint(self,depth=0,alreadyPrinted=None,sideview=False):
"""Print a readable representation of the view."""
if alreadyPrinted==None: alreadyPrinted = set()
tabStr = '| ' * depth
tagStr = str(self.tag)
sideviewIndicator = '*' if sideview else ''
if self.tag in alreadyPrinted:
print tabStr + sideviewIndicator + tagStr + ' = ' + '...'
else:
sideviewInfo = " sideviews: {"+",".join(map(lambda x:x.tag, self.sideviews))+"}" if self.sideviews else ""
print tabStr + sideviewIndicator + tagStr + ' = ' + str(self) + sideviewInfo
alreadyPrinted.add(self.tag)
for inner in self.inners:
inner.pprint(depth+1,alreadyPrinted)
for inner in self.sideviews:
inner.pprint(depth+1,alreadyPrinted,sideview=True)
@staticmethod
def asTag(view):
"""Helper for printing views."""
if not view: return '(null view)'
elif view.tag: return view.tag
else: return str(view)
#
# abstract view types
#
class Reader(View):
"""Read data stored on the file system and make it look like a View."""
def __init__(self,src):
View.__init__(self)
self.src = src
self.inners = []
def checkpoint(self):
return self.src
def checkpointPlan(self):
return Plan() #empty plan
def explanation(self):
return [ 'read %s with %s' % (str(self.src),self.tag) ]
def acceptInnerView(self,otherView):
assert False, "Reader views cannot be used as RHS of a pipe"
class Transformation(View):
"""Streaming transformation on a single inner view."""
def __init__(self,inner=None):
View.__init__(self)
self.inner = inner
# A transformation will stream on-the-fly through the inner
# relation, and produce a new version, so the checkpoint and plan
# to produce it are delegated to the inner View.
def checkpoint(self):
return self.inner.checkpoint()
def checkpointPlan(self):
return self.inner.checkpointPlan()
def explanation(self):
return self.inner.explanation() + [ 'transform to %s' % self.tag ]
class MapReduce(View):
"""A view that takes an inner relation and processes in a
map-reduce-like way."""
def __init__(self,inners,retaining):
View.__init__(self)
self.inners = inners
self.retainedPart = retaining
def _isReduceInputFile(self,fileName):
return fileName.endswith('.gpri')
def checkpoint(self):
## the checkpoint is the reducer input file
return self.planner.opts['viewdir'] + '/' + self.tag + '.gpri'
def checkpointPlan(self):
plan = Plan()
for inner in self.inners:
plan.includeStepsOf(inner.checkpointPlan())
plan.includeStepsOf(self.mapPlan())
return plan
def enforceStorageConstraints(self):
for inner in self.inners:
innerChkpt = inner.checkpoint()
#optimizations break if you chain two map-reduces together
if innerChkpt and innerChkpt.endswith(".gpri"):
if not inner.storeMe:
logging.info('making %s stored, to make possible a downstream map-reduce view' % inner.tag)
inner.storeMe = True
def mapPlan(self):
log.error("abstract method not implemented")
def doStoreKeyedRows(self,subview,key,index):
"""Utility method used by concrete map-reduce classes to compute keys
and store key-value pairs. Usually used as the main step in a
mapPlan. """
for row in subview.rowGenerator():
keyStr = self.planner._serializer.toString(key(row))
rrow = self.retainedPart(row) if self.retainedPart else row
valStr = self.planner._serializer.toString(rrow)
if index<0:
print "%s\t%s" % (keyStr,valStr)
else:
print "%s\t%d\t%s" % (keyStr,index,valStr)
##############################################################################
#
# concrete View classes
#
##############################################################################
class ReuseView(Reader):
"""Returns the objects in a previously stored view."""
def __init__(self,view):
if isinstance(view,View):
Reader.__init__(self,view.storedFile())
self.tag = "reuse_"+view.tag
self.reusedViewTag = view.tag
self.planner = view.planner
else:
assert False,'user-defined ReuseView not supported (yet)'
def rowGenerator(self):
for line in sys.stdin:
yield self.planner._serializer.fromString(line.strip())
def __str__(self):
return 'ReuseView("%s")' % self.src + self.showExtras()
class ReadLines(Reader):
""" Returns the lines in a file, as python strings."""
def __init__(self,src):
Reader.__init__(self,src)
def rowGenerator(self):
for line in sys.stdin:
yield line
def __str__(self):
return 'ReadLines("%s")' % self.src + self.showExtras()
class ReadCSV(Reader):
""" Returns the lines in a CSV file, converted to Python tuples."""
def __init__(self,src,**kw):
Reader.__init__(self,src)
self.kw = kw
def rowGenerator(self):
for tup in csv.reader(sys.stdin,**self.kw):
yield tup
def __str__(self):
return 'ReadCVS("%s",%s)' % (self.src,str(self.kw)) + self.showExtras()
class ReplaceEach(Transformation):
""" In 'by=f'' f is a python function that takes a row and produces
its replacement."""
def __init__(self,inner=None,by=lambda x:x):
Transformation.__init__(self,inner)
self.replaceBy = by
def rowGenerator(self):
for row in self.inner.rowGenerator():
yield self.replaceBy(row)
def explanation(self):
return self.inner.explanation() + [ 'replaced to %s' % self.tag ]
def __str__(self):
return 'ReplaceEach(%s, by=%s)' % (View.asTag(self.inner),str(self.replaceBy)) + self.showExtras()
class Augment(Transformation):
def __init__(self,inner=None,sideviews=None,sideview=None,loadedBy=lambda v:list(GPig.rowsOf(v))):
Transformation.__init__(self,inner)
assert not (sideviews and sideview), 'cannot specify both "sideview" and "sideviews"'
self.sideviews = list(sideviews) if sideviews else [sideview]
self.loader = loadedBy
assert self.loader,'must specify a "loadedBy" function for Augment'
def enforceStorageConstraints(self):
for sv in self.sideviews:
sv.storeMe = 'distributedCache'
def rowGenerator(self):
augend = self.loader(*self.sideviews)
for row in self.inner.rowGenerator():
yield (row,augend)
def checkpointPlan(self):
plan = Plan()
plan.includeStepsOf(self.inner.checkpointPlan())
#the sideviews should have been stored by the top-level
#planner already, but they will need to be moved to a
#distributable location
for sv in self.sideviews:
plan.append(DistributeStep(sv))
return plan
def explanation(self):
return self.inner.explanation() + [ 'augmented to %s' % self.tag ]
def __str__(self):
sideviewTags = loaderTag = '*UNSPECIFIED*'
if self.sideviews!=None: sideviewTags = ",".join(map(View.asTag,self.sideviews))
if self.loader!=None: loaderTag = str(self.loader)
return 'Augment(%s,sideviews=%s,loadedBy=s%s)' % (View.asTag(self.inner),sideviewTags,loaderTag) + self.showExtras()
class Format(ReplaceEach):
""" Like ReplaceEach, but output should be a string, and it will be be
stored as strings, ie without using the serializer."""
def __init__(self,inner=None,by=lambda x:str(x)):
ReplaceEach.__init__(self,inner,by)
def __str__(self):
return 'Format(%s, by=%s)' % (View.asTag(self.inner),str(self.replaceBy)) + self.showExtras()
def doStoreRows(self):
for row in self.rowGenerator():
print row
class Flatten(Transformation):
""" Like ReplaceEach, but output of 'by' is an iterable, and all
results will be returned. """
def __init__(self,inner=None,by=None):
Transformation.__init__(self,inner)
self.flattenBy = by
def rowGenerator(self):
for row in self.inner.rowGenerator():
for flatrow in self.flattenBy(row):
yield flatrow
def explanation(self):
return self.inner.explanation() + [ 'flatten to %s' % self.tag ]
def __str__(self):
return 'Flatten(%s, by=%s)' % (View.asTag(self.inner),str(self.flattenBy)) + self.showExtras()
class Filter(Transformation):
"""Filter out a subset of rows that match some predicate."""
def __init__(self,inner=None,by=lambda x:x):
Transformation.__init__(self,inner)
self.filterBy = by
def rowGenerator(self):
for row in self.inner.rowGenerator():
if self.filterBy(row):
yield row
def explanation(self):
return self.inner.explanation() + [ 'filtered to %s' % self.tag ]
def __str__(self):
return 'Filter(%s, by=%s)' % (View.asTag(self.inner),str(self.filterBy)) + self.showExtras()
class Distinct(MapReduce):
"""Remove duplicate rows."""
def __init__(self,inner=None,retaining=None):
MapReduce.__init__(self,[inner],retaining)
def mapPlan(self):
plan = Plan()
plan.append(PrereduceStep(view=self,whatToDo='doDistinctMap',srcs=[self.inner.checkpoint()],dst=self.checkpoint(),why=self.explanation()))
return plan
def rowGenerator(self):
"""Extract distinct elements from a sorted list."""
lastval = None
for line in sys.stdin:
valStr = line.strip()
val = self.planner._serializer.fromString(valStr)
if val != lastval and lastval:
yield lastval
lastval = val
if lastval:
yield lastval
def explanation(self):
return self.inner.explanation() + [ 'make distinct to %s' % self.tag]
def __str__(self):
return 'Distinct(%s)' % (View.asTag(self.inner)) + self.showExtras()
def doDistinctMap(self):
self.inner.doStoreRows()
class Group(MapReduce):
"""Group by some property of a row, defined with the 'by' option.
Default outputs are tuples (x,[r1,...,rk]) where the ri's are rows
that have 'by' values of x."""
def __init__(self,inner=None,by=lambda x:x,reducingTo=ReduceToList(),retaining=None):
MapReduce.__init__(self,[inner],retaining)
self.groupBy = by
self.reducingTo = reducingTo
def mapPlan(self):
plan = Plan()
plan.append(PrereduceStep(view=self,whatToDo='doGroupMap',srcs=[self.inner.checkpoint()],dst=self.checkpoint(),why=self.explanation()))
return plan
def rowGenerator(self):
"""Group objects from stdin by key, yielding tuples (key,[g1,..,gn])."""
lastkey = key = None
accum = self.reducingTo.baseType()
for line in sys.stdin:
keyStr,valStr = line.strip().split("\t")
key = self.planner._serializer.fromString(keyStr)
val = self.planner._serializer.fromString(valStr)
if key != lastkey and lastkey!=None:
yield (lastkey,accum)
accum = self.reducingTo.baseType()
accum = self.reducingTo.reduceBy(accum, val)
lastkey = key
if key:
yield (key,accum)
def explanation(self):
return self.inner.explanation() + ['group to %s' % self.tag]
def __str__(self):
return 'Group(%s,by=%s,reducingTo=%s)' % (View.asTag(self.inner),str(self.groupBy),str(self.reducingTo)) + self.showExtras()
def doGroupMap(self):
self.doStoreKeyedRows(self.inner,self.groupBy,-1)
class Join(MapReduce):
"""Outputs tuples of the form (row1,row2,...rowk) where
rowi is from the i-th join input, and the rowi's have the same
value of the property being joined on."""
def __init__(self,*joinInputs):
#sets self.inners
MapReduce.__init__(self,map(lambda x:x.view, joinInputs),None)
self.joinInputs = joinInputs
#re-interpret the 'outer' join parameters - semantically
#if jin[i] is outer, then all other inputs must be marked as _padWithNulls
if any(map(lambda jin:jin.outer, self.joinInputs)):
assert len(self.joinInputs)==2,'outer joins are only supported on two-way joins '+str(self.joinInputs)
for i in range(len(self.joinInputs)):
if self.joinInputs[i].outer:
j = 1-i #the other index
self.joinInputs[j]._padWithNulls = True
def acceptInnerView(self,otherView):
assert False, 'join cannot be RHS of a pipe - use JoinTo instead'
def mapPlan(self):
plan = Plan()
innerCheckpoints = map(lambda v:v.checkpoint(), self.inners)
step = PrereduceStep(view=self, whatToDo='doJoinMap',srcs=innerCheckpoints,dst=self.checkpoint(),why=self.explanation())
plan.append(step)
return plan
def applyDict(self,mapping,innerviewsOnly=False):
result = MapReduce.applyDict(self,mapping,innerviewsOnly=innerviewsOnly)
#also need to map over the join inputs
if isinstance(result,Join):
for i in range(len(result.joinInputs)):
result.joinInputs[i].view = result.inners[i]
return result
def rowGenerator(self):
"""Group objects from stdin by key, yielding tuples (row1,row2,...)."""
lastkey = None
lastIndex = len(self.joinInputs)-1
somethingProducedForLastKey = False
#accumulate a list of lists of all non-final inputs
accumList = [ [] for i in range(lastIndex) ]
for line in sys.stdin:
keyStr,indexStr,valStr = line.strip().split("\t")
key = self.planner._serializer.fromString(keyStr)
index = int(indexStr)
val = self.planner._serializer.fromString(valStr)
if key != lastkey and lastkey!=None:
#if the final join is marked as _padWithNulls, clear
#the accumulators, since we're doing an outer join
#with the last view
if self.joinInputs[lastIndex]._padWithNulls and not somethingProducedForLastKey:
for tup in self._joinAccumulatedValuesTo(accumList,lastIndex,None):
yield tup
#reset the accumulators, since they pertain to the
accumList = [ [] for i in range(lastIndex) ]
somethingProducedForLastKey = False
if index!=lastIndex:
#accumulate values to use in the join
accumList[index] = accumList[index] + [val]
else:
#produce tuples that match the key for the last view
for tup in self._joinAccumulatedValuesTo(accumList,lastIndex,val):
somethingProducedForLastKey = True
yield tup
lastkey = key
def _joinAccumulatedValuesTo(self,accumList,lastIndex,finalVal):
# _padWithNulls as needed
for i in range(lastIndex):
if self.joinInputs[i]._padWithNulls and not accumList[i]:
accumList[i] = [None]
tupbuf = [ None for i in range(lastIndex+1) ] #holds output
tupbuf[lastIndex] = finalVal
for i in range(lastIndex):
for a in accumList[i]:
tupbuf[i] = a
if i==lastIndex-1 and any(tupbuf):
yield tuple(tupbuf)
def explanation(self):
innerEx = []
for inner in self.inners:
if innerEx: innerEx += ['THEN']
innerEx += inner.explanation()
return innerEx + [ 'FINALLY join to %s' % self.tag ]
def __str__(self):
return "Join(%s)" % ",".join(map(str,self.joinInputs)) + self.showExtras()
def doJoinMap(self,i):
# called by joinMapPlan with argument index, and stdin pointing to innerCheckpoints[index]
self.doStoreKeyedRows(self.joinInputs[i].view,self.joinInputs[i].joinBy,i)
class JoinTo(Join):
"""Special case of Join which can be used as the RHS of a pipe operator."""
def __init__(self,joinInput,by=None):
Join.__init__(self,Jin(None,by),joinInput)
def acceptInnerView(self,otherView):
self.joinInputs[0].view = otherView
self.inners[0] = otherView
##############################################################################
#
# the top-level planner, and its supporting classes
#
##############################################################################
class Plan(object):
"""A plan constructed by a GuineaPig."""
def __init__(self):
self.steps = []
self.tasks = []
def append(self,step):
self.steps.append(step)
def includeStepsOf(self,subplan):
self.steps += subplan.steps
def execute(self,gp,echo=False):
script = self.compile(gp)
for shellcom in script:
if echo: print 'calling:',shellcom
subprocess.check_call(shellcom,shell=True)
def buildTasks(self):
"""Group the steps into AbstractMapReduceTask's"""
self.tasks = [AbstractMapReduceTask()]
for step in self.steps:
if not self.tasks[-1].insert(step):
self.tasks.append(AbstractMapReduceTask())
status = self.tasks[-1].insert(step)
assert status, 'failure to insert '+str(step)+' in fresh AbstractMapReduceTask'
def compile(self,gp):
"""Return a list of strings that can be run as shell commands."""
self.buildTasks()
logging.info("%d steps converted to %d abstract map-reduce tasks" % (len(self.steps),len(self.tasks)))
script = []
taskCompiler = GPig.getCompiler(gp.opts['target'])
for task in self.tasks:
script += taskCompiler.compile(task,gp)
return script
#
# a single step in a plan produced by the planner
#
class Step(object):
"""A single step of the plans produced by the planner, along with the
methods to convert the plans into executable shell commands."""
def __init__(self,view):
self.view = view
self.reused = [] # list of views reused at this point
self.why = []
def setReusedViews(self,views):
self.reused = list(views)
def explain(self):
"""Convert an explanation - which is a list of strings - into a string"""
return "...".join(self.why)
#
# a single step in a plan produced by the planner
#
class DistributeStep(Step):
"""Prepare a stored view for the dDistributed cache."""
def __init__(self,view):
Step.__init__(self,view)
def __str__(self):
return "DistributeStep(%s,reused=%s)" % (repr(self.view.tag),repr(self.reused))
class TransformStep(Step):
"""Tranform input to output."""
def __init__(self,view,whatToDo,srcs,dst,why):
Step.__init__(self,view)
self.whatToDo = whatToDo
self.srcs = srcs
self.dst = dst
self.why = why
def __str__(self):
return "TransformStep("+",".join(map(repr, [self.view.tag,self.whatToDo,self.srcs,self.dst,self.reused]))+")"
class PrereduceStep(Step):
def __init__(self,view,whatToDo,srcs,dst,why):
Step.__init__(self,view)
self.whatToDo = whatToDo
self.srcs = srcs
self.dst = dst
self.why = why
def __str__(self):
return "PrereduceStep("+",".join(map(repr, [self.view.tag,self.whatToDo,self.srcs,self.dst,self.reused]))+")"
# combine steps into something executable via hadoop - or shell
class AbstractMapReduceTask(object):
"""A collection of steps that can be executed as a single map-reduce operation,
possibly with some file managements steps to set up the task."""
def __init__(self):
self.distributeSteps = []
self.mapStep = None
self.reduceStep = None
def insert(self,step):
"""Treating the AbstractMapReduceTask as a buffer, add this step to it if possible."""
if isinstance(step,DistributeStep):
#we can accept any number of distribute steps
self.distributeSteps.append(step)
return True
elif self.mapStep==None and (isinstance(step,TransformStep) or isinstance(step,PrereduceStep)):
#we can only have one map step, so fill up an empty slot if possible
self.mapStep = step
return True
elif self.mapStep and isinstance(self.mapStep,PrereduceStep) and isinstance(step,TransformStep) and not self.reduceStep:
#if the mapstep is a prereduce, then we can also allow a reduce step
self.reduceStep = step
return True
else:
return False
def __str__(self):
buf = "mapreduce task:"
for step in self.distributeSteps:
buf += "\n - d "+str(step)
buf += "\n - m " + str(self.mapStep)
if self.reduceStep:
buf += "\n - r " + str(self.reduceStep)
return buf
class MRCompiler(object):
"""Abstract compiler class to convert a task to a list of commands that can be executed by the shell."""
def compile(self,task,gp):
script = []
# an explanation/header
if not task.reduceStep:
script += ['echo create '+task.mapStep.view.tag + ' via map: ' + task.mapStep.explain()]
else:
script += ['echo create '+task.reduceStep.view.tag +' via map/reduce: '+task.reduceStep.explain()]
for step in task.distributeSteps:
localCopy = step.view.distributableFile()
maybeRemoteCopy = step.view.storedFile()
echoCom = 'echo distribute %s: making a local copy of %s in %s' % (step.view.tag,maybeRemoteCopy,localCopy)
script += [echoCom] + self.distributeCommands(task, gp, maybeRemoteCopy,localCopy)
if not task.reduceStep and len(task.mapStep.srcs)==1:
mapCom = self._coreCommand(task.mapStep,gp)
script += self.simpleMapCommands(task, gp, mapCom, task.mapStep.srcs[0], task.mapStep.dst)
elif task.reduceStep and len(task.mapStep.srcs)==1:
mapCom = self._coreCommand(task.mapStep,gp)
reduceCom = self._coreCommand(task.reduceStep,gp)
script += self.simpleMapReduceCommands(task, gp, mapCom, reduceCom, task.mapStep.srcs[0], task.reduceStep.dst)
elif task.reduceStep and len(task.mapStep.srcs)>1:
mapComs = [self._ithCoreCommand(task.mapStep,gp,i) for i in range(len(task.mapStep.srcs))]
reduceCom = self._coreCommand(task.reduceStep,gp)
midpoint = gp.opts['viewdir']+'/'+task.mapStep.view.tag+'.gpmo'
script += self.joinCommands(task, gp, mapComs, reduceCom, task.mapStep.srcs, midpoint, task.reduceStep.dst)
else:
assert False,'cannot compile task '+str(task)
return script
# abstract routines
def distributeCommands(self,task,gp,maybeRemoteCopy,localCopy):
"""Distribute the remote copy to the local directory."""
assert False, 'abstract method called'
def simpleMapCommands(self,task,gp,mapCom,src,dst):
"""A map-only task with zero or one inputs."""
assert False, 'abstract method called'
def simpleMapReduceCommands(self,task,gp,mapCom,reduceCom,src,dst):
"""A map-reduce task with one inputs."""
assert False, 'abstract method called'
def joinCommands(self,task,gp,mapComs,reduceCom,srcs,midpoint,dst):
"""A map-reduce task with several inputs."""
assert False, 'abstract method called'
# utilities
def _stepSideviewFiles(self,step):
files = []
for sv in step.view.sideviewsNeeded():
files += [sv.distributableFile()]
def _coreCommand(self,step,gp):
"""Python command to call an individual plan step."""
return 'python %s --view=%s --do=%s' % (gp._gpigSourceFile,step.view.tag,step.whatToDo) + self.__coreCommandOptions(step,gp)
def _ithCoreCommand(self,step,gp,i):
"""Like _coreCommand but allows index parameter to 'do' option"""
return 'python %s --view=%s --do=%s.%d' % (gp._gpigSourceFile,step.view.tag,step.whatToDo,i) + self.__coreCommandOptions(step,gp)
def __coreCommandOptions(self,step,gp):
paramOpts = '' if not gp.param else " --params " + ",".join(map(lambda(k,v):k+':'+v, gp.param.items()))
nonDefaults = []
for (k,v) in gp.opts.items():
#pass in non-default options, or options computed from the environment
if (gp.opts[k] != GPig.DEFAULT_OPTS[k]) or ((k in GPig.COMPUTED_OPTION_DEFAULTS) and (gp.opts[k] != GPig.COMPUTED_OPTION_DEFAULTS[k])):
nonDefaults += ["%s:%s" % (k,str(v))]
optsOpts = '' if not nonDefaults else " --opts " + ",".join(nonDefaults)
reuseOpts = '' if not step.reused else " --reuse "+ " ".join(step.reused)
return paramOpts + optsOpts + reuseOpts
class ShellCompiler(MRCompiler):
"""Compile tasks to commands that are executable to most Unix shells."""
def distributeCommands(self,task,gp,maybeRemoteCopy,localCopy):
"""Distribute the remote copy to the local directory."""
return ['cp -f %s %s || echo warning: the copy failed!' % (maybeRemoteCopy,localCopy)]
def simpleMapCommands(self,task,gp,mapCom,src,dst):
"""A map-only job with zero or one inputs."""
if src: return [mapCom + ' < %s > %s' % (src,dst)]
else: return [self.mapCommand(gp) + (' > %s' % (dst))]
def simpleMapReduceCommands(self,task,gp,mapCom,reduceCom,src,dst):
"""A map-reduce job with one input."""
return [mapCom + ' < ' + src + ' | sort -k1 | '+reduceCom + ' > ' + dst]
def joinCommands(self,task,gp,mapComs,reduceCom,srcs,midpoint,dst):
"""A map-reduce job with several inputs."""
subplan = ['rm -f %s' % midpoint]
for i,ithMapCom in enumerate(mapComs):
subplan += [ithMapCom + ' < ' + srcs[i] + ' >> ' + midpoint]