-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathpreprocess.py
280 lines (236 loc) · 8.58 KB
/
preprocess.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
import datetime
import functools
import json
import logging
import os
from collections import defaultdict
import coiled
import dask
import dask_deltatable as ddt
import deltalake
import fsspec
import pandas as pd
import toolz
from dask.distributed import LocalCluster, print, wait
from prefect import flow
from upath import UPath as Path
dask.config.set({'dataframe.query-planning': True})
logger = logging.getLogger(__name__)
LOCAL = True
if LOCAL:
OUTDIR = Path(f"./data")
STORAGE_OPTIONS = {}
Cluster = LocalCluster
else:
OUTDIR = Path(f"s3://openscapes-scratch/etl-github-test")
STORAGE_OPTIONS = {"AWS_REGION": "us-west-2", "AWS_S3_ALLOW_UNSAFE_RENAME": "true"}
Cluster = functools.partial(
coiled.Cluster,
name="etl-github",
n_workers=1,
region="us-west-2",
shutdown_on_close=False,
)
def handle_PushEvent(d):
for commit in d["payload"]["commits"]:
yield {
"username": d["actor"]["login"],
"repo": d["repo"]["name"],
"sha": commit["sha"],
"message": commit["message"],
"created_at": datetime.datetime.fromisoformat(d["created_at"]),
}
def handle_CreateEvent(d):
return {
"username": d["actor"]["login"],
"repo": d["repo"]["name"],
"type": d["payload"]["ref_type"],
"name": d["payload"]["ref"],
"description": d["payload"]["description"],
"created_at": datetime.datetime.fromisoformat(d["created_at"]),
}
def handle_PullRequestEvent(d):
return {
"username": d["actor"]["login"],
"repo": d["repo"]["name"],
"action": d["payload"]["action"],
"number": d["payload"]["number"],
"title": d["payload"]["pull_request"]["title"],
"author": d["payload"]["pull_request"]["user"]["login"],
"body": d["payload"]["pull_request"]["body"],
"pr_created_at": datetime.datetime.fromisoformat(
d["payload"]["pull_request"]["created_at"]
),
"created_at": datetime.datetime.fromisoformat(d["created_at"]),
}
def handle_IssueCommentEvent(d):
return {
"username": d["actor"]["login"],
"repo": d["repo"]["name"],
"number": d["payload"]["issue"]["number"],
"title": d["payload"]["issue"]["title"],
"author": d["payload"]["issue"]["user"]["login"],
"issue_created_at": datetime.datetime.fromisoformat(
d["payload"]["issue"]["created_at"]
),
"comment": d["payload"]["comment"]["body"],
"association": d["payload"]["comment"]["author_association"],
"created_at": datetime.datetime.fromisoformat(d["created_at"]),
}
def handle_WatchEvent(d):
return {
"username": d["actor"]["login"],
"repo": d["repo"]["name"],
"action": d["payload"]["action"],
"created_at": datetime.datetime.fromisoformat(d["created_at"]),
}
def handle_ForkEvent(d):
return {
"username": d["actor"]["login"],
"repo": d["repo"]["name"],
"created_at": datetime.datetime.fromisoformat(d["created_at"]),
}
conversions = {
"PushEvent": handle_PushEvent,
"CreateEvent": handle_CreateEvent,
"PullRequestEvent": handle_PullRequestEvent,
"IssueCommentEvent": handle_IssueCommentEvent,
"WatchEvent": handle_WatchEvent,
"ForkEvent": handle_ForkEvent,
}
def process_records(seq):
out = defaultdict(list)
for record in seq:
try:
record = json.loads(record)
except Exception:
continue
if record["type"] in conversions:
convert = conversions[record["type"]]
try:
result = convert(record)
except Exception as e:
logger.error(f"Failed to parse record: {e}")
continue
out[record["type"]].append(result)
return out
def process_file(filename: str) -> dict[str, pd.DataFrame]:
with fsspec.open(
filename, compression="gzip" if filename.endswith("gz") else None
) as f:
out = process_records(f)
out["Commits"] = list(toolz.concat(out["PushEvent"]))
del out["PushEvent"]
out = toolz.valmap(pd.DataFrame, out)
out = {
"comment": out["IssueCommentEvent"],
"pr": out["PullRequestEvent"],
"commit": out["Commits"],
"create": out["CreateEvent"],
"watch": out["WatchEvent"],
"fork": out["ForkEvent"],
}
for df in out.values():
df["date"] = df.created_at.dt.date
return out
def write_delta(tables: dict[str, pd.DataFrame]):
for table, df in tables.items():
outfile = OUTDIR / table
outfile.fs.makedirs(outfile.parent, exist_ok=True)
deltalake.write_deltalake(
outfile,
df,
mode="append",
storage_options=STORAGE_OPTIONS,
partition_by="date",
)
def parse_start_stop(start, stop):
""" If no provided, determine start and stop times """
if start is None:
t = deltalake.DeltaTable(
OUTDIR / "comment",
storage_options=STORAGE_OPTIONS,
)
actions = t.get_add_actions(flatten=True).to_pandas()
start = actions["max.created_at"].max().ceil("1h")
if stop is None:
# Current hour won't be fully populated yet, so subtract an hour
stop = pd.Timestamp.now().floor("1h") - datetime.timedelta(hours=1)
return start, stop
def list_files(start, stop):
dts = pd.date_range(start, stop, freq="1h")
filenames = [f"https://data.gharchive.org/{dt.date()}-{dt.hour}.json.gz" for dt in dts]
return filenames
def compact(table):
t = deltalake.DeltaTable(
OUTDIR / table,
storage_options=STORAGE_OPTIONS,
)
t.optimize.compact()
t.vacuum(retention_hours=0, enforce_retention_duration=False, dry_run=False)
print(f"Compacted {table} table")
def query_data():
# These are not the actual queries we want.
# We're using these for now to get something going, but we
# should revisit them soon.
# (xref https://github.com/coiled/etl-github/pull/4#discussion_r1509525504)
watches = ddt.read_deltalake(OUTDIR / "watch")
repos = watches.repo.value_counts()
repos = repos[repos > 5].repartition(npartitions=1).to_frame()
commits = ddt.read_deltalake(OUTDIR / "commit")
commits = commits[~commits.username.str.contains("bot")] # remove bots
major_commits = commits.merge(repos, on="repo")
df = major_commits[
major_commits.message.str.lower().str.contains(" dask")
][["username", "repo", "message", "count"]]
df = df[~df.repo.str.startswith("dask/")]
out = df.sort_values("count", ascending=False)
outdir = OUTDIR / "dask" / "commits"
if outdir.exists():
outdir.fs.rm(str(outdir), recursive=True)
outdir.fs.makedirs(outdir)
ddt.to_deltalake(outdir, out, storage_options=STORAGE_OPTIONS)
# Same but for comments
comments = ddt.read_deltalake(OUTDIR / "comment")
major_comments = comments.merge(repos, on="repo")
df = major_comments[
major_comments.comment.str.lower().str.contains(" dask")
][["username", "repo", "comment", "count"]]
df = df[~df.repo.str.startswith("dask/")]
out = df.sort_values("count", ascending=False)
outdir = OUTDIR / "dask" / "comments"
if outdir.exists():
outdir.fs.rm(str(outdir), recursive=True)
outdir.fs.makedirs(outdir)
ddt.to_deltalake(outdir, out, storage_options=STORAGE_OPTIONS)
@flow(log_prints=True)
def workflow(start=None, stop=None):
start, stop = parse_start_stop(start, stop)
filenames = list_files(start, stop)
print(f"{filenames = }")
with Cluster() as cluster:
if not LOCAL:
cluster.send_private_envs({
"AWS_ACCESS_KEY_ID": os.environ["AWS_ACCESS_KEY_ID"],
"AWS_SECRET_ACCESS_KEY": os.environ["AWS_SECRET_ACCESS_KEY"],
})
cluster.adapt(minimum=1, maximum=100)
with cluster.get_client() as client:
futures = client.map(process_file, filenames)
futures = client.map(write_delta, futures, retries=10)
wait(futures)
tables = ["comment", "pr", "commit", "create", "watch", "fork"]
futures = client.map(compact, tables)
wait(futures)
query_data()
if __name__ == "__main__":
# Initial bulk processing of historical data
workflow(
start=datetime.datetime(2024, 2, 29, hour=0),
stop=datetime.datetime(2024, 3, 1, hour=3),
)
# Process new data hourly
workflow.serve(
name="github-processing",
interval=datetime.timedelta(hours=1),
)