Skip to content

Commit

Permalink
Merge pull request #1198 from PrefectHQ/remove-cloud-result
Browse files Browse the repository at this point in the history
Remove cloud result handler
  • Loading branch information
cicdw authored Jul 2, 2019
2 parents 9f86a58 + 7850c6d commit 076a267
Show file tree
Hide file tree
Showing 13 changed files with 26 additions and 250 deletions.
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ These changes are available in the [master branch](https://github.com/PrefectHQ/

- The CLI command `prefect execute-flow` and `prefect execute-cloud-flow` no longer exist - [#1059](https://github.com/PrefectHQ/prefect/pull/1059)
- The `slack_notifier` state handler now uses a `webhook_secret` kwarg to pull the URL from a Secret - [#1075](https://github.com/PrefectHQ/prefect/issues/1075)
- Remove the `CloudResultHandler` default result handler - [#1198](https://github.com/PrefectHQ/prefect/pull/1198)

### Contributors

Expand Down
2 changes: 1 addition & 1 deletion docs/outline.toml
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ classes = ["JSONResultHandler", "GCSResultHandler", "LocalResultHandler", "S3Res
[pages.engine.cloud]
title = "Cloud"
module = "prefect.engine.cloud"
classes = ["CloudFlowRunner", "CloudTaskRunner", "CloudResultHandler"]
classes = ["CloudFlowRunner", "CloudTaskRunner"]

[pages.environments.storage]
title = "Storage"
Expand Down
5 changes: 2 additions & 3 deletions src/prefect/config.toml
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,6 @@ debug = false
api = "https://api.prefect.io"
graphql = "${cloud.api}/graphql/alpha"
log = "${cloud.api}/log"
result_handler = "${cloud.api}/result-handler"
use_local_secrets = true
heartbeat_interval = 30.0

Expand Down Expand Up @@ -69,8 +68,8 @@ run_on_schedule = true
default_class = "prefect.engine.flow_runner.FlowRunner"

[engine.result_handler]
# the default task runner, specified using a full path
default_class = "prefect.engine.cloud.CloudResultHandler"
# the default result handler, specified using a full path
default_class = ""

[engine.task_runner]
# the default task runner, specified using a full path
Expand Down
11 changes: 5 additions & 6 deletions src/prefect/engine/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -86,18 +86,17 @@ def get_default_result_handler_class() -> type:
value is a string, it will attempt to load the already-imported object. Otherwise, the
value is returned.
Defaults to `CloudResultHandler` if the string config value can not be loaded
Defaults to `None` if the string config value can not be loaded
"""
config_value = config.get_nested("engine.result_handler.default_class")

if isinstance(config_value, str):
if not config_value:
return lambda *args, **kwargs: None # type: ignore
try:
return prefect.utilities.serialization.from_qualified_name(config_value)
except ValueError:
warn(
"Could not import {}; using "
"prefect.engine.cloud.CloudResultHandler instead.".format(config_value)
)
return prefect.engine.cloud.CloudResultHandler
warn("Could not import {}; using " "None instead.".format(config_value))
return lambda *args, **kwargs: None # type: ignore
else:
return config_value
1 change: 0 additions & 1 deletion src/prefect/engine/cloud/__init__.py
Original file line number Diff line number Diff line change
@@ -1,3 +1,2 @@
from prefect.engine.cloud.result_handler import CloudResultHandler
from prefect.engine.cloud.task_runner import CloudTaskRunner
from prefect.engine.cloud.flow_runner import CloudFlowRunner
94 changes: 0 additions & 94 deletions src/prefect/engine/cloud/result_handler.py

This file was deleted.

16 changes: 7 additions & 9 deletions src/prefect/environments/execution/cloud/environment.py
Original file line number Diff line number Diff line change
Expand Up @@ -259,12 +259,11 @@ def _populate_job_yaml(

env[0]["value"] = prefect.config.cloud.graphql
env[1]["value"] = prefect.config.cloud.log
env[2]["value"] = prefect.config.cloud.result_handler
env[3]["value"] = prefect.config.cloud.auth_token
env[4]["value"] = flow_run_id
env[5]["value"] = prefect.context.get("namespace", "")
env[6]["value"] = docker_name
env[7]["value"] = flow_file_path
env[2]["value"] = prefect.config.cloud.auth_token
env[3]["value"] = flow_run_id
env[4]["value"] = prefect.context.get("namespace", "")
env[5]["value"] = docker_name
env[6]["value"] = flow_file_path

# set image
yaml_obj["spec"]["template"]["spec"]["containers"][0]["image"] = docker_name
Expand Down Expand Up @@ -292,9 +291,8 @@ def _populate_worker_pod_yaml(self, yaml_obj: dict) -> dict:

env[0]["value"] = prefect.config.cloud.graphql
env[1]["value"] = prefect.config.cloud.log
env[2]["value"] = prefect.config.cloud.result_handler
env[3]["value"] = prefect.config.cloud.auth_token
env[4]["value"] = prefect.context.get("flow_run_id", "")
env[2]["value"] = prefect.config.cloud.auth_token
env[3]["value"] = prefect.context.get("flow_run_id", "")

if self.private_registry:
namespace = prefect.context.get("namespace", "")
Expand Down
9 changes: 0 additions & 9 deletions src/prefect/serialization/result_handlers.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@

from marshmallow import ValidationError, fields, post_load

from prefect.engine.cloud.result_handler import CloudResultHandler
from prefect.engine.result_handlers import (
GCSResultHandler,
JSONResultHandler,
Expand Down Expand Up @@ -39,13 +38,6 @@ def create_object(self, data: dict, **kwargs: Any) -> None:
return None


class CloudResultHandlerSchema(BaseResultHandlerSchema):
class Meta:
object_class = CloudResultHandler

result_handler_service = fields.String(allow_none=True)


class GCSResultHandlerSchema(BaseResultHandlerSchema):
class Meta:
object_class = GCSResultHandler
Expand Down Expand Up @@ -84,7 +76,6 @@ class ResultHandlerSchema(OneOfSchema):
"ResultHandler": BaseResultHandlerSchema,
"GCSResultHandler": GCSResultHandlerSchema,
"S3ResultHandler": S3ResultHandlerSchema,
"CloudResultHandler": CloudResultHandlerSchema,
"JSONResultHandler": JSONResultHandlerSchema,
"LocalResultHandler": LocalResultHandlerSchema,
}
Expand Down
64 changes: 0 additions & 64 deletions tests/engine/cloud/test_cloud_result_handler.py

This file was deleted.

2 changes: 1 addition & 1 deletion tests/engine/cloud/test_cloud_task_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@
from prefect.client import Client
from prefect.core import Edge, Task
from prefect.engine.cache_validators import all_inputs
from prefect.engine.cloud import CloudResultHandler, CloudTaskRunner
from prefect.engine.cloud import CloudTaskRunner
from prefect.engine.result import NoResult, Result, SafeResult
from prefect.engine.result_handlers import (
JSONResultHandler,
Expand Down
7 changes: 2 additions & 5 deletions tests/engine/test_defaults.py
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ def test_default_task_runner_with_bad_config():


def test_default_result_handler():
assert engine.get_default_result_handler_class() is engine.cloud.CloudResultHandler
assert engine.get_default_result_handler_class()() is None


def test_default_result_handler_responds_to_config():
Expand Down Expand Up @@ -121,7 +121,4 @@ def test_default_result_handler_with_bad_config():
{"engine.result_handler.default_class": "prefect.engine. bad import path"}
):
with pytest.warns(UserWarning):
assert (
engine.get_default_result_handler_class()
is engine.cloud.CloudResultHandler
)
assert engine.get_default_result_handler_class()() is None
19 changes: 7 additions & 12 deletions tests/environments/execution/test_cloud_environment.py
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,6 @@ def test_populate_job_yaml():
{
"cloud.graphql": "gql_test",
"cloud.log": "log_test",
"cloud.result_handler": "rh_test",
"cloud.auth_token": "auth_test",
}
):
Expand All @@ -205,12 +204,11 @@ def test_populate_job_yaml():

assert env[0]["value"] == "gql_test"
assert env[1]["value"] == "log_test"
assert env[2]["value"] == "rh_test"
assert env[3]["value"] == "auth_test"
assert env[4]["value"] == "id_test"
assert env[5]["value"] == "namespace_test"
assert env[6]["value"] == "test1/test2:test3"
assert env[7]["value"] == "test4"
assert env[2]["value"] == "auth_test"
assert env[3]["value"] == "id_test"
assert env[4]["value"] == "namespace_test"
assert env[5]["value"] == "test1/test2:test3"
assert env[6]["value"] == "test4"

assert (
yaml_obj["spec"]["template"]["spec"]["containers"][0]["image"]
Expand All @@ -232,7 +230,6 @@ def test_populate_worker_pod_yaml():
{
"cloud.graphql": "gql_test",
"cloud.log": "log_test",
"cloud.result_handler": "rh_test",
"cloud.auth_token": "auth_test",
}
):
Expand All @@ -246,9 +243,8 @@ def test_populate_worker_pod_yaml():

assert env[0]["value"] == "gql_test"
assert env[1]["value"] == "log_test"
assert env[2]["value"] == "rh_test"
assert env[3]["value"] == "auth_test"
assert env[4]["value"] == "id_test"
assert env[2]["value"] == "auth_test"
assert env[3]["value"] == "id_test"

assert yaml_obj["spec"]["containers"][0]["image"] == "my_image"

Expand All @@ -267,7 +263,6 @@ def test_populate_worker_pod_yaml_with_private_registry():
{
"cloud.graphql": "gql_test",
"cloud.log": "log_test",
"cloud.result_handler": "rh_test",
"cloud.auth_token": "auth_test",
}
):
Expand Down
Loading

0 comments on commit 076a267

Please sign in to comment.