Ver Fonte

Periodic Tasks deprecated in favor of CELERYBEAT_SCHEDULE setting.

Example::

CELERYBEAT_SCHEDULE = {
    "add.often": dict(name="tasks.add", schedule=timedelta(seconds=10),
                      args=(), kwargs={}, options={}),
}
Ask Solem há 15 anos atrás
pai
commit
f508afbf90
5 ficheiros alterados com 281 adições e 265 exclusões
  1. 28 32
      celery/beat.py
  2. 2 0
      celery/conf.py
  3. 225 0
      celery/schedules.py
  4. 25 10
      celery/task/base.py
  5. 1 223
      celery/task/schedules.py

+ 28 - 32
celery/beat.py

@@ -7,13 +7,14 @@ import time
 import shelve
 import shelve
 import threading
 import threading
 import multiprocessing
 import multiprocessing
-from datetime import datetime
+from datetime import datetime, timedelta
 from UserDict import UserDict
 from UserDict import UserDict
 
 
 from celery import log
 from celery import log
 from celery import conf
 from celery import conf
 from celery import platform
 from celery import platform
 from celery.execute import send_task
 from celery.execute import send_task
+from celery.schedules import schedule
 from celery.messaging import establish_connection
 from celery.messaging import establish_connection
 from celery.utils.info import humanize_seconds
 from celery.utils.info import humanize_seconds
 
 
@@ -159,15 +160,24 @@ class Scheduler(UserDict):
                     entry.name, exc))
                     entry.name, exc))
         return result
         return result
 
 
+    def maybe_schedule(self, s, relative=False):
+        if isinstance(s, int):
+            return timedelta(seconds=s)
+        if isinstance(s, timedelta):
+            return schedule(s, relative)
+        return s
+
     def setup_schedule(self):
     def setup_schedule(self):
-        from datetime import timedelta
-        from celery.task.schedules import schedule
-        self.schedule["add.often"] = ScheduleEntry("tasks.add",
-                                      schedule(timedelta(seconds=5)),
-                                      args=(4, 4))
-        self.schedule["sleep.often"] = ScheduleEntry("tasks.sleeptask",
-                                      schedule(timedelta(minutes=1)),
-                                      args=(2, ))
+        self.schedule = self.dict_to_entries(conf.CELERYBEAT_SCHEDULE)
+
+    def dict_to_entries(self, dict_):
+        entries = {}
+        for name, entry in dict_.items():
+            relative = entry.pop("relative", None)
+            entry["schedule"] = self.maybe_schedule(entry["schedule"],
+                                                    relative)
+            entries[name] = ScheduleEntry(**entry)
+        return entries
 
 
     def cleanup(self):
     def cleanup(self):
         pass
         pass
@@ -183,25 +193,23 @@ class ClockService(object):
 
 
     def __init__(self, logger=None,
     def __init__(self, logger=None,
             max_interval=conf.CELERYBEAT_MAX_LOOP_INTERVAL,
             max_interval=conf.CELERYBEAT_MAX_LOOP_INTERVAL,
+            schedule=conf.CELERYBEAT_SCHEDULE,
             schedule_filename=conf.CELERYBEAT_SCHEDULE_FILENAME):
             schedule_filename=conf.CELERYBEAT_SCHEDULE_FILENAME):
         self.logger = logger or log.get_default_logger()
         self.logger = logger or log.get_default_logger()
         self.max_interval = max_interval
         self.max_interval = max_interval
-        self.schedule_filename = schedule_filename
         self._shutdown = threading.Event()
         self._shutdown = threading.Event()
         self._stopped = threading.Event()
         self._stopped = threading.Event()
-        self._schedule = None
+        self.schedule = schedule
         self._scheduler = None
         self._scheduler = None
