- Notifications
You must be signed in to change notification settings - Fork 4.3k
/
Copy pathjavacount.py
104 lines (86 loc) · 3.3 KB
/
javacount.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
#
# 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.
#
importlogging
importre
importtyping
importapache_beamasbeam
fromapache_beam.ioimportReadFromText
fromapache_beam.ioimportWriteToText
fromapache_beam.transforms.externalimportImplicitSchemaPayloadBuilder
fromapache_beam.options.pipeline_optionsimportPipelineOptions
"""A Python multi-language pipeline that counts words.
This pipeline reads an input text file and counts the words using the Java SDK
transform `Count.perElement()`.
Example commands for executing the program:
DirectRunner:
$ python javacount.py --runner DirectRunner --environment_type=DOCKER --input <INPUT FILE> --output output --expansion_service_port <PORT>
DataflowRunner:
$ python javacount.py \
--runner DataflowRunner \
--temp_location $TEMP_LOCATION \
--project $GCP_PROJECT \
--region $GCP_REGION \
--job_name $JOB_NAME \
--num_workers $NUM_WORKERS \
--input "gs://dataflow-samples/shakespeare/kinglear.txt" \
--output "gs://$GCS_BUCKET/javacount/output" \
--expansion_service_port <PORT>
"""
classWordExtractingDoFn(beam.DoFn):
defprocess(self, element):
returnre.findall(r'[\w\']+', element, re.UNICODE)
defrun(input_path, output_path, expansion_service_port, pipeline_args):
pipeline_options=PipelineOptions(pipeline_args)
withbeam.Pipeline(options=pipeline_options) asp:
lines=p|'Read'>>ReadFromText(input_path).with_output_types(str)
words=lines|'Split'>> (beam.ParDo(WordExtractingDoFn()).with_output_types(str))
java_output= (
words
|'JavaCount'>>beam.ExternalTransform(
'beam:transform:org.apache.beam:javacount:v1',
None,
('localhost:%s'%expansion_service_port)))
defformat(kv):
key, value=kv
return'%s:%s'% (key, value)
output=java_output|'Format'>>beam.Map(format)
output|'Write'>>WriteToText(output_path)
if__name__=='__main__':
logging.getLogger().setLevel(logging.INFO)
importargparse
parser=argparse.ArgumentParser()
parser.add_argument(
'--input',
dest='input',
required=True,
help='Input file')
parser.add_argument(
'--output',
dest='output',
required=True,
help='Output file')
parser.add_argument(
'--expansion_service_port',
dest='expansion_service_port',
required=True,
help='Expansion service port')
known_args, pipeline_args=parser.parse_known_args()
run(
known_args.input,
known_args.output,
known_args.expansion_service_port,
pipeline_args)