Skip to content

Commit

Permalink
[AIRFLOW-2928] Use uuid4 instead of uuid1 (#3779)
Browse files Browse the repository at this point in the history
for better randomness.
  • Loading branch information
Ken Kawamoto authored and msumit committed Aug 29, 2018
1 parent 314232c commit c3939c8
Show file tree
Hide file tree
Showing 7 changed files with 12 additions and 12 deletions.
2 changes: 1 addition & 1 deletion airflow/contrib/hooks/gcp_dataflow_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -250,7 +250,7 @@ def _build_dataflow_job_name(task_id, append_job_name=True):
'letter and ending with a letter or number '.format(task_id))

if append_job_name:
job_name = task_id + "-" + str(uuid.uuid1())[:8]
job_name = task_id + "-" + str(uuid.uuid4())[:8]
else:
job_name = task_id

Expand Down
4 changes: 2 additions & 2 deletions airflow/contrib/hooks/gcp_dataproc_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ def get(self):

class _DataProcJobBuilder:
def __init__(self, project_id, task_id, cluster_name, job_type, properties):
name = task_id + "_" + str(uuid.uuid1())[:8]
name = task_id + "_" + str(uuid.uuid4())[:8]
self.job_type = job_type
self.job = {
"job": {
Expand Down Expand Up @@ -141,7 +141,7 @@ def set_python_main(self, main):
self.job["job"][self.job_type]["mainPythonFileUri"] = main

def set_job_name(self, name):
self.job["job"]["reference"]["jobId"] = name + "_" + str(uuid.uuid1())[:8]
self.job["job"]["reference"]["jobId"] = name + "_" + str(uuid.uuid4())[:8]

def build(self):
return self.job
Expand Down
2 changes: 1 addition & 1 deletion airflow/contrib/kubernetes/pod_generator.py
Original file line number Diff line number Diff line change
Expand Up @@ -149,7 +149,7 @@ def make_pod(self, namespace, image, pod_id, cmds, arguments, labels):

return Pod(
namespace=namespace,
name=pod_id + "-" + str(uuid.uuid1())[:8],
name=pod_id + "-" + str(uuid.uuid4())[:8],
image=image,
cmds=cmds,
args=arguments,
Expand Down
2 changes: 1 addition & 1 deletion airflow/contrib/operators/dataflow_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -365,7 +365,7 @@ def google_cloud_to_local(self, file_name):

bucket_id = path_components[0]
object_id = '/'.join(path_components[1:])
local_file = '/tmp/dataflow{}-{}'.format(str(uuid.uuid1())[:8],
local_file = '/tmp/dataflow{}-{}'.format(str(uuid.uuid4())[:8],
path_components[-1])
file_size = self._gcs_hook.download(bucket_id, object_id, local_file)

Expand Down
6 changes: 3 additions & 3 deletions airflow/contrib/operators/dataproc_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -1158,7 +1158,7 @@ class DataProcPySparkOperator(BaseOperator):
@staticmethod
def _generate_temp_filename(filename):
dt = time.strftime('%Y%m%d%H%M%S')
return "{}_{}_{}".format(dt, str(uuid.uuid1())[:8], ntpath.basename(filename))
return "{}_{}_{}".format(dt, str(uuid.uuid4())[:8], ntpath.basename(filename))

"""
Upload a local file to a Google Cloud Storage bucket
Expand Down Expand Up @@ -1312,7 +1312,7 @@ def start(self):
.instantiate(
name=('projects/%s/regions/%s/workflowTemplates/%s' %
(self.project_id, self.region, self.template_id)),
body={'instanceId': str(uuid.uuid1())})
body={'instanceId': str(uuid.uuid4())})
.execute())


Expand Down Expand Up @@ -1355,6 +1355,6 @@ def start(self):
self.hook.get_conn().projects().regions().workflowTemplates()
.instantiateInline(
parent='projects/%s/regions/%s' % (self.project_id, self.region),
instanceId=str(uuid.uuid1()),
instanceId=str(uuid.uuid4()),
body=self.template)
.execute())
2 changes: 1 addition & 1 deletion airflow/contrib/task_runner/cgroup_task_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ def start(self):
# Create a unique cgroup name
cgroup_name = "airflow/{}/{}".format(datetime.datetime.utcnow().
strftime("%Y-%m-%d"),
str(uuid.uuid1()))
str(uuid.uuid4()))

self.mem_cgroup_name = "memory/{}".format(cgroup_name)
self.cpu_cgroup_name = "cpu/{}".format(cgroup_name)
Expand Down
6 changes: 3 additions & 3 deletions tests/contrib/hooks/test_gcp_dataflow_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ def setUp(self):
new=mock_init):
self.dataflow_hook = DataFlowHook(gcp_conn_id='test')

@mock.patch(DATAFLOW_STRING.format('uuid.uuid1'))
@mock.patch(DATAFLOW_STRING.format('uuid.uuid4'))
@mock.patch(DATAFLOW_STRING.format('_DataflowJob'))
@mock.patch(DATAFLOW_STRING.format('_Dataflow'))
@mock.patch(DATAFLOW_STRING.format('DataFlowHook.get_conn'))
Expand All @@ -103,7 +103,7 @@ def test_start_python_dataflow(self, mock_conn,
self.assertListEqual(sorted(mock_dataflow.call_args[0][0]),
sorted(EXPECTED_CMD))

@mock.patch(DATAFLOW_STRING.format('uuid.uuid1'))
@mock.patch(DATAFLOW_STRING.format('uuid.uuid4'))
@mock.patch(DATAFLOW_STRING.format('_DataflowJob'))
@mock.patch(DATAFLOW_STRING.format('_Dataflow'))
@mock.patch(DATAFLOW_STRING.format('DataFlowHook.get_conn'))
Expand All @@ -127,7 +127,7 @@ def test_start_java_dataflow(self, mock_conn,
self.assertListEqual(sorted(mock_dataflow.call_args[0][0]),
sorted(EXPECTED_CMD))

@mock.patch(DATAFLOW_STRING.format('uuid.uuid1'))
@mock.patch(DATAFLOW_STRING.format('uuid.uuid4'))
@mock.patch(DATAFLOW_STRING.format('_DataflowJob'))
@mock.patch(DATAFLOW_STRING.format('_Dataflow'))
@mock.patch(DATAFLOW_STRING.format('DataFlowHook.get_conn'))
Expand Down

0 comments on commit c3939c8

Please sign in to comment.