-        self._in_sync = False
         silence = self.max_interval < 60 and 10 or 1
         silence = self.max_interval < 60 and 10 or 1
         self.debug = log.SilenceRepeated(self.logger.debug,
         self.debug = log.SilenceRepeated(self.logger.debug,
                                          max_iterations=silence)
                                          max_iterations=silence)
 
 
     def start(self, embedded_process=False):
     def start(self, embedded_process=False):
-        self.logger.info("ClockService: Starting...")
-        self.logger.debug("ClockService: "
-            "Ticking with max interval->%s, schedule->%s" % (
-                    humanize_seconds(self.max_interval),
-                    self.schedule_filename))
+        self.logger.info("Celerybeat: Starting...")
+        self.logger.debug("Celerybeat: "
+            "Ticking with max interval->%s" % (
+                    humanize_seconds(self.max_interval)))
 
 
         if embedded_process:
         if embedded_process:
             platform.set_process_title("celerybeat")
             platform.set_process_title("celerybeat")
@@ -212,7 +220,7 @@ class ClockService(object):
                     if self._shutdown.isSet():
                     if self._shutdown.isSet():
                         break
                         break
                     interval = self.scheduler.tick()
                     interval = self.scheduler.tick()
-                    self.debug("ClockService: Waking up %s." % (
+                    self.debug("Celerybeat: Waking up %s." % (
                             humanize_seconds(interval, prefix="in ")))
                             humanize_seconds(interval, prefix="in ")))
                     time.sleep(interval)
                     time.sleep(interval)
             except (KeyboardInterrupt, SystemExit):
             except (KeyboardInterrupt, SystemExit):
@@ -221,25 +229,13 @@ class ClockService(object):
             self.sync()
             self.sync()
 
 
     def sync(self):
     def sync(self):
-        if self._schedule is not None and not self._in_sync:
-            self.logger.debug("ClockService: Syncing schedule to disk...")
-            self._schedule.sync()
-            self._schedule.close()
-            self._in_sync = True
-            self._stopped.set()
+        self._stopped.set()
 
 
     def stop(self, wait=False):
     def stop(self, wait=False):
-        self.logger.info("ClockService: Shutting down...")
+        self.logger.info("Celerybeat: Shutting down...")
         self._shutdown.set()
         self._shutdown.set()
         wait and self._stopped.wait() # block until shutdown done.
         wait and self._stopped.wait() # block until shutdown done.
 
 
-    @property
-    def schedule(self):
-        if self._schedule is None:
-            filename = self.schedule_filename
-            self._schedule = self.open_schedule(filename=filename)
-        return self._schedule
-
     @property
     @property
     def scheduler(self):
     def scheduler(self):
         if self._scheduler is None:
         if self._scheduler is None:

+ 2 - 0
celery/conf.py

