From cc3af3138ecd816a01a591fe6dee0cbedd224866 Mon Sep 17 00:00:00 2001 From: EphraimBuddy Date: Fri, 3 Jul 2020 23:29:45 +0100 Subject: [PATCH 01/11] update api_connexion endpoints and schemas to use marshmallow 3 --- .../endpoints/config_endpoint.py | 2 +- .../endpoints/connection_endpoint.py | 8 +++--- .../endpoints/import_error_endpoint.py | 2 +- .../api_connexion/endpoints/pool_endpoint.py | 10 +++---- .../endpoints/variable_endpoint.py | 13 ++++----- .../api_connexion/schemas/common_schema.py | 6 ++--- .../api_connexion/schemas/config_schema.py | 2 +- .../schemas/connection_schema.py | 18 +++---------- .../api_connexion/schemas/dag_run_schema.py | 6 ++--- airflow/api_connexion/schemas/dag_schema.py | 3 +-- airflow/api_connexion/schemas/error_schema.py | 6 ++--- .../api_connexion/schemas/event_log_schema.py | 8 +++--- airflow/api_connexion/schemas/log_schema.py | 2 +- airflow/api_connexion/schemas/pool_schema.py | 15 +++-------- .../api_connexion/schemas/variable_schema.py | 4 +-- .../api_connexion/schemas/version_schema.py | 2 +- airflow/api_connexion/schemas/xcom_schema.py | 6 ++--- setup.py | 4 +-- .../endpoints/test_connection_endpoint.py | 4 +-- .../endpoints/test_dag_endpoint.py | 2 ++ .../endpoints/test_pool_endpoint.py | 4 +-- .../endpoints/test_variable_endpoint.py | 2 +- .../schemas/test_common_schema.py | 20 +++++++------- .../schemas/test_config_schema.py | 2 +- .../schemas/test_connection_schema.py | 12 ++++----- .../schemas/test_dag_run_schema.py | 27 ++++++------------- .../api_connexion/schemas/test_dag_schema.py | 7 ++--- .../schemas/test_error_schema.py | 14 +++++----- .../schemas/test_event_log_schema.py | 4 +-- .../schemas/test_pool_schemas.py | 6 ++--- .../api_connexion/schemas/test_task_schema.py | 4 +-- .../schemas/test_version_schema.py | 2 +- .../api_connexion/schemas/test_xcom_schema.py | 10 +++---- 33 files changed, 102 insertions(+), 135 deletions(-) diff --git a/airflow/api_connexion/endpoints/config_endpoint.py b/airflow/api_connexion/endpoints/config_endpoint.py index 2261cac28255b..ddac883acd334 100644 --- a/airflow/api_connexion/endpoints/config_endpoint.py +++ b/airflow/api_connexion/endpoints/config_endpoint.py @@ -59,7 +59,7 @@ def _config_to_text(config: Config) -> str: def _config_to_json(config: Config) -> str: """Convert a Config object to a JSON formatted string""" - return json.dumps(config_schema.dump(config).data, indent=4) + return json.dumps(config_schema.dump(config), indent=4) def get_config() -> Response: diff --git a/airflow/api_connexion/endpoints/connection_endpoint.py b/airflow/api_connexion/endpoints/connection_endpoint.py index 628fedd588975..321e5089d929f 100644 --- a/airflow/api_connexion/endpoints/connection_endpoint.py +++ b/airflow/api_connexion/endpoints/connection_endpoint.py @@ -73,11 +73,10 @@ def patch_connection(connection_id, session, update_mask=None): Update a connection entry """ try: - body = connection_schema.load(request.json, partial=True) + data = connection_schema.load(request.json, partial=True) except ValidationError as err: # If validation get to here, it is extra field validation. - raise BadRequest(detail=err.messages.get('_schema', [err.messages])[0]) - data = body.data + raise BadRequest(detail=str(err.messages)) non_update_fields = ['connection_id', 'conn_id'] connection = session.query(Connection).filter_by(conn_id=connection_id).first() if connection is None: @@ -107,10 +106,9 @@ def post_connection(session): """ body = request.json try: - result = connection_schema.load(body) + data = connection_schema.load(body) except ValidationError as err: raise BadRequest(detail=str(err.messages)) - data = result.data conn_id = data['conn_id'] query = session.query(Connection) connection = query.filter_by(conn_id=conn_id).first() diff --git a/airflow/api_connexion/endpoints/import_error_endpoint.py b/airflow/api_connexion/endpoints/import_error_endpoint.py index e054777ce66a7..2b571d57b6cb1 100644 --- a/airflow/api_connexion/endpoints/import_error_endpoint.py +++ b/airflow/api_connexion/endpoints/import_error_endpoint.py @@ -51,4 +51,4 @@ def get_import_errors(session, limit, offset=None): import_errors = session.query(ImportError).order_by(ImportError.id).offset(offset).limit(limit).all() return import_error_collection_schema.dump( ImportErrorCollection(import_errors=import_errors, total_entries=total_entries) - ).data + ) diff --git a/airflow/api_connexion/endpoints/pool_endpoint.py b/airflow/api_connexion/endpoints/pool_endpoint.py index fd69e72caf55c..357e96afa1cbe 100644 --- a/airflow/api_connexion/endpoints/pool_endpoint.py +++ b/airflow/api_connexion/endpoints/pool_endpoint.py @@ -63,7 +63,7 @@ def get_pools(session, limit, offset=None): pools = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all() return pool_collection_schema.dump( PoolCollection(pools=pools, total_entries=total_entries) - ).data + ) @provide_session @@ -86,9 +86,9 @@ def patch_pool(pool_name, session, update_mask=None): raise NotFound(detail=f"Pool with name:'{pool_name}' not found") try: - patch_body = pool_schema.load(request.json).data + patch_body = pool_schema.load(request.json) except ValidationError as err: - raise BadRequest(detail=err.messages.get("_schema", [err.messages])[0]) + raise BadRequest(detail=str(err.messages)) if update_mask: update_mask = [i.strip() for i in update_mask] @@ -127,9 +127,9 @@ def post_pool(session): raise BadRequest(detail=f"'{field}' is a required property") try: - post_body = pool_schema.load(request.json, session=session).data + post_body = pool_schema.load(request.json, session=session) except ValidationError as err: - raise BadRequest(detail=err.messages.get("_schema", [err.messages])[0]) + raise BadRequest(detail=str(err.messages)) pool = Pool(**post_body) try: diff --git a/airflow/api_connexion/endpoints/variable_endpoint.py b/airflow/api_connexion/endpoints/variable_endpoint.py index c77f38cadc407..494e8e6c1cfcc 100644 --- a/airflow/api_connexion/endpoints/variable_endpoint.py +++ b/airflow/api_connexion/endpoints/variable_endpoint.py @@ -73,11 +73,11 @@ def patch_variable(variable_key: str, update_mask: Optional[List[str]] = None) - Update a variable by key """ try: - var = variable_schema.load(request.json) + data = variable_schema.load(request.json) except ValidationError as err: raise BadRequest("Invalid Variable schema", detail=str(err.messages)) - if var.data["key"] != variable_key: + if data["key"] != variable_key: raise BadRequest("Invalid post body", detail="key from request body doesn't match uri parameter") if update_mask: @@ -86,7 +86,7 @@ def patch_variable(variable_key: str, update_mask: Optional[List[str]] = None) - if "value" not in update_mask: raise BadRequest("No field to update") - Variable.set(var.data["key"], var.data["val"]) + Variable.set(data["key"], data["val"]) return Response(status=204) @@ -95,8 +95,9 @@ def post_variables() -> Response: Create a variable """ try: - var = variable_schema.load(request.json) + data = variable_schema.load(request.json) + except ValidationError as err: raise BadRequest("Invalid Variable schema", detail=str(err.messages)) - Variable.set(var.data["key"], var.data["val"]) - return variable_schema.dump(var) + Variable.set(data["key"], data["val"]) + return variable_schema.dump(data) diff --git a/airflow/api_connexion/schemas/common_schema.py b/airflow/api_connexion/schemas/common_schema.py index 5e3afe69389f9..160e12cc40b7d 100644 --- a/airflow/api_connexion/schemas/common_schema.py +++ b/airflow/api_connexion/schemas/common_schema.py @@ -36,7 +36,7 @@ class CronExpression(typing.NamedTuple): class TimeDeltaSchema(Schema): """Time delta schema""" - objectType = fields.Constant("TimeDelta", dump_to="__type") + objectType = fields.Constant("TimeDelta", data_key="__type") days = fields.Integer() seconds = fields.Integer() microseconds = fields.Integer() @@ -53,7 +53,7 @@ def make_time_delta(self, data, **kwargs): class RelativeDeltaSchema(Schema): """Relative delta schema""" - objectType = fields.Constant("RelativeDelta", dump_to="__type") + objectType = fields.Constant("RelativeDelta", data_key="__type") years = fields.Integer() months = fields.Integer() days = fields.Integer() @@ -83,7 +83,7 @@ def make_relative_delta(self, data, **kwargs): class CronExpressionSchema(Schema): """Cron expression schema""" - objectType = fields.Constant("CronExpression", dump_to="__type", required=True) + objectType = fields.Constant("CronExpression", data_key="__type", required=True) value = fields.String(required=True) @marshmallow.post_load diff --git a/airflow/api_connexion/schemas/config_schema.py b/airflow/api_connexion/schemas/config_schema.py index 4af3db6063b19..e1665a7fbaae6 100644 --- a/airflow/api_connexion/schemas/config_schema.py +++ b/airflow/api_connexion/schemas/config_schema.py @@ -54,4 +54,4 @@ class Config(NamedTuple): sections: List[ConfigSection] -config_schema = ConfigSchema(strict=True) +config_schema = ConfigSchema() diff --git a/airflow/api_connexion/schemas/connection_schema.py b/airflow/api_connexion/schemas/connection_schema.py index 451a3f5376bea..9c2eba368b6aa 100644 --- a/airflow/api_connexion/schemas/connection_schema.py +++ b/airflow/api_connexion/schemas/connection_schema.py @@ -17,7 +17,7 @@ # under the License. from typing import List, NamedTuple -from marshmallow import Schema, ValidationError, fields, validates_schema +from marshmallow import Schema, fields from marshmallow_sqlalchemy import SQLAlchemySchema, auto_field from airflow.models.connection import Connection @@ -39,16 +39,6 @@ class Meta: schema = auto_field() port = auto_field() - # Marshmallow 2 doesn't have support for excluding extra field - # We will be able to remove this when we upgrade to marshmallow 3. - # To remove it, we would need to set unknown=EXCLUDE in Meta - @validates_schema(pass_original=True) - def check_unknown_fields(self, data, original_data): # pylint: disable=unused-argument - """ Validates unknown field """ - unknown = set(original_data) - set(self.fields) - if unknown: - raise ValidationError(f'Extra arguments passed: {list(unknown)}') - class ConnectionSchema(ConnectionCollectionItemSchema): # pylint: disable=too-many-ancestors """ @@ -71,6 +61,6 @@ class ConnectionCollectionSchema(Schema): total_entries = fields.Int() -connection_schema = ConnectionSchema(strict=True) -connection_collection_item_schema = ConnectionCollectionItemSchema(strict=True) -connection_collection_schema = ConnectionCollectionSchema(strict=True) +connection_schema = ConnectionSchema() +connection_collection_item_schema = ConnectionCollectionItemSchema() +connection_collection_schema = ConnectionCollectionSchema() diff --git a/airflow/api_connexion/schemas/dag_run_schema.py b/airflow/api_connexion/schemas/dag_run_schema.py index 0af33eef972d3..f6b4cbccfcc5b 100644 --- a/airflow/api_connexion/schemas/dag_run_schema.py +++ b/airflow/api_connexion/schemas/dag_run_schema.py @@ -28,12 +28,12 @@ class ConfObject(fields.Field): """ The conf field""" - def _serialize(self, value, attr, obj): + def _serialize(self, value, attr, obj, **kwargs): if not value: return {} return json.loads(value) if isinstance(value, str) else value - def _deserialize(self, value, attr, data): + def _deserialize(self, value, attr, data, **kwargs): if isinstance(value, str): return json.loads(value) return value @@ -49,7 +49,7 @@ class Meta: model = DagRun dateformat = 'iso' - run_id = auto_field(dump_to='dag_run_id', load_from='dag_run_id') + run_id = auto_field(data_key='dag_run_id') dag_id = auto_field(dump_only=True) execution_date = auto_field() start_date = auto_field(dump_only=True) diff --git a/airflow/api_connexion/schemas/dag_schema.py b/airflow/api_connexion/schemas/dag_schema.py index 5104d708c0821..aff859a0f1e3b 100644 --- a/airflow/api_connexion/schemas/dag_schema.py +++ b/airflow/api_connexion/schemas/dag_schema.py @@ -39,7 +39,6 @@ class DAGSchema(SQLAlchemySchema): class Meta: """Meta""" - model = DagModel dag_id = auto_field(dump_only=True) @@ -56,7 +55,7 @@ class Meta: def get_owners(obj: DagModel): """Convert owners attribute to DAG representation""" - if not obj.owners: + if not getattr(obj, 'owners', None): return [] return obj.owners.split(",") diff --git a/airflow/api_connexion/schemas/error_schema.py b/airflow/api_connexion/schemas/error_schema.py index d9fb310344c57..3fee9a9f73600 100644 --- a/airflow/api_connexion/schemas/error_schema.py +++ b/airflow/api_connexion/schemas/error_schema.py @@ -29,8 +29,6 @@ class Meta: """Meta""" model = ImportError - load_instance = True - exclude = ("id", "stacktrace") import_error_id = auto_field("id", dump_only=True) timestamp = auto_field(format="iso") @@ -52,5 +50,5 @@ class ImportErrorCollectionSchema(Schema): total_entries = fields.Int() -import_error_schema = ImportErrorSchema(strict=True) -import_error_collection_schema = ImportErrorCollectionSchema(strict=True) +import_error_schema = ImportErrorSchema() +import_error_collection_schema = ImportErrorCollectionSchema() diff --git a/airflow/api_connexion/schemas/event_log_schema.py b/airflow/api_connexion/schemas/event_log_schema.py index 480325e9b57e8..0753a8a104a44 100644 --- a/airflow/api_connexion/schemas/event_log_schema.py +++ b/airflow/api_connexion/schemas/event_log_schema.py @@ -30,8 +30,8 @@ class Meta: """ Meta """ model = Log - id = auto_field(dump_to='event_log_id', dump_only=True) - dttm = auto_field(dump_to='when', dump_only=True) + id = auto_field(data_key='event_log_id', dump_only=True) + dttm = auto_field(data_key='when', dump_only=True) dag_id = auto_field(dump_only=True) task_id = auto_field(dump_only=True) event = auto_field(dump_only=True) @@ -53,5 +53,5 @@ class EventLogCollectionSchema(Schema): total_entries = fields.Int() -event_log_schema = EventLogSchema(strict=True) -event_log_collection_schema = EventLogCollectionSchema(strict=True) +event_log_schema = EventLogSchema() +event_log_collection_schema = EventLogCollectionSchema() diff --git a/airflow/api_connexion/schemas/log_schema.py b/airflow/api_connexion/schemas/log_schema.py index 2e48fc732d93d..e59416fe4e0bc 100644 --- a/airflow/api_connexion/schemas/log_schema.py +++ b/airflow/api_connexion/schemas/log_schema.py @@ -32,4 +32,4 @@ class LogResponseObject(NamedTuple): continuation_token: str -logs_schema = LogsSchema(strict=True) +logs_schema = LogsSchema() diff --git a/airflow/api_connexion/schemas/pool_schema.py b/airflow/api_connexion/schemas/pool_schema.py index 1c8b938ddb3f9..a785c26c622e5 100644 --- a/airflow/api_connexion/schemas/pool_schema.py +++ b/airflow/api_connexion/schemas/pool_schema.py @@ -17,7 +17,7 @@ from typing import List, NamedTuple -from marshmallow import Schema, ValidationError, fields, validates_schema +from marshmallow import Schema, fields from marshmallow_sqlalchemy import SQLAlchemySchema, auto_field from airflow.models.pool import Pool @@ -28,9 +28,7 @@ class PoolSchema(SQLAlchemySchema): class Meta: """Meta""" - model = Pool - exclude = ("pool",) name = auto_field("pool") slots = auto_field() @@ -67,13 +65,6 @@ def get_open_slots(obj: Pool) -> int: """ return obj.open_slots() - @validates_schema(pass_original=True) - def check_unknown_fields(self, data, original_data): # pylint: disable=unused-argument - """ Validates unknown field """ - unknown = set(original_data) - set(self.fields) - if unknown: - raise ValidationError(f"Extra arguments passed: {list(unknown)}") - class PoolCollection(NamedTuple): """List of Pools with metadata""" @@ -89,5 +80,5 @@ class PoolCollectionSchema(Schema): total_entries = fields.Int() -pool_collection_schema = PoolCollectionSchema(strict=True) -pool_schema = PoolSchema(strict=True) +pool_collection_schema = PoolCollectionSchema() +pool_schema = PoolSchema() diff --git a/airflow/api_connexion/schemas/variable_schema.py b/airflow/api_connexion/schemas/variable_schema.py index 37198cddb96d9..c0c7bd1585dcd 100644 --- a/airflow/api_connexion/schemas/variable_schema.py +++ b/airflow/api_connexion/schemas/variable_schema.py @@ -30,5 +30,5 @@ class VariableCollectionSchema(Schema): total_entries = fields.Int() -variable_schema = VariableSchema(strict=True) -variable_collection_schema = VariableCollectionSchema(strict=True) +variable_schema = VariableSchema() +variable_collection_schema = VariableCollectionSchema() diff --git a/airflow/api_connexion/schemas/version_schema.py b/airflow/api_connexion/schemas/version_schema.py index 7a982ebcf583e..e2ca25528b86c 100644 --- a/airflow/api_connexion/schemas/version_schema.py +++ b/airflow/api_connexion/schemas/version_schema.py @@ -24,4 +24,4 @@ class VersionInfoSchema(Schema): git_version = fields.String(dump_only=True) -version_info_schema = VersionInfoSchema(strict=True) +version_info_schema = VersionInfoSchema() diff --git a/airflow/api_connexion/schemas/xcom_schema.py b/airflow/api_connexion/schemas/xcom_schema.py index 5adc36da34da6..9d06c1b05ac0f 100644 --- a/airflow/api_connexion/schemas/xcom_schema.py +++ b/airflow/api_connexion/schemas/xcom_schema.py @@ -58,6 +58,6 @@ class XComCollectionSchema(Schema): total_entries = fields.Int() -xcom_schema = XComSchema(strict=True) -xcom_collection_item_schema = XComCollectionItemSchema(strict=True) -xcom_collection_schema = XComCollectionSchema(strict=True) +xcom_schema = XComSchema() +xcom_collection_item_schema = XComCollectionItemSchema() +xcom_collection_schema = XComCollectionSchema() diff --git a/setup.py b/setup.py index 502f28988cebd..7006ff9d52080 100644 --- a/setup.py +++ b/setup.py @@ -694,7 +694,7 @@ def is_package_excluded(package: str, exclusion_list: List[str]): 'cryptography>=0.9.3', 'dill>=0.2.2, <0.4', 'flask>=1.1.0, <2.0', - 'flask-appbuilder~=2.3.4', + 'flask-appbuilder>2.3.4,~=3.0', 'flask-caching>=1.3.3, <1.4.0', 'flask-login>=0.3, <0.5', 'flask-swagger==0.2.13', @@ -710,7 +710,7 @@ def is_package_excluded(package: str, exclusion_list: List[str]): 'lockfile>=0.12.2', 'markdown>=2.5.2, <3.0', 'markupsafe>=1.1.1, <2.0', - 'marshmallow-oneofschema<2', + 'marshmallow-oneofschema>=2.0.1', 'pandas>=0.17.1, <2.0', 'pendulum~=2.0', 'pep562~=1.0;python_version<"3.7"', diff --git a/tests/api_connexion/endpoints/test_connection_endpoint.py b/tests/api_connexion/endpoints/test_connection_endpoint.py index 45f38b0f2b9c7..b3a623ce5c703 100644 --- a/tests/api_connexion/endpoints/test_connection_endpoint.py +++ b/tests/api_connexion/endpoints/test_connection_endpoint.py @@ -385,7 +385,7 @@ def test_patch_should_response_400_for_invalid_fields_in_update_mask( "connection_id": "test-connection-id", "conn_type": "test-type", "extras": "{}", # extras not a known field e.g typo - }, "Extra arguments passed: ['extras']" + }, "extras" ), ( { @@ -393,7 +393,7 @@ def test_patch_should_response_400_for_invalid_fields_in_update_mask( "conn_type": "test-type", "invalid_field": "invalid field", # unknown field "_password": "{}", # _password not a known field - }, "Extra arguments passed:" + }, "_password" ), ] ) diff --git a/tests/api_connexion/endpoints/test_dag_endpoint.py b/tests/api_connexion/endpoints/test_dag_endpoint.py index 926140160c281..6289b6fb26677 100644 --- a/tests/api_connexion/endpoints/test_dag_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_endpoint.py @@ -82,6 +82,7 @@ def test_should_response_200(self): 'is_paused': None, 'is_subdag': False, 'orientation': 'LR', + 'owners': [], 'schedule_interval': { '__type': 'TimeDelta', 'days': 1, @@ -115,6 +116,7 @@ def test_should_response_200_serialized(self): 'is_paused': None, 'is_subdag': False, 'orientation': 'LR', + 'owners': [], 'schedule_interval': { '__type': 'TimeDelta', 'days': 1, diff --git a/tests/api_connexion/endpoints/test_pool_endpoint.py b/tests/api_connexion/endpoints/test_pool_endpoint.py index 5f036549f8618..e8e4cfb7630f9 100644 --- a/tests/api_connexion/endpoints/test_pool_endpoint.py +++ b/tests/api_connexion/endpoints/test_pool_endpoint.py @@ -244,7 +244,7 @@ def test_response_409(self, session): ( "for extra fields", {"name": "invalid_pool", "slots": 3, "extra_field_1": "extra"}, - "Extra arguments passed: ['extra_field_1']", + "{'extra_field_1': ['Unknown field.']}", ), ] ) @@ -292,7 +292,7 @@ def test_response_200(self, session): ("'slots' is a required property", {"name": "test_pool_a"}), # Extra properties ( - "Extra arguments passed: ['extra_field']", + "{'extra_field': ['Unknown field.']}", {"name": "test_pool_a", "slots": 3, "extra_field": "extra"}, ), ] diff --git a/tests/api_connexion/endpoints/test_variable_endpoint.py b/tests/api_connexion/endpoints/test_variable_endpoint.py index 72d61fc124725..46238c40f1251 100644 --- a/tests/api_connexion/endpoints/test_variable_endpoint.py +++ b/tests/api_connexion/endpoints/test_variable_endpoint.py @@ -181,5 +181,5 @@ def test_should_reject_invalid_request(self): "title": "Invalid Variable schema", "status": 400, "type": "about:blank", - "detail": "{'value': ['Missing data for required field.']}", + "detail": "{'value': ['Missing data for required field.'], 'v': ['Unknown field.']}", } diff --git a/tests/api_connexion/schemas/test_common_schema.py b/tests/api_connexion/schemas/test_common_schema.py index d0419b0895875..618989dfca340 100644 --- a/tests/api_connexion/schemas/test_common_schema.py +++ b/tests/api_connexion/schemas/test_common_schema.py @@ -32,7 +32,7 @@ def test_should_serialize(self): result = schema_instance.dump(instance) self.assertEqual( {"__type": "TimeDelta", "days": 12, "seconds": 0, "microseconds": 0}, - result.data + result ) def test_should_deserialize(self): @@ -40,7 +40,7 @@ def test_should_deserialize(self): schema_instance = TimeDeltaSchema() result = schema_instance.load(instance) expected_instance = datetime.timedelta(days=12) - self.assertEqual(expected_instance, result.data) + self.assertEqual(expected_instance, result) class TestRelativeDeltaSchema(unittest.TestCase): @@ -67,7 +67,7 @@ def test_should_serialize(self): "year": None, "years": 0, }, - result.data, + result, ) def test_should_deserialize(self): @@ -75,7 +75,7 @@ def test_should_deserialize(self): schema_instance = RelativeDeltaSchema() result = schema_instance.load(instance) expected_instance = relativedelta.relativedelta(days=+12) - self.assertEqual(expected_instance, result.data) + self.assertEqual(expected_instance, result) class TestCronExpressionSchema(unittest.TestCase): @@ -84,7 +84,7 @@ def test_should_deserialize(self): schema_instance = CronExpressionSchema() result = schema_instance.load(instance) expected_instance = CronExpression("5 4 * * *") - self.assertEqual(expected_instance, result.data) + self.assertEqual(expected_instance, result) class TestScheduleIntervalSchema(unittest.TestCase): @@ -94,7 +94,7 @@ def test_should_serialize_timedelta(self): result = schema_instance.dump(instance) self.assertEqual( {"__type": "TimeDelta", "days": 12, "seconds": 0, "microseconds": 0}, - result.data + result ) def test_should_deserialize_timedelta(self): @@ -102,7 +102,7 @@ def test_should_deserialize_timedelta(self): schema_instance = ScheduleIntervalSchema() result = schema_instance.load(instance) expected_instance = datetime.timedelta(days=12) - self.assertEqual(expected_instance, result.data) + self.assertEqual(expected_instance, result) def test_should_serialize_relative_delta(self): instance = relativedelta.relativedelta(days=+12) @@ -127,7 +127,7 @@ def test_should_serialize_relative_delta(self): "year": None, "years": 0, }, - result.data, + result, ) def test_should_deserialize_relative_delta(self): @@ -135,11 +135,11 @@ def test_should_deserialize_relative_delta(self): schema_instance = ScheduleIntervalSchema() result = schema_instance.load(instance) expected_instance = relativedelta.relativedelta(days=+12) - self.assertEqual(expected_instance, result.data) + self.assertEqual(expected_instance, result) def test_should_serialize_cron_expresssion(self): instance = "5 4 * * *" schema_instance = ScheduleIntervalSchema() result = schema_instance.dump(instance) expected_instance = {"__type": "CronExpression", "value": "5 4 * * *"} - self.assertEqual(expected_instance, result.data) + self.assertEqual(expected_instance, result) diff --git a/tests/api_connexion/schemas/test_config_schema.py b/tests/api_connexion/schemas/test_config_schema.py index 34941599c8610..473dabc37a0a4 100644 --- a/tests/api_connexion/schemas/test_config_schema.py +++ b/tests/api_connexion/schemas/test_config_schema.py @@ -55,4 +55,4 @@ def test_serialize(self): }, ] } - assert result.data == expected + assert result == expected diff --git a/tests/api_connexion/schemas/test_connection_schema.py b/tests/api_connexion/schemas/test_connection_schema.py index 5afe1d069db60..dea887fb50c9a 100644 --- a/tests/api_connexion/schemas/test_connection_schema.py +++ b/tests/api_connexion/schemas/test_connection_schema.py @@ -51,7 +51,7 @@ def test_serialize(self, session): connection_model = session.query(Connection).first() deserialized_connection = connection_collection_item_schema.dump(connection_model) self.assertEqual( - deserialized_connection[0], + deserialized_connection, { 'connection_id': "mysql_default", 'conn_type': 'mysql', @@ -79,7 +79,7 @@ def test_deserialize(self): result_2 = connection_collection_item_schema.load(connection_dump_2) self.assertEqual( - result_1[0], + result_1, { 'conn_id': "mysql_default_1", 'conn_type': 'mysql', @@ -90,7 +90,7 @@ def test_deserialize(self): } ) self.assertEqual( - result_2[0], + result_2, { 'conn_id': "mysql_default_2", 'conn_type': "postgres", @@ -136,7 +136,7 @@ def test_serialize(self, session): ) deserialized_connections = connection_collection_schema.dump(instance) self.assertEqual( - deserialized_connections[0], + deserialized_connections, { 'connections': [ { @@ -187,7 +187,7 @@ def test_serialize(self, session): connection_model = session.query(Connection).first() deserialized_connection = connection_schema.dump(connection_model) self.assertEqual( - deserialized_connection[0], + deserialized_connection, { 'connection_id': "mysql_default", 'conn_type': 'mysql', @@ -211,7 +211,7 @@ def test_deserialize(self): } result = connection_schema.load(den) self.assertEqual( - result[0], + result, { 'conn_id': "mysql_default", 'conn_type': 'mysql', diff --git a/tests/api_connexion/schemas/test_dag_run_schema.py b/tests/api_connexion/schemas/test_dag_run_schema.py index f39ce24c44274..18b7859b29f9f 100644 --- a/tests/api_connexion/schemas/test_dag_run_schema.py +++ b/tests/api_connexion/schemas/test_dag_run_schema.py @@ -18,6 +18,7 @@ import unittest from dateutil.parser import parse +from marshmallow import ValidationError from airflow.api_connexion.schemas.dag_run_schema import ( DAGRunCollection, dagrun_collection_schema, dagrun_schema, @@ -55,7 +56,7 @@ def test_serialze(self, session): deserialized_dagrun = dagrun_schema.dump(dagrun_model) self.assertEqual( - deserialized_dagrun[0], + deserialized_dagrun, { 'dag_id': None, 'dag_run_id': 'my-dag-run', @@ -73,19 +74,15 @@ def test_deserialize(self): # and conf are loaded. # dag_run_id should be loaded as run_id serialized_dagrun = { - 'dag_id': None, 'dag_run_id': 'my-dag-run', - 'end_date': None, 'state': 'failed', 'execution_date': self.default_time, - 'external_trigger': True, - 'start_date': self.default_time, 'conf': '{"start": "stop"}' } result = dagrun_schema.load(serialized_dagrun) self.assertEqual( - result.data, + result, { 'run_id': 'my-dag-run', 'execution_date': parse(self.default_time), @@ -95,27 +92,19 @@ def test_deserialize(self): ) def test_deserialize_2(self): - # Invalid state field should return None + # loading dump_only field raises serialized_dagrun = { 'dag_id': None, 'dag_run_id': 'my-dag-run', 'end_date': None, - 'state': 'faileds', + 'state': 'failed', 'execution_date': self.default_time, 'external_trigger': True, 'start_date': self.default_time, 'conf': {"start": "stop"} } - - result = dagrun_schema.load(serialized_dagrun) - self.assertEqual( - result.data, - { - 'run_id': 'my-dag-run', - 'execution_date': parse(self.default_time), - 'conf': {"start": "stop"} - } - ) + with self.assertRaises(ValidationError): + dagrun_schema.load(serialized_dagrun) class TestDagRunCollection(TestDAGRunBase): @@ -142,7 +131,7 @@ def test_serialize(self, session): total_entries=2) deserialized_dagruns = dagrun_collection_schema.dump(instance) self.assertEqual( - deserialized_dagruns.data, + deserialized_dagruns, { 'dag_runs': [ { diff --git a/tests/api_connexion/schemas/test_dag_schema.py b/tests/api_connexion/schemas/test_dag_schema.py index 327bce5c0d06a..1fb0f0b30f653 100644 --- a/tests/api_connexion/schemas/test_dag_schema.py +++ b/tests/api_connexion/schemas/test_dag_schema.py @@ -51,7 +51,7 @@ def test_serialize(self): "schedule_interval": {"__type": "CronExpression", "value": "5 4 * * *"}, "tags": [{"name": "tag-1"}, {"name": "tag-2"}], }, - serialized_dag.data, + serialized_dag, ) @@ -89,7 +89,7 @@ def test_serialize(self): ], "total_entries": 2, }, - schema.dump(instance).data, + schema.dump(instance), ) @@ -115,9 +115,10 @@ def test_serialize(self): 'is_paused': None, 'is_subdag': False, 'orientation': 'LR', + 'owners': [], 'schedule_interval': {'__type': 'TimeDelta', 'days': 1, 'seconds': 0, 'microseconds': 0}, 'start_date': '2020-06-19T00:00:00+00:00', 'tags': None, 'timezone': "Timezone('UTC')" } - assert schema.dump(dag).data == expected + assert schema.dump(dag) == expected diff --git a/tests/api_connexion/schemas/test_error_schema.py b/tests/api_connexion/schemas/test_error_schema.py index 4f349f5bbb762..521e3e6a76948 100644 --- a/tests/api_connexion/schemas/test_error_schema.py +++ b/tests/api_connexion/schemas/test_error_schema.py @@ -44,7 +44,7 @@ def test_serialize(self, session): ) session.add(import_error) session.commit() - serialized_data = import_error_schema.dump(import_error).data + serialized_data = import_error_schema.dump(import_error) serialized_data["import_error_id"] = 1 self.assertEqual( { @@ -76,14 +76,12 @@ def test_serialize(self, session): session.commit() query = session.query(ImportError) query_list = query.all() - serialized_data = ( - import_error_collection_schema.dump( - ImportErrorCollection(import_errors=query_list, total_entries=2) - ).data, + serialized_data = import_error_collection_schema.dump( + ImportErrorCollection(import_errors=query_list, total_entries=2) ) # To maintain consistency in the key sequence accross the db in tests - serialized_data[0]["import_errors"][0]["import_error_id"] = 1 - serialized_data[0]["import_errors"][1]["import_error_id"] = 2 + serialized_data["import_errors"][0]["import_error_id"] = 1 + serialized_data["import_errors"][1]["import_error_id"] = 2 self.assertEqual( { "import_errors": [ @@ -102,5 +100,5 @@ def test_serialize(self, session): ], "total_entries": 2, }, - serialized_data[0], + serialized_data, ) diff --git a/tests/api_connexion/schemas/test_event_log_schema.py b/tests/api_connexion/schemas/test_event_log_schema.py index d6c2730ee3977..8e58c923722a1 100644 --- a/tests/api_connexion/schemas/test_event_log_schema.py +++ b/tests/api_connexion/schemas/test_event_log_schema.py @@ -60,7 +60,7 @@ def test_serialize(self, session): log_model = session.query(Log).first() deserialized_log = event_log_schema.dump(log_model) self.assertEqual( - deserialized_log[0], + deserialized_log, { "event_log_id": event_log_model.id, "event": "TEST_EVENT", @@ -95,7 +95,7 @@ def test_serialize(self, session): total_entries=2) deserialized_event_logs = event_log_collection_schema.dump(instance) self.assertEqual( - deserialized_event_logs.data, + deserialized_event_logs, { "event_logs": [ { diff --git a/tests/api_connexion/schemas/test_pool_schemas.py b/tests/api_connexion/schemas/test_pool_schemas.py index 18c2f34b0bbb2..6750ad10c999c 100644 --- a/tests/api_connexion/schemas/test_pool_schemas.py +++ b/tests/api_connexion/schemas/test_pool_schemas.py @@ -38,7 +38,7 @@ def test_serialize(self, session): pool_instance = session.query(Pool).filter(Pool.pool == pool_model.pool).first() serialized_pool = pool_schema.dump(pool_instance) self.assertEqual( - serialized_pool.data, + serialized_pool, { "name": "test_pool", "slots": 2, @@ -53,7 +53,7 @@ def test_serialize(self, session): def test_desearialize(self, session): pool_dict = {"name": "test_pool", "slots": 3} deserialized_pool = pool_schema.load(pool_dict, session=session) - self.assertNotIsInstance(deserialized_pool.data, Pool) # Checks if load_instance is set to True + self.assertNotIsInstance(deserialized_pool, Pool) # Checks if load_instance is set to True class TestPoolCollectionSchema(unittest.TestCase): @@ -89,5 +89,5 @@ def test_serialize(self): ], "total_entries": 2, }, - pool_collection_schema.dump(instance).data, + pool_collection_schema.dump(instance), ) diff --git a/tests/api_connexion/schemas/test_task_schema.py b/tests/api_connexion/schemas/test_task_schema.py index a8048698dc0e7..96ad28a487ecd 100644 --- a/tests/api_connexion/schemas/test_task_schema.py +++ b/tests/api_connexion/schemas/test_task_schema.py @@ -56,7 +56,7 @@ def test_serialize(self): "wait_for_downstream": False, "weight_rule": "downstream", } - assert expected == result.data + assert expected == result class TestTaskCollectionSchema: @@ -96,4 +96,4 @@ def test_serialize(self): ], "total_entries": 1, } - assert expected == result.data + assert expected == result diff --git a/tests/api_connexion/schemas/test_version_schema.py b/tests/api_connexion/schemas/test_version_schema.py index 7b96308e5d630..b5f7b2476e39d 100644 --- a/tests/api_connexion/schemas/test_version_schema.py +++ b/tests/api_connexion/schemas/test_version_schema.py @@ -31,7 +31,7 @@ class TestVersionInfoSchema(unittest.TestCase): ]) def test_serialize(self, git_commit): version_info = VersionInfo("VERSION", git_commit) - current_data = version_info_schema.dump(version_info).data + current_data = version_info_schema.dump(version_info) expected_result = {'version': 'VERSION', 'git_version': git_commit} self.assertEqual(expected_result, current_data) diff --git a/tests/api_connexion/schemas/test_xcom_schema.py b/tests/api_connexion/schemas/test_xcom_schema.py index d66c8ce58f89b..ef079834b9ba0 100644 --- a/tests/api_connexion/schemas/test_xcom_schema.py +++ b/tests/api_connexion/schemas/test_xcom_schema.py @@ -64,7 +64,7 @@ def test_serialize(self, session): xcom_model = session.query(XCom).first() deserialized_xcom = xcom_collection_item_schema.dump(xcom_model) self.assertEqual( - deserialized_xcom[0], + deserialized_xcom, { 'key': 'test_key', 'timestamp': self.default_time, @@ -84,7 +84,7 @@ def test_deserialize(self): } result = xcom_collection_item_schema.load(xcom_dump) self.assertEqual( - result[0], + result, { 'key': 'test_key', 'timestamp': self.default_time_parsed, @@ -132,7 +132,7 @@ def test_serialize(self, session): total_entries=xcom_models_query.count(), )) self.assertEqual( - deserialized_xcoms[0], + deserialized_xcoms, { 'xcom_entries': [ { @@ -177,7 +177,7 @@ def test_serialize(self, session): xcom_model = session.query(XCom).first() deserialized_xcom = xcom_schema.dump(xcom_model) self.assertEqual( - deserialized_xcom[0], + deserialized_xcom, { 'key': 'test_key', 'timestamp': self.default_time, @@ -199,7 +199,7 @@ def test_deserialize(self): } result = xcom_schema.load(xcom_dump) self.assertEqual( - result[0], + result, { 'key': 'test_key', 'timestamp': self.default_time_parsed, From aafb56fe6adc39d709407d5946518527ccfde4f0 Mon Sep 17 00:00:00 2001 From: EphraimBuddy Date: Sat, 4 Jul 2020 01:06:44 +0100 Subject: [PATCH 02/11] generate requirements --- requirements/requirements-python3.6.txt | 42 ++++++++++++------------- requirements/requirements-python3.7.txt | 40 +++++++++++------------ requirements/requirements-python3.8.txt | 39 ++++++++++++----------- requirements/setup-3.6.md5 | 2 +- requirements/setup-3.7.md5 | 2 +- requirements/setup-3.8.md5 | 2 +- 6 files changed, 64 insertions(+), 63 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 7ad9df2bea022..0099c9fca6af8 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -1,7 +1,7 @@ # Editable install with no version control (apache-airflow==2.0.0.dev0) Authlib==0.14.3 Babel==2.8.0 -Flask-AppBuilder==2.3.4 +Flask-AppBuilder==3.0.0 Flask-Babel==1.0.0 Flask-Bcrypt==0.7.1 Flask-Caching==1.3.3 @@ -27,7 +27,7 @@ PySmbClient==0.1.5 PyYAML==5.3.1 Pygments==2.6.1 SQLAlchemy-JSONField==0.9.0 -SQLAlchemy-Utils==0.36.6 +SQLAlchemy-Utils==0.36.7 SQLAlchemy==1.3.18 Sphinx==3.1.1 Unidecode==1.1.1 @@ -41,7 +41,7 @@ amqp==2.6.0 analytics-python==1.2.9 ansiwrap==0.8.4 apipkg==1.5 -apispec==1.3.3 +apispec==3.3.1 appdirs==1.4.4 argcomplete==1.11.1 asn1crypto==1.3.0 @@ -57,11 +57,11 @@ azure-common==1.1.25 azure-cosmos==3.1.2 azure-datalake-store==0.0.48 azure-kusto-data==0.0.45 -azure-mgmt-containerinstance==1.5.0 +azure-mgmt-containerinstance==2.0.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 azure-mgmt-nspkg==3.0.2 -azure-mgmt-resource==10.0.0 +azure-mgmt-resource==10.1.0 azure-nspkg==3.0.2 azure-storage-blob==2.1.0 azure-storage-common==2.1.0 @@ -72,9 +72,9 @@ beautifulsoup4==4.7.1 billiard==3.6.3.0 black==19.10b0 blinker==1.4 -boto3==1.14.14 +boto3==1.14.16 boto==2.49.0 -botocore==1.17.14 +botocore==1.17.16 bowler==0.8.0 cached-property==1.5.1 cachetools==4.1.1 @@ -90,7 +90,7 @@ chardet==3.0.4 click==6.7 clickclick==1.2.2 cloudant==2.13.0 -cloudpickle==1.4.1 +cloudpickle==1.5.0 colorama==0.4.3 colorlog==4.0.2 connexion==2.7.0 @@ -100,13 +100,13 @@ croniter==0.3.34 cryptography==2.9.2 curlify==2.2.1 cx-Oracle==8.0.0 -dask==2.19.0 -datadog==0.37.1 +dask==2.20.0 +datadog==0.38.0 decorator==4.4.2 defusedxml==0.6.0 dill==0.3.2 distlib==0.3.1 -distributed==2.19.0 +distributed==2.20.0 dnspython==1.16.0 docker-pycreds==0.4.0 docker==3.7.3 @@ -128,7 +128,7 @@ flake8-colors==0.1.6 flake8==3.8.3 flaky==3.6.1 flask-swagger==0.2.13 -flower==0.9.4 +flower==0.9.5 freezegun==0.3.15 fsspec==0.7.4 funcsigs==1.0.2 @@ -178,16 +178,15 @@ gunicorn==19.10.0 hdfs==2.5.8 hmsclient==0.1.1 httplib2==0.18.1 -humanize==0.5.1 +humanize==2.4.1 hvac==0.10.4 -identify==1.4.20 +identify==1.4.21 idna-ssl==1.1.0 idna==2.10 -ijson==2.6.1 imagesize==1.2.0 immutables==0.14 importlib-metadata==1.7.0 -importlib-resources==2.0.1 +importlib-resources==3.0.0 inflection==0.5.0 ipdb==0.13.3 ipython-genutils==0.2.0 @@ -214,9 +213,9 @@ lazy-object-proxy==1.5.0 ldap3==2.7 lockfile==0.12.2 marshmallow-enum==1.5.1 -marshmallow-oneofschema==1.0.6 +marshmallow-oneofschema==2.0.1 marshmallow-sqlalchemy==0.23.1 -marshmallow==2.21.0 +marshmallow==3.6.1 mccabe==0.6.1 mock==4.0.2 mongomock==3.19.0 @@ -260,9 +259,10 @@ pickleshare==0.7.5 pinotdb==0.1.1 pipdeptree==1.0.0 pluggy==0.13.1 -pre-commit==2.5.1 +pre-commit==2.6.0 presto-python-client==0.7.0 prison==0.1.3 +prometheus-client==0.8.0 prompt-toolkit==3.0.5 protobuf==3.12.2 psutil==5.7.0 @@ -275,7 +275,7 @@ pyarrow==0.17.1 pyasn1-modules==0.2.8 pyasn1==0.4.8 pycodestyle==2.6.0 -pycountry==19.8.18 +pycountry==20.7.3 pycparser==2.20 pycryptodomex==3.9.8 pydata-google-auth==1.1.0 @@ -328,7 +328,7 @@ sasl==0.2.1 semver==2.10.2 sendgrid==6.4.1 sentinels==1.0.0 -sentry-sdk==0.15.1 +sentry-sdk==0.16.0 setproctitle==1.1.10 sh==1.13.1 simple-salesforce==1.1.0 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index f35909f598668..518688db18f34 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -1,7 +1,7 @@ # Editable install with no version control (apache-airflow==2.0.0.dev0) Authlib==0.14.3 Babel==2.8.0 -Flask-AppBuilder==2.3.4 +Flask-AppBuilder==3.0.0 Flask-Babel==1.0.0 Flask-Bcrypt==0.7.1 Flask-Caching==1.3.3 @@ -27,7 +27,7 @@ PySmbClient==0.1.5 PyYAML==5.3.1 Pygments==2.6.1 SQLAlchemy-JSONField==0.9.0 -SQLAlchemy-Utils==0.36.6 +SQLAlchemy-Utils==0.36.7 SQLAlchemy==1.3.18 Sphinx==3.1.1 Unidecode==1.1.1 @@ -41,7 +41,7 @@ amqp==2.6.0 analytics-python==1.2.9 ansiwrap==0.8.4 apipkg==1.5 -apispec==1.3.3 +apispec==3.3.1 appdirs==1.4.4 argcomplete==1.11.1 asn1crypto==1.3.0 @@ -57,11 +57,11 @@ azure-common==1.1.25 azure-cosmos==3.1.2 azure-datalake-store==0.0.48 azure-kusto-data==0.0.45 -azure-mgmt-containerinstance==1.5.0 +azure-mgmt-containerinstance==2.0.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 azure-mgmt-nspkg==3.0.2 -azure-mgmt-resource==10.0.0 +azure-mgmt-resource==10.1.0 azure-nspkg==3.0.2 azure-storage-blob==2.1.0 azure-storage-common==2.1.0 @@ -72,9 +72,9 @@ beautifulsoup4==4.7.1 billiard==3.6.3.0 black==19.10b0 blinker==1.4 -boto3==1.14.14 +boto3==1.14.16 boto==2.49.0 -botocore==1.17.14 +botocore==1.17.16 bowler==0.8.0 cached-property==1.5.1 cachetools==4.1.1 @@ -90,7 +90,7 @@ chardet==3.0.4 click==6.7 clickclick==1.2.2 cloudant==2.13.0 -cloudpickle==1.4.1 +cloudpickle==1.5.0 colorama==0.4.3 colorlog==4.0.2 connexion==2.7.0 @@ -99,13 +99,13 @@ croniter==0.3.34 cryptography==2.9.2 curlify==2.2.1 cx-Oracle==8.0.0 -dask==2.19.0 -datadog==0.37.1 +dask==2.20.0 +datadog==0.38.0 decorator==4.4.2 defusedxml==0.6.0 dill==0.3.2 distlib==0.3.1 -distributed==2.19.0 +distributed==2.20.0 dnspython==1.16.0 docker-pycreds==0.4.0 docker==3.7.3 @@ -127,7 +127,7 @@ flake8-colors==0.1.6 flake8==3.8.3 flaky==3.6.1 flask-swagger==0.2.13 -flower==0.9.4 +flower==0.9.5 freezegun==0.3.15 fsspec==0.7.4 funcsigs==1.0.2 @@ -177,11 +177,10 @@ gunicorn==19.10.0 hdfs==2.5.8 hmsclient==0.1.1 httplib2==0.18.1 -humanize==0.5.1 +humanize==2.4.1 hvac==0.10.4 -identify==1.4.20 +identify==1.4.21 idna==2.10 -ijson==2.6.1 imagesize==1.2.0 importlib-metadata==1.7.0 inflection==0.5.0 @@ -210,9 +209,9 @@ lazy-object-proxy==1.5.0 ldap3==2.7 lockfile==0.12.2 marshmallow-enum==1.5.1 -marshmallow-oneofschema==1.0.6 +marshmallow-oneofschema==2.0.1 marshmallow-sqlalchemy==0.23.1 -marshmallow==2.21.0 +marshmallow==3.6.1 mccabe==0.6.1 mock==4.0.2 mongomock==3.19.0 @@ -255,9 +254,10 @@ pickleshare==0.7.5 pinotdb==0.1.1 pipdeptree==1.0.0 pluggy==0.13.1 -pre-commit==2.5.1 +pre-commit==2.6.0 presto-python-client==0.7.0 prison==0.1.3 +prometheus-client==0.8.0 prompt-toolkit==3.0.5 protobuf==3.12.2 psutil==5.7.0 @@ -270,7 +270,7 @@ pyarrow==0.17.1 pyasn1-modules==0.2.8 pyasn1==0.4.8 pycodestyle==2.6.0 -pycountry==19.8.18 +pycountry==20.7.3 pycparser==2.20 pycryptodomex==3.9.8 pydata-google-auth==1.1.0 @@ -323,7 +323,7 @@ sasl==0.2.1 semver==2.10.2 sendgrid==6.4.1 sentinels==1.0.0 -sentry-sdk==0.15.1 +sentry-sdk==0.16.0 setproctitle==1.1.10 sh==1.13.1 simple-salesforce==1.1.0 diff --git a/requirements/requirements-python3.8.txt b/requirements/requirements-python3.8.txt index 147d4ef1c4b5c..a218319147f6d 100644 --- a/requirements/requirements-python3.8.txt +++ b/requirements/requirements-python3.8.txt @@ -1,7 +1,7 @@ # Editable install with no version control (apache-airflow==2.0.0.dev0) Authlib==0.14.3 Babel==2.8.0 -Flask-AppBuilder==2.3.4 +Flask-AppBuilder==3.0.0 Flask-Babel==1.0.0 Flask-Bcrypt==0.7.1 Flask-Caching==1.3.3 @@ -27,7 +27,7 @@ PySmbClient==0.1.5 PyYAML==5.3.1 Pygments==2.6.1 SQLAlchemy-JSONField==0.9.0 -SQLAlchemy-Utils==0.36.6 +SQLAlchemy-Utils==0.36.7 SQLAlchemy==1.3.18 Sphinx==3.1.1 Unidecode==1.1.1 @@ -41,7 +41,7 @@ amqp==2.6.0 analytics-python==1.2.9 ansiwrap==0.8.4 apipkg==1.5 -apispec==1.3.3 +apispec==3.3.1 appdirs==1.4.4 argcomplete==1.11.1 asn1crypto==1.3.0 @@ -57,11 +57,11 @@ azure-common==1.1.25 azure-cosmos==3.1.2 azure-datalake-store==0.0.48 azure-kusto-data==0.0.45 -azure-mgmt-containerinstance==1.5.0 +azure-mgmt-containerinstance==2.0.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 azure-mgmt-nspkg==3.0.2 -azure-mgmt-resource==10.0.0 +azure-mgmt-resource==10.1.0 azure-nspkg==3.0.2 azure-storage-blob==2.1.0 azure-storage-common==2.1.0 @@ -72,9 +72,9 @@ beautifulsoup4==4.7.1 billiard==3.6.3.0 black==19.10b0 blinker==1.4 -boto3==1.14.14 +boto3==1.14.16 boto==2.49.0 -botocore==1.17.14 +botocore==1.17.16 bowler==0.8.0 cached-property==1.5.1 cachetools==4.1.1 @@ -90,7 +90,7 @@ chardet==3.0.4 click==6.7 clickclick==1.2.2 cloudant==2.13.0 -cloudpickle==1.4.1 +cloudpickle==1.5.0 colorama==0.4.3 colorlog==4.0.2 connexion==2.7.0 @@ -99,13 +99,13 @@ croniter==0.3.34 cryptography==2.9.2 curlify==2.2.1 cx-Oracle==8.0.0 -dask==2.19.0 -datadog==0.37.1 +dask==2.20.0 +datadog==0.38.0 decorator==4.4.2 defusedxml==0.6.0 dill==0.3.2 distlib==0.3.1 -distributed==2.19.0 +distributed==2.20.0 dnspython==1.16.0 docker-pycreds==0.4.0 docker==3.7.3 @@ -127,7 +127,7 @@ flake8-colors==0.1.6 flake8==3.8.3 flaky==3.6.1 flask-swagger==0.2.13 -flower==0.9.4 +flower==0.9.5 freezegun==0.3.15 fsspec==0.7.4 funcsigs==1.0.2 @@ -177,9 +177,9 @@ gunicorn==19.10.0 hdfs==2.5.8 hmsclient==0.1.1 httplib2==0.18.1 -humanize==0.5.1 +humanize==2.4.1 hvac==0.10.4 -identify==1.4.20 +identify==1.4.21 idna==2.10 ijson==2.6.1 imagesize==1.2.0 @@ -210,9 +210,9 @@ lazy-object-proxy==1.5.0 ldap3==2.7 lockfile==0.12.2 marshmallow-enum==1.5.1 -marshmallow-oneofschema==1.0.6 +marshmallow-oneofschema==2.0.1 marshmallow-sqlalchemy==0.23.1 -marshmallow==2.21.0 +marshmallow==3.6.1 mccabe==0.6.1 mock==4.0.2 mongomock==3.19.0 @@ -255,9 +255,10 @@ pickleshare==0.7.5 pinotdb==0.1.1 pipdeptree==1.0.0 pluggy==0.13.1 -pre-commit==2.5.1 +pre-commit==2.6.0 presto-python-client==0.7.0 prison==0.1.3 +prometheus-client==0.8.0 prompt-toolkit==3.0.5 protobuf==3.12.2 psutil==5.7.0 @@ -270,7 +271,7 @@ pyarrow==0.17.1 pyasn1-modules==0.2.8 pyasn1==0.4.8 pycodestyle==2.6.0 -pycountry==19.8.18 +pycountry==20.7.3 pycparser==2.20 pycryptodomex==3.9.8 pydata-google-auth==1.1.0 @@ -322,7 +323,7 @@ sasl==0.2.1 semver==2.10.2 sendgrid==6.4.1 sentinels==1.0.0 -sentry-sdk==0.15.1 +sentry-sdk==0.16.0 setproctitle==1.1.10 sh==1.13.1 simple-salesforce==1.1.0 diff --git a/requirements/setup-3.6.md5 b/requirements/setup-3.6.md5 index 8665cb3828be6..17731ae1cd942 100644 --- a/requirements/setup-3.6.md5 +++ b/requirements/setup-3.6.md5 @@ -1 +1 @@ -e50c855fa7de97b2eca8152389400a5b /opt/airflow/setup.py +f952ff749a71cc655549773ea2d9396c /opt/airflow/setup.py diff --git a/requirements/setup-3.7.md5 b/requirements/setup-3.7.md5 index 8665cb3828be6..17731ae1cd942 100644 --- a/requirements/setup-3.7.md5 +++ b/requirements/setup-3.7.md5 @@ -1 +1 @@ -e50c855fa7de97b2eca8152389400a5b /opt/airflow/setup.py +f952ff749a71cc655549773ea2d9396c /opt/airflow/setup.py diff --git a/requirements/setup-3.8.md5 b/requirements/setup-3.8.md5 index 8665cb3828be6..17731ae1cd942 100644 --- a/requirements/setup-3.8.md5 +++ b/requirements/setup-3.8.md5 @@ -1 +1 @@ -e50c855fa7de97b2eca8152389400a5b /opt/airflow/setup.py +f952ff749a71cc655549773ea2d9396c /opt/airflow/setup.py From 46cb523b62d5404222bbdcc45e3005eecdcd8c12 Mon Sep 17 00:00:00 2001 From: EphraimBuddy Date: Sat, 4 Jul 2020 13:45:36 +0100 Subject: [PATCH 03/11] pin azure-mgmt-containerinstance to less than 2.0 --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index 7006ff9d52080..c818450042301 100644 --- a/setup.py +++ b/setup.py @@ -194,7 +194,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version 'azure-cosmos>=3.0.1,<4', 'azure-datalake-store>=0.0.45', 'azure-kusto-data>=0.0.43,<0.1', - 'azure-mgmt-containerinstance>=1.5.0', + 'azure-mgmt-containerinstance>=1.5.0,<2.0', 'azure-mgmt-datalake-store>=0.5.0', 'azure-mgmt-resource>=2.2.0', 'azure-storage>=0.34.0, <0.37.0', From aef92c0122aeb68aba668c1d681f27893f4700c3 Mon Sep 17 00:00:00 2001 From: EphraimBuddy Date: Sun, 5 Jul 2020 01:41:08 +0100 Subject: [PATCH 04/11] fixup! generate requirements --- requirements/requirements-python3.6.txt | 2 +- requirements/requirements-python3.7.txt | 2 +- requirements/requirements-python3.8.txt | 2 +- requirements/setup-3.6.md5 | 2 +- requirements/setup-3.7.md5 | 2 +- requirements/setup-3.8.md5 | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 0099c9fca6af8..3027a3cd8d6c3 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -57,7 +57,7 @@ azure-common==1.1.25 azure-cosmos==3.1.2 azure-datalake-store==0.0.48 azure-kusto-data==0.0.45 -azure-mgmt-containerinstance==2.0.0 +azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 azure-mgmt-nspkg==3.0.2 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index 518688db18f34..ab1e7fc2d59a3 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -57,7 +57,7 @@ azure-common==1.1.25 azure-cosmos==3.1.2 azure-datalake-store==0.0.48 azure-kusto-data==0.0.45 -azure-mgmt-containerinstance==2.0.0 +azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 azure-mgmt-nspkg==3.0.2 diff --git a/requirements/requirements-python3.8.txt b/requirements/requirements-python3.8.txt index a218319147f6d..357fdc9b18555 100644 --- a/requirements/requirements-python3.8.txt +++ b/requirements/requirements-python3.8.txt @@ -57,7 +57,7 @@ azure-common==1.1.25 azure-cosmos==3.1.2 azure-datalake-store==0.0.48 azure-kusto-data==0.0.45 -azure-mgmt-containerinstance==2.0.0 +azure-mgmt-containerinstance==1.5.0 azure-mgmt-datalake-nspkg==3.0.1 azure-mgmt-datalake-store==0.5.0 azure-mgmt-nspkg==3.0.2 diff --git a/requirements/setup-3.6.md5 b/requirements/setup-3.6.md5 index 17731ae1cd942..833489473cffc 100644 --- a/requirements/setup-3.6.md5 +++ b/requirements/setup-3.6.md5 @@ -1 +1 @@ -f952ff749a71cc655549773ea2d9396c /opt/airflow/setup.py +74363dd615d8faff7cac7bd7a917b470 /opt/airflow/setup.py diff --git a/requirements/setup-3.7.md5 b/requirements/setup-3.7.md5 index 17731ae1cd942..833489473cffc 100644 --- a/requirements/setup-3.7.md5 +++ b/requirements/setup-3.7.md5 @@ -1 +1 @@ -f952ff749a71cc655549773ea2d9396c /opt/airflow/setup.py +74363dd615d8faff7cac7bd7a917b470 /opt/airflow/setup.py diff --git a/requirements/setup-3.8.md5 b/requirements/setup-3.8.md5 index 17731ae1cd942..833489473cffc 100644 --- a/requirements/setup-3.8.md5 +++ b/requirements/setup-3.8.md5 @@ -1 +1 @@ -f952ff749a71cc655549773ea2d9396c /opt/airflow/setup.py +74363dd615d8faff7cac7bd7a917b470 /opt/airflow/setup.py From 42daf4abd4b72325b8b467ae0cc73d1277345428 Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Sun, 5 Jul 2020 08:28:22 +0100 Subject: [PATCH 05/11] Set distributed to 1.19 on requirements --- requirements/requirements-python3.6.txt | 2 +- requirements/requirements-python3.7.txt | 2 +- requirements/requirements-python3.8.txt | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 3027a3cd8d6c3..a38e731fe1a90 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -106,7 +106,7 @@ decorator==4.4.2 defusedxml==0.6.0 dill==0.3.2 distlib==0.3.1 -distributed==2.20.0 +distributed==2.19.0 dnspython==1.16.0 docker-pycreds==0.4.0 docker==3.7.3 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index ab1e7fc2d59a3..aa98525ca639c 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -105,7 +105,7 @@ decorator==4.4.2 defusedxml==0.6.0 dill==0.3.2 distlib==0.3.1 -distributed==2.20.0 +distributed==2.19.0 dnspython==1.16.0 docker-pycreds==0.4.0 docker==3.7.3 diff --git a/requirements/requirements-python3.8.txt b/requirements/requirements-python3.8.txt index 357fdc9b18555..e1ca809444f9b 100644 --- a/requirements/requirements-python3.8.txt +++ b/requirements/requirements-python3.8.txt @@ -105,7 +105,7 @@ decorator==4.4.2 defusedxml==0.6.0 dill==0.3.2 distlib==0.3.1 -distributed==2.20.0 +distributed==2.19.0 dnspython==1.16.0 docker-pycreds==0.4.0 docker==3.7.3 From 0ba4d6e132c6056fe790f02fdc05fffc006311ad Mon Sep 17 00:00:00 2001 From: EphraimBuddy Date: Sun, 5 Jul 2020 10:27:31 +0100 Subject: [PATCH 06/11] pin distributed to 2.19 as 2.20 has broken --- setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/setup.py b/setup.py index c818450042301..59a91118dc361 100644 --- a/setup.py +++ b/setup.py @@ -215,7 +215,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version 'cloudant>=2.0', ] dask = [ - 'distributed>=2.11.1, <3', + 'distributed>=2.11.1, <2.20', ] databricks = [ 'requests>=2.20.0, <3', From f76ea32c73c8cd6cb600f89af5d9d32eb0b925b8 Mon Sep 17 00:00:00 2001 From: EphraimBuddy Date: Sun, 5 Jul 2020 11:27:31 +0100 Subject: [PATCH 07/11] fixup! generate requirements --- requirements/setup-3.6.md5 | 2 +- requirements/setup-3.7.md5 | 2 +- requirements/setup-3.8.md5 | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/requirements/setup-3.6.md5 b/requirements/setup-3.6.md5 index 833489473cffc..975416b07cf42 100644 --- a/requirements/setup-3.6.md5 +++ b/requirements/setup-3.6.md5 @@ -1 +1 @@ -74363dd615d8faff7cac7bd7a917b470 /opt/airflow/setup.py +5bf5f3a314d75a5bce72cb3c27e5e7fb /opt/airflow/setup.py diff --git a/requirements/setup-3.7.md5 b/requirements/setup-3.7.md5 index 833489473cffc..975416b07cf42 100644 --- a/requirements/setup-3.7.md5 +++ b/requirements/setup-3.7.md5 @@ -1 +1 @@ -74363dd615d8faff7cac7bd7a917b470 /opt/airflow/setup.py +5bf5f3a314d75a5bce72cb3c27e5e7fb /opt/airflow/setup.py diff --git a/requirements/setup-3.8.md5 b/requirements/setup-3.8.md5 index 833489473cffc..975416b07cf42 100644 --- a/requirements/setup-3.8.md5 +++ b/requirements/setup-3.8.md5 @@ -1 +1 @@ -74363dd615d8faff7cac7bd7a917b470 /opt/airflow/setup.py +5bf5f3a314d75a5bce72cb3c27e5e7fb /opt/airflow/setup.py From f9c02eaa3d24f3e8becfce6512664446090a7626 Mon Sep 17 00:00:00 2001 From: EphraimBuddy Date: Sun, 5 Jul 2020 21:50:46 +0100 Subject: [PATCH 08/11] pin cloudpickle to v1.4.1 --- setup.py | 1 + 1 file changed, 1 insertion(+) diff --git a/setup.py b/setup.py index 59a91118dc361..a956049a07a24 100644 --- a/setup.py +++ b/setup.py @@ -216,6 +216,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version ] dask = [ 'distributed>=2.11.1, <2.20', + 'cloudpickle>=1.4.1, <1.5' ] databricks = [ 'requests>=2.20.0, <3', From 72dae45ad0f251bd66636077a675ed8955fd3cab Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi Date: Sun, 5 Jul 2020 21:58:45 +0100 Subject: [PATCH 09/11] pin cloudpickle to 1.4.1 --- requirements/requirements-python3.6.txt | 2 +- requirements/requirements-python3.7.txt | 2 +- requirements/requirements-python3.8.txt | 2 +- setup.py | 10 +++++----- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index a38e731fe1a90..0a2bfe8e83b31 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -90,7 +90,7 @@ chardet==3.0.4 click==6.7 clickclick==1.2.2 cloudant==2.13.0 -cloudpickle==1.5.0 +cloudpickle==1.4.1 colorama==0.4.3 colorlog==4.0.2 connexion==2.7.0 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index aa98525ca639c..eb28f957e8fe2 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -90,7 +90,7 @@ chardet==3.0.4 click==6.7 clickclick==1.2.2 cloudant==2.13.0 -cloudpickle==1.5.0 +cloudpickle==1.4.1 colorama==0.4.3 colorlog==4.0.2 connexion==2.7.0 diff --git a/requirements/requirements-python3.8.txt b/requirements/requirements-python3.8.txt index e1ca809444f9b..ae7aa8bb817de 100644 --- a/requirements/requirements-python3.8.txt +++ b/requirements/requirements-python3.8.txt @@ -90,7 +90,7 @@ chardet==3.0.4 click==6.7 clickclick==1.2.2 cloudant==2.13.0 -cloudpickle==1.5.0 +cloudpickle==1.4.1 colorama==0.4.3 colorlog==4.0.2 connexion==2.7.0 diff --git a/setup.py b/setup.py index a956049a07a24..cc932e81f7c8d 100644 --- a/setup.py +++ b/setup.py @@ -215,8 +215,8 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version 'cloudant>=2.0', ] dask = [ - 'distributed>=2.11.1, <2.20', - 'cloudpickle>=1.4.1, <1.5' + 'cloudpickle>=1.4.1, <1.5.0', + 'distributed>=2.11.1, <2.20' ] databricks = [ 'requests>=2.20.0, <3', @@ -589,7 +589,7 @@ def write_version(filename: str = os.path.join(*[my_dir, "airflow", "git_version 'jdbc': jdbc, 'jira': jira, 'kerberos': kerberos, - 'kubernetes': kubernetes, # TODO: remove this in Airflow 2.1 + 'kubernetes': kubernetes, # TODO: remove this in Airflow 2.1 'ldap': ldap, "microsoft.azure": azure, "microsoft.mssql": mssql, @@ -665,11 +665,11 @@ def is_package_excluded(package: str, exclusion_list: List[str]): devel_all = [package for package in devel_all if not is_package_excluded( package=package, exclusion_list=PACKAGES_EXCLUDED_FOR_ALL) -] + ] devel_ci = [package for package in devel_all if not is_package_excluded( package=package, exclusion_list=PACKAGES_EXCLUDED_FOR_CI + PACKAGES_EXCLUDED_FOR_ALL) -] + ] EXTRAS_REQUIREMENTS.update( { From 3be83b7b2adaaf9e42304367eb42e96a4c0a2951 Mon Sep 17 00:00:00 2001 From: EphraimBuddy Date: Mon, 6 Jul 2020 00:46:15 +0100 Subject: [PATCH 10/11] fixup generate-requirements --- requirements/requirements-python3.6.txt | 4 ++-- requirements/requirements-python3.7.txt | 4 ++-- requirements/requirements-python3.8.txt | 4 ++-- requirements/setup-3.6.md5 | 2 +- requirements/setup-3.7.md5 | 2 +- requirements/setup-3.8.md5 | 2 +- setup.py | 4 ++-- 7 files changed, 11 insertions(+), 11 deletions(-) diff --git a/requirements/requirements-python3.6.txt b/requirements/requirements-python3.6.txt index 0a2bfe8e83b31..f349dbdd9a018 100644 --- a/requirements/requirements-python3.6.txt +++ b/requirements/requirements-python3.6.txt @@ -29,7 +29,7 @@ Pygments==2.6.1 SQLAlchemy-JSONField==0.9.0 SQLAlchemy-Utils==0.36.7 SQLAlchemy==1.3.18 -Sphinx==3.1.1 +Sphinx==3.1.2 Unidecode==1.1.1 WTForms==2.3.1 Werkzeug==0.16.1 @@ -178,7 +178,7 @@ gunicorn==19.10.0 hdfs==2.5.8 hmsclient==0.1.1 httplib2==0.18.1 -humanize==2.4.1 +humanize==2.5.0 hvac==0.10.4 identify==1.4.21 idna-ssl==1.1.0 diff --git a/requirements/requirements-python3.7.txt b/requirements/requirements-python3.7.txt index eb28f957e8fe2..ec496aacd9130 100644 --- a/requirements/requirements-python3.7.txt +++ b/requirements/requirements-python3.7.txt @@ -29,7 +29,7 @@ Pygments==2.6.1 SQLAlchemy-JSONField==0.9.0 SQLAlchemy-Utils==0.36.7 SQLAlchemy==1.3.18 -Sphinx==3.1.1 +Sphinx==3.1.2 Unidecode==1.1.1 WTForms==2.3.1 Werkzeug==0.16.1 @@ -177,7 +177,7 @@ gunicorn==19.10.0 hdfs==2.5.8 hmsclient==0.1.1 httplib2==0.18.1 -humanize==2.4.1 +humanize==2.5.0 hvac==0.10.4 identify==1.4.21 idna==2.10 diff --git a/requirements/requirements-python3.8.txt b/requirements/requirements-python3.8.txt index ae7aa8bb817de..e074d8bcc695b 100644 --- a/requirements/requirements-python3.8.txt +++ b/requirements/requirements-python3.8.txt @@ -29,7 +29,7 @@ Pygments==2.6.1 SQLAlchemy-JSONField==0.9.0 SQLAlchemy-Utils==0.36.7 SQLAlchemy==1.3.18 -Sphinx==3.1.1 +Sphinx==3.1.2 Unidecode==1.1.1 WTForms==2.3.1 Werkzeug==0.16.1 @@ -177,7 +177,7 @@ gunicorn==19.10.0 hdfs==2.5.8 hmsclient==0.1.1 httplib2==0.18.1 -humanize==2.4.1 +humanize==2.5.0 hvac==0.10.4 identify==1.4.21 idna==2.10 diff --git a/requirements/setup-3.6.md5 b/requirements/setup-3.6.md5 index 975416b07cf42..3bb8aeca1cd4a 100644 --- a/requirements/setup-3.6.md5 +++ b/requirements/setup-3.6.md5 @@ -1 +1 @@ -5bf5f3a314d75a5bce72cb3c27e5e7fb /opt/airflow/setup.py +b700789f2707044af6738608f83d481c /opt/airflow/setup.py diff --git a/requirements/setup-3.7.md5 b/requirements/setup-3.7.md5 index 975416b07cf42..3bb8aeca1cd4a 100644 --- a/requirements/setup-3.7.md5 +++ b/requirements/setup-3.7.md5 @@ -1 +1 @@ -5bf5f3a314d75a5bce72cb3c27e5e7fb /opt/airflow/setup.py +b700789f2707044af6738608f83d481c /opt/airflow/setup.py diff --git a/requirements/setup-3.8.md5 b/requirements/setup-3.8.md5 index 975416b07cf42..3bb8aeca1cd4a 100644 --- a/requirements/setup-3.8.md5 +++ b/requirements/setup-3.8.md5 @@ -1 +1 @@ -5bf5f3a314d75a5bce72cb3c27e5e7fb /opt/airflow/setup.py +b700789f2707044af6738608f83d481c /opt/airflow/setup.py diff --git a/setup.py b/setup.py index cc932e81f7c8d..c285380b38963 100644 --- a/setup.py +++ b/setup.py @@ -665,11 +665,11 @@ def is_package_excluded(package: str, exclusion_list: List[str]): devel_all = [package for package in devel_all if not is_package_excluded( package=package, exclusion_list=PACKAGES_EXCLUDED_FOR_ALL) - ] +] devel_ci = [package for package in devel_all if not is_package_excluded( package=package, exclusion_list=PACKAGES_EXCLUDED_FOR_CI + PACKAGES_EXCLUDED_FOR_ALL) - ] +] EXTRAS_REQUIREMENTS.update( { From f5d79f78ce8a8243438d8d15ca3baf9c17f694ef Mon Sep 17 00:00:00 2001 From: EphraimBuddy Date: Mon, 6 Jul 2020 02:01:04 +0100 Subject: [PATCH 11/11] fix generate requirements --- requirements/setup-3.6.md5 | 2 +- requirements/setup-3.7.md5 | 2 +- requirements/setup-3.8.md5 | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/requirements/setup-3.6.md5 b/requirements/setup-3.6.md5 index 3bb8aeca1cd4a..5453c7532106f 100644 --- a/requirements/setup-3.6.md5 +++ b/requirements/setup-3.6.md5 @@ -1 +1 @@ -b700789f2707044af6738608f83d481c /opt/airflow/setup.py +2f6a2d89bfb00fa5bc04bc00f2d488e7 /opt/airflow/setup.py diff --git a/requirements/setup-3.7.md5 b/requirements/setup-3.7.md5 index 3bb8aeca1cd4a..5453c7532106f 100644 --- a/requirements/setup-3.7.md5 +++ b/requirements/setup-3.7.md5 @@ -1 +1 @@ -b700789f2707044af6738608f83d481c /opt/airflow/setup.py +2f6a2d89bfb00fa5bc04bc00f2d488e7 /opt/airflow/setup.py diff --git a/requirements/setup-3.8.md5 b/requirements/setup-3.8.md5 index 3bb8aeca1cd4a..5453c7532106f 100644 --- a/requirements/setup-3.8.md5 +++ b/requirements/setup-3.8.md5 @@ -1 +1 @@ -b700789f2707044af6738608f83d481c /opt/airflow/setup.py +2f6a2d89bfb00fa5bc04bc00f2d488e7 /opt/airflow/setup.py