[ 
https://issues.apache.org/jira/browse/AIRFLOW-2799?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16653249#comment-16653249
 ] 

ASF GitHub Bot commented on AIRFLOW-2799:
-----------------------------------------

ashb closed pull request #3743: [AIRFLOW-2799] Filtering UI objects by datetime 
is broken
URL: https://github.com/apache/incubator-airflow/pull/3743
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/airflow/www/utils.py b/airflow/www/utils.py
index 0c4f4b05d6..941b9c2819 100644
--- a/airflow/www/utils.py
+++ b/airflow/www/utils.py
@@ -29,11 +29,19 @@
 import gzip
 import json
 import time
+import datetime
 
 from flask import after_this_request, request, Response
 from flask_admin.contrib.sqla.filters import FilterConverter
+from flask_admin.model.filters import BaseDateTimeFilter
 from flask_admin.model import filters
 from flask_login import current_user
+from flask_admin.contrib.sqla.filters import ( 
+    DateTimeEqualFilter, DateTimeNotEqualFilter,
+    DateTimeGreaterFilter, DateTimeSmallerFilter,
+    DateTimeBetweenFilter, DateTimeNotBetweenFilter,
+    FilterEmpty
+)
 import wtforms
 from wtforms.compat import text_type
 
@@ -41,6 +49,7 @@
 from airflow.utils.db import create_session
 from airflow.utils import timezone
 from airflow.utils.json import AirflowJsonEncoder
+from airflow.utils.timezone import make_aware
 
 AUTHENTICATE = configuration.conf.getboolean('webserver', 'AUTHENTICATE')
 
@@ -428,7 +437,26 @@ def __call__(self, field, **kwargs):
         return wtforms.widgets.core.HTMLString(html)
 
 
