def extract(self, what, df: SeriesOrScalar): df = convert_to_datetime(df) if what == "CENTURY": return da.trunc(df.year / 100) elif what == "DAY": return df.day elif what == "DECADE": return da.trunc(df.year / 10) elif what == "DOW": return (df.dayofweek + 1) % 7 elif what == "DOY": return df.dayofyear elif what == "HOUR": return df.hour elif what == "MICROSECOND": return df.microsecond elif what == "MILLENNIUM": return da.trunc(df.year / 1000) elif what == "MILLISECOND": return da.trunc(1000 * df.microsecond) elif what == "MINUTE": return df.minute elif what == "MONTH": return df.month elif what == "QUARTER": return df.quarter elif what == "SECOND": return df.second elif what == "WEEK": return df.week elif what == "YEAR": return df.year else: raise NotImplementedError(f"Extraction of {what} is not (yet) implemented.")
def _round_datetime(self, *operands): df, unit = operands df = convert_to_datetime(df) unit_map = { "DAY": "D", "HOUR": "H", "MINUTE": "T", "SECOND": "S", "MICROSECOND": "U", "MILLISECOND": "L", } try: freq = unit_map[unit.upper()] return getattr(df, self.round_method)(freq) except KeyError: raise NotImplementedError( f"{self.round_method} TO {unit} is not (yet) implemented.")
class RexCallPlugin(BaseRexPlugin): """ RexCall is used for expressions, which calculate something. An example is SELECT a + b FROM ... but also a > 3 Typically, a RexCall has inputs (which can be RexNodes again) and calls a function on these inputs. The inputs can either be a column or a scalar value. """ class_name = "org.apache.calcite.rex.RexCall" OPERATION_MAPPING = { # "binary" functions "and": ReduceOperation(operation=operator.and_), "or": ReduceOperation(operation=operator.or_), ">": ReduceOperation(operation=operator.gt), ">=": ReduceOperation(operation=operator.ge), "<": ReduceOperation(operation=operator.lt), "<=": ReduceOperation(operation=operator.le), "=": ReduceOperation(operation=operator.eq), "<>": ReduceOperation(operation=operator.ne), "+": ReduceOperation(operation=operator.add), "-": ReduceOperation(operation=operator.sub), "/": ReduceOperation(operation=SQLDivisionOperator()), "*": ReduceOperation(operation=operator.mul), # special operations "case": CaseOperation(), "like": LikeOperation(), "similar to": SimilarOperation(), "not": NotOperation(), "is null": IsNullOperation(), "is not null": NotOperation().of(IsNullOperation()), "is true": IsTrueOperation(), "is not true": NotOperation().of(IsTrueOperation()), "is false": IsFalseOperation(), "is not false": NotOperation().of(IsFalseOperation()), "is unknown": IsNullOperation(), "is not unknown": NotOperation().of(IsNullOperation()), "rand": RandOperation(), "rand_integer": RandIntegerOperation(), # Unary math functions "abs": TensorScalarOperation(lambda x: x.abs(), np.abs), "acos": Operation(da.arccos), "asin": Operation(da.arcsin), "atan": Operation(da.arctan), "atan2": Operation(da.arctan2), "cbrt": Operation(da.cbrt), "ceil": CeilFloorOperation("ceil"), "cos": Operation(da.cos), "cot": Operation(lambda x: 1 / da.tan(x)), "degrees": Operation(da.degrees), "exp": Operation(da.exp), "floor": CeilFloorOperation("floor"), "log10": Operation(da.log10), "ln": Operation(da.log), # "mod": Operation(da.mod), # needs cast "power": Operation(da.power), "radians": Operation(da.radians), "round": TensorScalarOperation(lambda x, *ops: x.round(*ops), np.round), "sign": Operation(da.sign), "sin": Operation(da.sin), "tan": Operation(da.tan), "truncate": Operation(da.trunc), # string operations "||": ReduceOperation(operation=operator.add), "char_length": TensorScalarOperation(lambda x: x.str.len(), lambda x: len(x)), "upper": TensorScalarOperation(lambda x: x.str.upper(), lambda x: x.upper()), "lower": TensorScalarOperation(lambda x: x.str.lower(), lambda x: x.lower()), "position": PositionOperation(), "trim": TrimOperation(), "overlay": OverlayOperation(), "substring": SubStringOperation(), "initcap": TensorScalarOperation(lambda x: x.str.title(), lambda x: x.title()), # date/time operations "extract": ExtractOperation(), "localtime": Operation(lambda *args: pd.Timestamp.now()), "localtimestamp": Operation(lambda *args: pd.Timestamp.now()), "current_time": Operation(lambda *args: pd.Timestamp.now()), "current_date": Operation(lambda *args: pd.Timestamp.now()), "current_timestamp": Operation(lambda *args: pd.Timestamp.now()), "last_day": TensorScalarOperation( lambda x: x + pd.tseries.offsets.MonthEnd(1), lambda x: convert_to_datetime(x) + pd.tseries.offsets.MonthEnd(1), ), } def convert( self, rex: "org.apache.calcite.rex.RexNode", dc: DataContainer, context: "dask_sql.Context", ) -> SeriesOrScalar: # Prepare the operands by turning the RexNodes into python expressions operands = [ RexConverter.convert(o, dc, context=context) for o in rex.getOperands() ] # Now use the operator name in the mapping operator_name = str(rex.getOperator().getName()) operator_name = operator_name.lower() try: operation = self.OPERATION_MAPPING[operator_name] except KeyError: try: operation = context.functions[operator_name] except KeyError: raise NotImplementedError( f"{operator_name} not (yet) implemented") logger.debug( f"Executing {operator_name} on {[str(LoggableDataFrame(df)) for df in operands]}" ) if hasattr(operation, "needs_dc") and operation.needs_dc: return operation(*operands, dc=dc) else: return operation(*operands)