@@ -56,6 +56,7 @@ _DEFAULTS = {
     "CELERYD_LOG_COLOR": False,
     "CELERYD_LOG_COLOR": False,
     "CELERYD_LOG_LEVEL": "WARN",
     "CELERYD_LOG_LEVEL": "WARN",
     "CELERYD_LOG_FILE": None, # stderr
     "CELERYD_LOG_FILE": None, # stderr
+    "CELERYBEAT_SCHEDULE": {},
     "CELERYBEAT_SCHEDULE_FILENAME": "celerybeat-schedule",
     "CELERYBEAT_SCHEDULE_FILENAME": "celerybeat-schedule",
     "CELERYBEAT_MAX_LOOP_INTERVAL": 5 * 60, # five minutes.
     "CELERYBEAT_MAX_LOOP_INTERVAL": 5 * 60, # five minutes.
     "CELERYBEAT_LOG_LEVEL": "INFO",
     "CELERYBEAT_LOG_LEVEL": "INFO",
@@ -196,6 +197,7 @@ RESULT_PERSISTENT = _get("CELERY_RESULT_PERSISTENT")
 # :--- Celery Beat                                  <-   --   --- - ----- -- #
 # :--- Celery Beat                                  <-   --   --- - ----- -- #
 CELERYBEAT_LOG_LEVEL = _get("CELERYBEAT_LOG_LEVEL")
 CELERYBEAT_LOG_LEVEL = _get("CELERYBEAT_LOG_LEVEL")
 CELERYBEAT_LOG_FILE = _get("CELERYBEAT_LOG_FILE")
 CELERYBEAT_LOG_FILE = _get("CELERYBEAT_LOG_FILE")
+CELERYBEAT_SCHEDULE = _get("CELERYBEAT_SCHEDULE")
 CELERYBEAT_SCHEDULE_FILENAME = _get("CELERYBEAT_SCHEDULE_FILENAME")
 CELERYBEAT_SCHEDULE_FILENAME = _get("CELERYBEAT_SCHEDULE_FILENAME")
 CELERYBEAT_MAX_LOOP_INTERVAL = _get("CELERYBEAT_MAX_LOOP_INTERVAL")
 CELERYBEAT_MAX_LOOP_INTERVAL = _get("CELERYBEAT_MAX_LOOP_INTERVAL")
 
 

+ 225 - 0
celery/schedules.py

@@ -0,0 +1,225 @@
+from datetime import datetime
+from pyparsing import Word, Literal, ZeroOrMore, Optional, Group, StringEnd, alphas
+
+from celery.utils import is_iterable
+from celery.utils.timeutils import timedelta_seconds, weekday, remaining
+
+__all__ = ["schedule", "crontab"]
+
+
+class schedule(object):
+    relative = False
+
+    def __init__(self, run_every=None, relative=False):
+        self.run_every = run_every
+        self.relative = relative
+
+    def remaining_estimate(self, last_run_at):
+        """Returns when the periodic task should run next as a timedelta."""
+        return remaining(last_run_at, self.run_every, relative=self.relative)
+
+    def is_due(self, last_run_at):
+        """Returns tuple of two items ``(is_due, next_time_to_run)``,
+        where next time to run is in seconds.
+
+        See :meth:`celery.task.base.PeriodicTask.is_due` for more information.
+
+        """
+        rem_delta = self.remaining_estimate(last_run_at)
+        rem = timedelta_seconds(rem_delta)
+        if rem == 0:
+            return True, timedelta_seconds(self.run_every)
+        return False, rem
+
+
+class crontab_parser(object):
+    """Parser for crontab expressions. Any expression of the form 'groups' (see
+    BNF grammar below) is accepted and expanded to a set of numbers.  These
+    numbers represent the units of time that the crontab needs to run on::
+
+        digit   :: '0'..'9'
+        dow     :: 'a'..'z'
+        number  :: digit+ | dow+
+        steps   :: number
+        range   :: number ( '-' number ) ?
+        numspec :: '*' | range
+        expr    :: numspec ( '/' steps ) ?
+        groups  :: expr ( ',' expr ) *
+
+    The parser is a general purpose one, useful for parsing hours, minutes and
+    day_of_week expressions.  Example usage::
+
+        minutes = crontab_parser(60).parse("*/15")  # yields [0,15,30,45]
+        hours = crontab_parser(24).parse("*/4")     # yields [0,4,8,12,16,20]
+        day_of_week = crontab_parser(7).parse("*")  # yields [0,1,2,3,4,5,6]
+
+    """
+
+    def __init__(self, max_=60):
+        # define the grammar structure
+        digits = "0123456789"
+        star = Literal('*')
+        number = Word(digits) | Word(alphas)
+        steps = number
+        range_ = number + Optional(Literal('-') + number)
+        numspec = star | range_
+        expr = Group(numspec) + Optional(Literal('/') + steps)
+        extra_groups = ZeroOrMore(Literal(',') + expr)
+        groups = expr + extra_groups + StringEnd()
+
+        # define parse actions
+        star.setParseAction(self._expand_star)
+        number.setParseAction(self._expand_number)
+        range_.setParseAction(self._expand_range)
+        expr.setParseAction(self._filter_steps)
+        extra_groups.setParseAction(self._ignore_comma)
+        groups.setParseAction(self._join_to_set)
+
+        self.max_ = max_
+        self.parser = groups
+
+    @staticmethod
+    def _expand_number(toks):
+        try:
+            i = int(toks[0])
+        except ValueError:
+            try:
+                i = weekday(toks[0])
+            except KeyError:
+                raise ValueError("Invalid weekday literal '%s'." % toks[0])
+        return [i]
+
+    @staticmethod
+    def _expand_range(toks):
+        if len(toks) > 1:
+            return range(toks[0], int(toks[2])+1)
+        else:
+            return toks[0]
+
+    def _expand_star(self, toks):
+        return range(self.max_)
+
+    @staticmethod
+    def _filter_steps(toks):
+        numbers = toks[0]
+        if len(toks) > 1:
+            steps = toks[2]
+            return [n for n in numbers if n % steps == 0]
+        else:
+            return numbers
+
+    @staticmethod
+    def _ignore_comma(toks):
+        return filter(lambda x: x != ',', toks)
+
+    @staticmethod
+    def _join_to_set(toks):
+        return set(toks.asList())
+
+    def parse(self, cronspec):
+        return self.parser.parseString(cronspec).pop()
+
+
+class crontab(schedule):
+    """A crontab can be used as the ``run_every`` value of a
+    :class:`PeriodicTask` to add cron-like scheduling.
+
+    Like a :manpage:`cron` job, you can specify units of time of when
+    you would like the task to execute. It is a reasonably complete
+    implementation of cron's features, so it should provide a fair
+    degree of scheduling needs.
+
+    You can specify a minute, an hour, and/or a day of the week in any
+    of the following formats:
+
+    .. attribute:: minute
+
+        - A (list of) integers from 0-59 that represent the minutes of
+          an hour of when execution should occur; or
+        - A string representing a crontab pattern.  This may get pretty
+          advanced, like `minute="*/15"` (for every quarter) or
+          `minute="1,13,30-45,50-59/2"`.
+
+    .. attribute:: hour
+
+        - A (list of) integers from 0-23 that represent the hours of
+          a day of when execution should occur; or
+        - A string representing a crontab pattern.  This may get pretty
+          advanced, like `hour="*/3"` (for every three hours) or
+          `hour="0,8-17/2"` (at midnight, and every two hours during
+          office hours).
+
+    .. attribute:: day_of_week
+
+        - A (list of) integers from 0-6, where Sunday = 0 and Saturday =
+          6, that represent the days of a week that execution should
+          occur.
+        - A string representing a crontab pattern.  This may get pretty
+          advanced, like `day_of_week="mon-fri"` (for weekdays only).
+          (Beware that `day_of_week="*/2"` does not literally mean
+          "every two days", but "every day that is divisible by two"!)
+
+    """
+
+    @staticmethod
+    def _expand_cronspec(cronspec, max_):
+        """Takes the given cronspec argument in one of the forms::
+
+            int         (like 7)
+            basestring  (like '3-5,*/15', '*', or 'monday')
+            set         (like set([0,15,30,45]))
+            list        (like [8-17])
+
+        And convert it to an (expanded) set representing all time unit
+        values on which the crontab triggers.  Only in case of the base
+        type being 'basestring', parsing occurs.  (It is fast and
+        happens only once for each crontab instance, so there is no
+        significant performance overhead involved.)
+
+        For the other base types, merely Python type conversions happen.
+
+        The argument `max_` is needed to determine the expansion of '*'.
+
+        """
+        if isinstance(cronspec, int):
+            result = set([cronspec])
+        elif isinstance(cronspec, basestring):
+            result = crontab_parser(max_).parse(cronspec)
+        elif isinstance(cronspec, set):
+            result = cronspec
+        elif is_iterable(cronspec):
+            result = set(cronspec)
+        else:
+            raise TypeError("Argument cronspec needs to be of any of the " + \
+                    "following types: int, basestring, or an iterable type. " + \
+                    "'%s' was given." % type(cronspec))
+
+        # assure the result does not exceed the max
+        for number in result:
+            if number >= max_:
+                raise ValueError("Invalid crontab pattern. Valid " + \
+                "range is 0-%d. '%d' was found." % (max_, number))
+
+        return result
+
+    def __init__(self, minute='*', hour='*', day_of_week='*',
+            nowfun=datetime.now):
+        self.hour = self._expand_cronspec(hour, 24)
+        self.minute = self._expand_cronspec(minute, 60)
+        self.day_of_week = self._expand_cronspec(day_of_week, 7)
+        self.nowfun = nowfun
+
+    def remaining_estimate(self, last_run_at):
+        # remaining_estimate controls the frequency of scheduler
+        # ticks. The scheduler needs to wake up every second in this case.
+        return 1
+
+    def is_due(self, last_run_at):
+        now = self.nowfun()
+        last = now - last_run_at
+        due, when = False, 1
+        if last.days > 0 or last.seconds > 60:
+            due = now.isoweekday() % 7 in self.day_of_week and \
+                  now.hour in self.hour and \
+                  now.minute in self.minute
+        return due, when

+ 25 - 10
celery/task/base.py

@@ -1,4 +1,5 @@
 import sys
 import sys
+import warnings
 from datetime import timedelta
 from datetime import timedelta
 
 
 from billiard.serialization import pickle
 from billiard.serialization import pickle
@@ -15,7 +16,20 @@ from celery.messaging import TaskPublisher, TaskConsumer
 from celery.messaging import establish_connection as _establish_connection
 from celery.messaging import establish_connection as _establish_connection
 from celery.exceptions import MaxRetriesExceededError, RetryTaskError
 from celery.exceptions import MaxRetriesExceededError, RetryTaskError
 
 
-from celery.task.schedules import schedule
+from celery.schedules import schedule
+
+PERIODIC_DEPRECATION_TEXT = """\
+Periodic task classes has been deprecated and will be removed
+in celery v1.4.
+
+Please use the CELERYBEAT_SCHEDULE setting instead:
+
+    CELERYBEAT_SCHEDULE = {
+        name: dict(name=task_name, schedule=run_every,
+                   args=(), kwargs={}, options={}, relative=False)
+    }
+
+"""
 
 
 
 
 class TaskType(type):
 class TaskType(type):
@@ -737,16 +751,17 @@ class PeriodicTask(Task):
             raise NotImplementedError(
             raise NotImplementedError(
                     "Periodic tasks must have a run_every attribute")
                     "Periodic tasks must have a run_every attribute")
 
 
-        # If run_every is a integer, convert it to timedelta seconds.
-        # Operate on the original class attribute so anyone accessing
-        # it directly gets the right value.
-        if isinstance(self.__class__.run_every, int):
-            self.__class__.run_every = timedelta(seconds=self.run_every)
 
 
-        # Convert timedelta to instance of schedule.
-        if isinstance(self.__class__.run_every, timedelta):
-            self.__class__.run_every = schedule(self.__class__.run_every,
-                                                self.relative)
+        warnings.warn(PERIODIC_DEPRECATION_TEXT,
+                        DeprecationWarning)
+        conf.CELERYBEAT_SCHEDULE[self.name] = {
+                "name": self.name,
+                "schedule": self.run_every,
+                "args": (),
+                "kwargs": {},
+                "options": {},
+                "relative": self.relative,
+        }
 
 
         super(PeriodicTask, self).__init__()
         super(PeriodicTask, self).__init__()
 
 

+ 1 - 223
celery/task/schedules.py

@@ -1,223 +1 @@
-from datetime import datetime
-from pyparsing import Word, Literal, ZeroOrMore, Optional, Group, StringEnd, alphas
-
-from celery.utils import is_iterable
-from celery.utils.timeutils import timedelta_seconds, weekday, remaining
-
-
-class schedule(object):
-    relative = False
-
-    def __init__(self, run_every=None, relative=False):
-        self.run_every = run_every
-        self.relative = relative
-
-    def remaining_estimate(self, last_run_at):
-        """Returns when the periodic task should run next as a timedelta."""
-        return remaining(last_run_at, self.run_every, relative=self.relative)
-
-    def is_due(self, last_run_at):
-        """Returns tuple of two items ``(is_due, next_time_to_run)``,
-        where next time to run is in seconds.
-
-        See :meth:`celery.task.base.PeriodicTask.is_due` for more information.
-
-        """
-        rem_delta = self.remaining_estimate(last_run_at)
-        rem = timedelta_seconds(rem_delta)
-        if rem == 0:
-            return True, timedelta_seconds(self.run_every)
-        return False, rem
-
-
-class crontab_parser(object):
-    """Parser for crontab expressions. Any expression of the form 'groups' (see
-    BNF grammar below) is accepted and expanded to a set of numbers.  These
-    numbers represent the units of time that the crontab needs to run on::
-
-        digit   :: '0'..'9'
-        dow     :: 'a'..'z'
-        number  :: digit+ | dow+
-        steps   :: number
-        range   :: number ( '-' number ) ?
-        numspec :: '*' | range
-        expr    :: numspec ( '/' steps ) ?
-        groups  :: expr ( ',' expr ) *
-
-    The parser is a general purpose one, useful for parsing hours, minutes and
-    day_of_week expressions.  Example usage::
-
-        minutes = crontab_parser(60).parse("*/15")  # yields [0,15,30,45]
-        hours = crontab_parser(24).parse("*/4")     # yields [0,4,8,12,16,20]
-        day_of_week = crontab_parser(7).parse("*")  # yields [0,1,2,3,4,5,6]
-
-    """
-
-    def __init__(self, max_=60):
-        # define the grammar structure
-        digits = "0123456789"
-        star = Literal('*')
-        number = Word(digits) | Word(alphas)
-        steps = number
-        range_ = number + Optional(Literal('-') + number)
-        numspec = star | range_
-        expr = Group(numspec) + Optional(Literal('/') + steps)
-        extra_groups = ZeroOrMore(Literal(',') + expr)
-        groups = expr + extra_groups + StringEnd()
-
-        # define parse actions
-        star.setParseAction(self._expand_star)
-        number.setParseAction(self._expand_number)
-        range_.setParseAction(self._expand_range)
-        expr.setParseAction(self._filter_steps)
-        extra_groups.setParseAction(self._ignore_comma)
-        groups.setParseAction(self._join_to_set)
-
-        self.max_ = max_
-        self.parser = groups
-
-    @staticmethod
-    def _expand_number(toks):
-        try:
-            i = int(toks[0])
-        except ValueError:
-            try:
-                i = weekday(toks[0])
-            except KeyError:
-                raise ValueError("Invalid weekday literal '%s'." % toks[0])
-        return [i]
-
-    @staticmethod
-    def _expand_range(toks):
-        if len(toks) > 1:
-            return range(toks[0], int(toks[2])+1)
-        else:
-            return toks[0]
-
-    def _expand_star(self, toks):
-        return range(self.max_)
-
-    @staticmethod
-    def _filter_steps(toks):
-        numbers = toks[0]
-        if len(toks) > 1:
-            steps = toks[2]
-            return [n for n in numbers if n % steps == 0]
-        else:
-            return numbers
-
-    @staticmethod
-    def _ignore_comma(toks):
-        return filter(lambda x: x != ',', toks)
-
-    @staticmethod
-    def _join_to_set(toks):
-        return set(toks.asList())
-
-    def parse(self, cronspec):
-        return self.parser.parseString(cronspec).pop()
-
-
-class crontab(schedule):
-    """A crontab can be used as the ``run_every`` value of a
-    :class:`PeriodicTask` to add cron-like scheduling.
-
-    Like a :manpage:`cron` job, you can specify units of time of when
-    you would like the task to execute. It is a reasonably complete
-    implementation of cron's features, so it should provide a fair
-    degree of scheduling needs.
-
-    You can specify a minute, an hour, and/or a day of the week in any
-    of the following formats:
-
-    .. attribute:: minute
-
-        - A (list of) integers from 0-59 that represent the minutes of
-          an hour of when execution should occur; or
-        - A string representing a crontab pattern.  This may get pretty
-          advanced, like `minute="*/15"` (for every quarter) or
-          `minute="1,13,30-45,50-59/2"`.
-
-    .. attribute:: hour
-
-        - A (list of) integers from 0-23 that represent the hours of
-          a day of when execution should occur; or
-        - A string representing a crontab pattern.  This may get pretty
-          advanced, like `hour="*/3"` (for every three hours) or
-          `hour="0,8-17/2"` (at midnight, and every two hours during
-          office hours).
-
-    .. attribute:: day_of_week
-
-        - A (list of) integers from 0-6, where Sunday = 0 and Saturday =
-          6, that represent the days of a week that execution should
-          occur.
-        - A string representing a crontab pattern.  This may get pretty
-          advanced, like `day_of_week="mon-fri"` (for weekdays only).
-          (Beware that `day_of_week="*/2"` does not literally mean
-          "every two days", but "every day that is divisible by two"!)
-
-    """
-
-    @staticmethod
-    def _expand_cronspec(cronspec, max_):
-        """Takes the given cronspec argument in one of the forms::
-
-            int         (like 7)
-            basestring  (like '3-5,*/15', '*', or 'monday')
-            set         (like set([0,15,30,45]))
-            list        (like [8-17])
-
-        And convert it to an (expanded) set representing all time unit
-        values on which the crontab triggers.  Only in case of the base
-        type being 'basestring', parsing occurs.  (It is fast and
-        happens only once for each crontab instance, so there is no
-        significant performance overhead involved.)
-
-        For the other base types, merely Python type conversions happen.
-
-        The argument `max_` is needed to determine the expansion of '*'.
-
-        """
-        if isinstance(cronspec, int):
-            result = set([cronspec])
-        elif isinstance(cronspec, basestring):
-            result = crontab_parser(max_).parse(cronspec)
-        elif isinstance(cronspec, set):
-            result = cronspec
-        elif is_iterable(cronspec):
-            result = set(cronspec)
-        else:
-            raise TypeError("Argument cronspec needs to be of any of the " + \
-                    "following types: int, basestring, or an iterable type. " + \
-                    "'%s' was given." % type(cronspec))
-
-        # assure the result does not exceed the max
-        for number in result:
-            if number >= max_:
-                raise ValueError("Invalid crontab pattern. Valid " + \
-                "range is 0-%d. '%d' was found." % (max_, number))
-
-        return result
-
-    def __init__(self, minute='*', hour='*', day_of_week='*',
-            nowfun=datetime.now):
-        self.hour = self._expand_cronspec(hour, 24)
-        self.minute = self._expand_cronspec(minute, 60)
-        self.day_of_week = self._expand_cronspec(day_of_week, 7)
-        self.nowfun = nowfun
-
-    def remaining_estimate(self, last_run_at):
-        # remaining_estimate controls the frequency of scheduler
-        # ticks. The scheduler needs to wake up every second in this case.
-        return 1
-
-    def is_due(self, last_run_at):
-        now = self.nowfun()
-        last = now - last_run_at
-        due, when = False, 1
-        if last.days > 0 or last.seconds > 60:
-            due = now.isoweekday() % 7 in self.day_of_week and \
-                  now.hour in self.hour and \
-                  now.minute in self.minute
-        return due, when
+from celery.schedules import *