+# This class simply does the same as the base class, but returns
+# an aware datetime object (should be in UTC)
+class BaseUTCDateTimeFilter(BaseDateTimeFilter):
+    def clean(self, value):
+        return make_aware(datetime.datetime.strptime(value, '%Y-%m-%d 
%H:%M:%S'))
+
+class UTCDateTimeEqualFilter(DateTimeEqualFilter, BaseUTCDateTimeFilter): pass
+class UTCDateTimeNotEqualFilter(DateTimeNotEqualFilter, 
BaseUTCDateTimeFilter): pass
+class UTCDateTimeGreaterFilter(DateTimeGreaterFilter, BaseUTCDateTimeFilter): 
pass
+class UTCDateTimeSmallerFilter(DateTimeSmallerFilter, BaseUTCDateTimeFilter): 
pass
+class UTCDateTimeBetweenFilter(DateTimeBetweenFilter, BaseUTCDateTimeFilter): 
pass
+class UTCDateTimeNotBetweenFilter(DateTimeNotBetweenFilter, 
BaseUTCDateTimeFilter): pass
+
+
 class UtcFilterConverter(FilterConverter):
+    utcdatetime_filters = (UTCDateTimeEqualFilter, UTCDateTimeNotEqualFilter,
+                           UTCDateTimeGreaterFilter, UTCDateTimeSmallerFilter,
+                           UTCDateTimeBetweenFilter, 
UTCDateTimeNotBetweenFilter,
+                           FilterEmpty)
+
     @filters.convert('utcdatetime')
     def conv_utcdatetime(self, column, name, **kwargs):
-        return self.conv_datetime(column, name, **kwargs)
+        return [f(column, name, **kwargs) for f in self.utcdatetime_filters]


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Filtering UI objects by datetime is broken 
> -------------------------------------------
>
>                 Key: AIRFLOW-2799
>                 URL: https://issues.apache.org/jira/browse/AIRFLOW-2799
>             Project: Apache Airflow
>          Issue Type: Bug
>          Components: ui, webserver
>    Affects Versions: 1.10.0
>         Environment: Debian Stretch, Python 3.5.3
>            Reporter: Kevin Campbell
>            Assignee: Ash Berlin-Taylor
>            Priority: Major
>             Fix For: 1.10.1
>
>
> On master (49fd23a3ee0269e2b974648f4a823c1d0b6c12ec) searching objects via 
> the user interface is broken for datetime fields.
> Create a new installation
>  Create a test dag (example_bash_operator)
>  Start webserver and scheduler
>  Enable dag
> On web UI, go to Browse > Task Instances
>  Search for task instances with execution_date greater than 5 days ago
>  You will get an exception
> {code:java}
>                           ____/ (  (    )   )  \___
>                          /( (  (  )   _    ))  )   )\
>                        ((     (   )(    )  )   (   )  )
>                      ((/  ( _(   )   (   _) ) (  () )  )
>                     ( (  ( (_)   ((    (   )  .((_ ) .  )_
>                    ( (  )    (      (  )    )   ) . ) (   )
>                   (  (   (  (   ) (  _  ( _) ).  ) . ) ) ( )
>                   ( (  (   ) (  )   (  ))     ) _)(   )  )  )
>                  ( (  ( \ ) (    (_  ( ) ( )  )   ) )  )) ( )
>                   (  (   (  (   (_ ( ) ( _    )  ) (  )  )   )
>                  ( (  ( (  (  )     (_  )  ) )  _)   ) _( ( )
>                   ((  (   )(    (     _    )   _) _(_ (  (_ )
>                    (_((__(_(__(( ( ( |  ) ) ) )_))__))_)___)
>                    ((__)        \\||lll|l||///          \_))
>                             (   /(/ (  )  ) )\   )
>                           (    ( ( ( | | ) ) )\   )
>                            (   /(| / ( )) ) ) )) )
>                          (     ( ((((_(|)_)))))     )
>                           (      ||\(|(|)|/||     )
>                         (        |(||(||)||||        )
>                           (     //|/l|||)|\\ \     )
>                         (/ / //  /|//||||\\  \ \  \ _)
> -------------------------------------------------------------------------------
> Node: wave.diffractive.io
> -------------------------------------------------------------------------------
> Traceback (most recent call last):
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/base.py",
>  line 1116, in _execute_context
>     context = constructor(dialect, self, conn, *args)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/default.py",
>  line 649, in _init_compiled
>     for key in compiled_params
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/default.py",
>  line 649, in <genexpr>
>     for key in compiled_params
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/sql/type_api.py",
>  line 1078, in process
>     return process_param(value, dialect)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy_utc/sqltypes.py",
>  line 30, in process_bind_param
>     raise ValueError('naive datetime is disallowed')
> ValueError: naive datetime is disallowed
> The above exception was the direct cause of the following exception:
> Traceback (most recent call last):
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask/app.py",
>  line 1982, in wsgi_app
>     response = self.full_dispatch_request()
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask/app.py",
>  line 1614, in full_dispatch_request
>     rv = self.handle_user_exception(e)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask/app.py",
>  line 1517, in handle_user_exception
>     reraise(exc_type, exc_value, tb)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask/_compat.py",
>  line 33, in reraise
>     raise value
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask/app.py",
>  line 1612, in full_dispatch_request
>     rv = self.dispatch_request()
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask/app.py",
>  line 1598, in dispatch_request
>     return self.view_functions[rule.endpoint](**req.view_args)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask_admin/base.py",
>  line 69, in inner
>     return self._run_view(f, *args, **kwargs)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask_admin/base.py",
>  line 368, in _run_view
>     return fn(self, *args, **kwargs)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask_admin/model/base.py",
>  line 1818, in index_view
>     view_args.search, view_args.filters)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/flask_admin/contrib/sqla/view.py",
>  line 969, in get_list
>     count = count_query.scalar() if count_query else None
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/orm/query.py",
>  line 2843, in scalar
>     ret = self.one()
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/orm/query.py",
>  line 2814, in one
>     ret = self.one_or_none()
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/orm/query.py",
>  line 2784, in one_or_none
>     ret = list(self)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/orm/query.py",
>  line 2855, in __iter__
>     return self._execute_and_instances(context)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/orm/query.py",
>  line 2878, in _execute_and_instances
>     result = conn.execute(querycontext.statement, self._params)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/base.py",
>  line 945, in execute
>     return meth(self, multiparams, params)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/sql/elements.py",
>  line 263, in _execute_on_connection
>     return connection._execute_clauseelement(self, multiparams, params)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/base.py",
>  line 1053, in _execute_clauseelement
>     compiled_sql, distilled_params
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/base.py",
>  line 1121, in _execute_context
>     None, None)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/base.py",
>  line 1402, in _handle_dbapi_exception
>     exc_info
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/util/compat.py",
>  line 203, in raise_from_cause
>     reraise(type(exception), exception, tb=exc_tb, cause=cause)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/util/compat.py",
>  line 186, in reraise
>     raise value.with_traceback(tb)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/base.py",
>  line 1116, in _execute_context
>     context = constructor(dialect, self, conn, *args)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/default.py",
>  line 649, in _init_compiled
>     for key in compiled_params
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/engine/default.py",
>  line 649, in <genexpr>
>     for key in compiled_params
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy/sql/type_api.py",
>  line 1078, in process
>     return process_param(value, dialect)
>   File 
> "/home/kev/.virtualenvs/airflow/local/lib/python3.5/site-packages/sqlalchemy_utc/sqltypes.py",
>  line 30, in process_bind_param
>     raise ValueError('naive datetime is disallowed')
> sqlalchemy.exc.StatementError: (builtins.ValueError) naive datetime is 
> disallowed [SQL: 'SELECT count(%(count_2)s) AS count_1 \nFROM task_instance 
> \nWHERE task_instance.execution_date > %(execution_date_1)s'] [parameters: 
> [{}]]
> {code}
> This appears to have been introduced in 
> https://issues.apache.org/jira/browse/AIRFLOW-288
> I've written a patch for this, which appears to resolve the issue. Will raise 
> a PR.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to