Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
59 changes: 59 additions & 0 deletions sdks/python/apache_beam/ml/anomaly/detectors/offline.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#

from typing import Any
from typing import Dict
from typing import Optional

import apache_beam as beam
from apache_beam.ml.anomaly.base import AnomalyDetector
from apache_beam.ml.anomaly.specifiable import specifiable
from apache_beam.ml.inference.base import KeyedModelHandler


@specifiable
class OfflineDetector(AnomalyDetector):
"""A offline anomaly detector that uses a provided model handler for scoring.

Args:
keyed_model_handler: The model handler to use for inference.
Requires a `KeyModelHandler[Any, Row, float, Any]` instance.
run_inference_args: Optional arguments to pass to RunInference
**kwargs: Additional keyword arguments to pass to the base
AnomalyDetector class.
"""
def __init__(
self,
keyed_model_handler: KeyedModelHandler[Any, beam.Row, float, Any],
run_inference_args: Optional[Dict[str, Any]] = None,
**kwargs):
super().__init__(**kwargs)

# TODO: validate the model handler type
self._keyed_model_handler = keyed_model_handler
self._run_inference_args = run_inference_args or {}

# always override model_identifier with model_id from the detector
self._run_inference_args["model_identifier"] = self._model_id

def learn_one(self, x: beam.Row) -> None:
"""Not implemented since OfflineDetector invokes RunInference directly."""
raise NotImplementedError

def score_one(self, x: beam.Row) -> Optional[float]:
"""Not implemented since OfflineDetector invokes RunInference directly."""
raise NotImplementedError
98 changes: 92 additions & 6 deletions sdks/python/apache_beam/ml/anomaly/transforms.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,11 @@

import dataclasses
import uuid
from typing import Any
from typing import Callable
from typing import Dict
from typing import Iterable
from typing import List
from typing import Optional
from typing import Tuple
from typing import TypeVar
Expand All @@ -33,8 +35,10 @@
from apache_beam.ml.anomaly.base import AnomalyResult
from apache_beam.ml.anomaly.base import EnsembleAnomalyDetector
from apache_beam.ml.anomaly.base import ThresholdFn
from apache_beam.ml.anomaly.detectors.offline import OfflineDetector
from apache_beam.ml.anomaly.specifiable import Spec
from apache_beam.ml.anomaly.specifiable import Specifiable
from apache_beam.ml.inference.base import RunInference
from apache_beam.transforms.userstate import ReadModifyWriteStateSpec

KeyT = TypeVar('KeyT')
Expand Down Expand Up @@ -97,9 +101,11 @@ def process(
yield k1, (k2,
AnomalyResult(
example=data,
predictions=[AnomalyPrediction(
model_id=self._underlying._model_id,
score=self.score_and_learn(data))]))
predictions=[
AnomalyPrediction(
model_id=self._underlying._model_id,
score=self.score_and_learn(data))
]))

model_state.write(self._underlying)

Expand Down Expand Up @@ -325,7 +331,8 @@ def expand(
if self._aggregation_fn is None:
# simply put predictions into an iterable (list)
ret = (
post_gbk | beam.MapTuple(
post_gbk
| beam.MapTuple(
lambda k,
v: (
k[0],
Expand Down Expand Up @@ -353,7 +360,8 @@ def expand(
# We use (original_key, temp_key) as the key for GroupByKey() so that
# scores from multiple detectors per data point are grouped.
ret = (
post_gbk | beam.MapTuple(
post_gbk
| beam.MapTuple(
lambda k,
v,
agg=aggregation_fn: (
Expand Down Expand Up @@ -406,6 +414,76 @@ def expand(
return ret


class RunOfflineDetector(beam.PTransform[beam.PCollection[KeyedInputT],
beam.PCollection[KeyedOutputT]]):
"""Runs a offline anomaly detector on a PCollection of data.

This PTransform applies a `OfflineDetector` to the input data, handling
custom input/output conversion and inference.

Args:
offline_detector: The `OfflineDetector` to run.
"""
def __init__(self, offline_detector: OfflineDetector):
self._offline_detector = offline_detector

def unnest_and_convert(
self, nested: Tuple[Tuple[Any, Any], dict[str, List]]) -> KeyedOutputT:
"""Unnests and converts the model output to AnomalyResult.

Args:
nested: A tuple containing the combined key (origin key, temp key) and
a dictionary of input and output from RunInference.

Returns:
A tuple containing the original key and AnomalyResult.
"""
key, value_dict = nested
score = value_dict['output'][0]
result = AnomalyResult(
example=value_dict['input'][0],
predictions=[
AnomalyPrediction(
model_id=self._offline_detector._model_id, score=score)
])
return key[0], (key[1], result)

def expand(
self,
input: beam.PCollection[KeyedInputT]) -> beam.PCollection[KeyedOutputT]:
model_uuid = f"{self._offline_detector._model_id}:{uuid.uuid4().hex[:6]}"

# Call RunInference Transform with the keyed model handler
run_inference = RunInference(
self._offline_detector._keyed_model_handler,
**self._offline_detector._run_inference_args)

# ((orig_key, temp_key), beam.Row)
rekeyed_model_input = input | "Rekey" >> beam.Map(
lambda x: ((x[0], x[1][0]), x[1][1]))

# ((orig_key, temp_key), float)
rekeyed_model_output = (
rekeyed_model_input
| f"Call RunInference ({model_uuid})" >> run_inference)

# ((orig_key, temp_key), {'input':[row], 'output:[float]})
rekeyed_cogbk = {
'input': rekeyed_model_input, 'output': rekeyed_model_output
} | beam.CoGroupByKey()

ret = (
rekeyed_cogbk |
"Unnest and convert model output" >> beam.Map(self.unnest_and_convert))

if self._offline_detector._threshold_criterion:
ret = (
ret | f"Run Threshold Criterion ({model_uuid})" >>
RunThresholdCriterion(self._offline_detector._threshold_criterion))

return ret


class RunEnsembleDetector(beam.PTransform[beam.PCollection[KeyedInputT],
beam.PCollection[KeyedOutputT]]):
"""Runs an ensemble of anomaly detectors on a PCollection of data.
Expand All @@ -432,8 +510,14 @@ def expand(
for idx, detector in enumerate(self._ensemble_detector._sub_detectors):
if isinstance(detector, EnsembleAnomalyDetector):
results.append(
input | f"Run Ensemble Detector at index {idx} ({model_uuid})" >>
input
| f"Run Ensemble Detector at index {idx} ({model_uuid})" >>
RunEnsembleDetector(detector))
elif isinstance(detector, OfflineDetector):
results.append(
input
| f"Run Offline Detector at index {idx} ({model_uuid})" >>
RunOfflineDetector(detector))
else:
results.append(
input
Expand Down Expand Up @@ -518,6 +602,8 @@ def expand(

if isinstance(self._root_detector, EnsembleAnomalyDetector):
keyed_output = (keyed_input | RunEnsembleDetector(self._root_detector))
elif isinstance(self._root_detector, OfflineDetector):
keyed_output = (keyed_input | RunOfflineDetector(self._root_detector))
else:
keyed_output = (keyed_input | RunOneDetector(self._root_detector))

Expand Down
Loading
Loading