From 461ba01f08e16ba592b65c3bf475a78871b96be4 Mon Sep 17 00:00:00 2001 From: Helen Yang Date: Thu, 29 Sep 2022 09:45:54 -0700 Subject: [PATCH 01/68] Add Data Models in Feathr (#659) * Add Data Models in Feathr This RB is to create data models based on proposal: https://microsoft-my.sharepoint.com/:w:/g/personal/djkim_linkedin_biz/EZspGt7jJlRAqHTICZg3UbcBgQQ_VncOgM48hKW--T8qkg?e=T4N3zw * Update models.py * Update models.py * Update models.py * Update models.py * Update models.py --- registry/data-models/__init__.py | 0 registry/data-models/data-model-diagram.md | 61 +++++++++ registry/data-models/models.py | 146 +++++++++++++++++++++ 3 files changed, 207 insertions(+) create mode 100644 registry/data-models/__init__.py create mode 100644 registry/data-models/data-model-diagram.md create mode 100644 registry/data-models/models.py diff --git a/registry/data-models/__init__.py b/registry/data-models/__init__.py new file mode 100644 index 000000000..e69de29bb diff --git a/registry/data-models/data-model-diagram.md b/registry/data-models/data-model-diagram.md new file mode 100644 index 000000000..e43ffa0af --- /dev/null +++ b/registry/data-models/data-model-diagram.md @@ -0,0 +1,61 @@ + +# Feathr Abstract backend Data Model Diagram + +This file defines abstract backend data models diagram for feature registry. +[Python Code](./models.py) + +```mermaid +classDiagram + Project "1" --> "n" FeatureName : contains + Project "1" --> "n" Anchor : contains + FeatureName "1" --> "n" Feature : contains + Anchor "1" --> "n" Feature : contains + Feature <|-- AnchorFeature : extends + Feature <|-- DerivedFeature: extends + Feature --> Transformation + Feature --> Transformation : contains + Source <|-- DataSource: extends + Source <|-- MultiFeatureSource: extends + MultiFeatureSource "1" --> "1..n" FeatureSource: contains + AnchorFeature --> DataSource : contains + DerivedFeature --> MultiFeatureSource: contains + + class Source{ + } + class DataSource{ + } + class FeatureSource{ + +FeatureNameId feature_name_id + } + class MultiFeatureSource{ + +List[FeatureSource] sources + } + class Feature{ + +FeatureId id + +FeatureNameId feature_namme_id + +Source source + +Transformation transformation + } + class AnchorFeature{ + +DataSource source + } + class DerivedFeature{ + +MultiFeatureSource source + } + class FeatureName{ + +FeatureNameId id + +ProjectId project_id + +List[FeatureId] feature_ids + } + class Project{ + +ProjectId id + +List[FeatureNameId] feature_name_ids + +List[AnchorId] anchor_ids + } + class Anchor{ + +AnchorId id + +ProjectId project_id + +DataSource source + +List[FeatureId] anchor_feature_ids + } +``` \ No newline at end of file diff --git a/registry/data-models/models.py b/registry/data-models/models.py new file mode 100644 index 000000000..c4ae31f68 --- /dev/null +++ b/registry/data-models/models.py @@ -0,0 +1,146 @@ +from pydantic import BaseModel +from typing import List + +""" +This file defines abstract backend data models for feature registry. +Backend data models will be used by backend API server to talk to feature registry backend. +Purpose of this is to decouple backend data models from API specific data models. +For each feature registry provider/implementation, they will extend this abstract +data models and backend API. +Diagram of the data models: ./data-model-diagram.md +""" + + +class FeatureId(BaseModel): + """ + Id for Feature, it's unique ID represents Feature. + Id can be a simple string, int or complex key. + """ + id: str # id of a feature + + +class FeatureNameId(BaseModel): + """ + Id for FeatureName, it's unique ID represents FeatureName. + Id can be a simple string, int or complex key. + """ + id: str # id of a FeatureName + + +class AnchorId(BaseModel): + """ + Id for Anchor, it's unique ID represents Anchor. + Id can be a simple string, int or complex key. + """ + id: str # id of a anchor + + +class ProjectId(BaseModel): + """ + Id for Project, it's unique ID represents Project. + Id can be a simple string, int or complex key. + """ + id: str # id of a project + + +class Source(BaseModel): + """ + Source of the feature. + It defines where the feature is extracted or derived from. + """ + pass + + +class DataSource(Source): + """ + Data source of the feature. + It defines the raw data source the feature is extracted from. + """ + pass + + +class FeatureSource(BaseModel): + """ + Represents a feature source for a derived feature. That is, it is a source 'FeatureName' which is used for + creating other derived features. + """ + input_feature_name_id: FeatureNameId # Input feature name Key + + +class MultiFeatureSource(Source): + """ + Feature sources of the feature. + It defines one to many features where the feature is derived from. + """ + sources: List[FeatureSource] # All source features which the feature is derived from + pass + + +class Transformation(BaseModel): + """ + The transformation of a Feature. + A transformation function represents the transformation logic to produce feature value from the source of FeatureAnchor + """ + pass + + +class Feature(BaseModel): + """ + Actual implementation of FeatureName. + An implementation defines where a feature is extracted from (Source) and how it is computed (Transformation). + The Source of a feature can be raw data sources and/or other features. + """ + id: FeatureId # Unique ID for Feature + feature_name_id: FeatureNameId # Id of the feature name that the feature belongs to + source: Source # Source can be either data source or feature source + transformation: Transformation # transformation logic to produce feature value + + +class AnchorFeature(Feature): + """ + Feature implementation of FeatureName which anchored to a data source. + """ + source: DataSource # Raw data source where the feature is extracted from + + +class DerivedFeature(Feature): + """ + Feature implementation that is derived from other FeatureNames. + """ + source: MultiFeatureSource # Source features where the feature is derived from + + +class FeatureName(BaseModel): + """ + Named Feature Interface that can be backed by multiple Feature implementations across + different environments accessing different sources (data lake access for batch training, + KV store access for online serving). Each FeatureName is defined by feature producer. + Feature consumers reference a feature by that name to access that feature data, + agnostic of runtime environment. Each FeatureName also encloses attributes that does not + change across implementations. + """ + id: FeatureNameId # unique ID for FeatureName, used to extract data for current FeatureName + project_id: ProjectId # ID of the project the FeatureName belongs to + feature_ids: List[FeatureId] # List of ids of feature that the FeatureName has + + +class Project(BaseModel): + """ + Group of FeatureNames. It can be a project the team is working on, + or a namespace which related FeatureNames have. + """ + id: ProjectId # Unique ID of the project. + feature_name_ids: List[FeatureNameId] # List of feature name ids that the project has + anchor_ids: List[AnchorId] # List of Anchor ids that the project has + + +class Anchor(BaseModel): + """ + Group of AnchorFeatures which anchored on same DataSource. + This is mainly used by feature producer gather information about DataSource + and FeatureImplementations associated with the DataSource. + """ + id: AnchorId # Unique ID for Anchor + project_id: ProjectId # ID of Project that the anchor belongs to + source: DataSource # data source of the Anchor + anchor_feature_ids: List[FeatureId] # List of anchor features that the anchor has From d187ae277113ec50b6f5461f3eff1f9eed48a198 Mon Sep 17 00:00:00 2001 From: Blair Chen Date: Fri, 30 Sep 2022 12:50:02 +0800 Subject: [PATCH 02/68] Revert "Enhance purview registry error messages (#709)" (#720) This reverts commit 059f2b4b6311632b15adf64b1a38f493981b155e. --- ...d-and-push-feathr-registry-docker-image.md | 6 +-- registry/purview-registry/main.py | 47 +------------------ .../registry/purview_registry.py | 34 ++++++-------- 3 files changed, 17 insertions(+), 70 deletions(-) diff --git a/docs/dev_guide/build-and-push-feathr-registry-docker-image.md b/docs/dev_guide/build-and-push-feathr-registry-docker-image.md index 04b1fe487..873c6a141 100644 --- a/docs/dev_guide/build-and-push-feathr-registry-docker-image.md +++ b/docs/dev_guide/build-and-push-feathr-registry-docker-image.md @@ -76,8 +76,4 @@ docker push feathrfeaturestore/feathr-registry ## Published Feathr Registry Image -The published feathr feature registry is located in [DockerHub here](https://hub.docker.com/r/feathrfeaturestore/feathr-registry). - -## Include the detailed track back info in registry api HTTP error response - -Set environment REGISTRY_DEBUGGING to any non empty string will enable the detailed track back info in registry api http response. This variable is helpful for python client debugging and should only be used for debugging purposes. +The published feathr feature registry is located in [DockerHub here](https://hub.docker.com/r/feathrfeaturestore/feathr-registry). \ No newline at end of file diff --git a/registry/purview-registry/main.py b/registry/purview-registry/main.py index 92aa8dc49..5d38adf74 100644 --- a/registry/purview-registry/main.py +++ b/registry/purview-registry/main.py @@ -1,12 +1,11 @@ import os -import traceback from re import sub from typing import Optional from uuid import UUID from fastapi import APIRouter, FastAPI, HTTPException -from fastapi.responses import JSONResponse from starlette.middleware.cors import CORSMiddleware -from registry.purview_registry import PurviewRegistry, ConflictError +from registry import * +from registry.purview_registry import PurviewRegistry from registry.models import AnchorDef, AnchorFeatureDef, DerivedFeatureDef, EntityType, ProjectDef, SourceDef, to_snake rp = "/v1" @@ -44,48 +43,6 @@ def to_camel(s): allow_headers=["*"], ) -def exc_to_content(e: Exception) -> dict: - content={"message": str(e)} - if os.environ.get("REGISTRY_DEBUGGING"): - content["traceback"] = "".join(traceback.TracebackException.from_exception(e).format()) - return content - -@app.exception_handler(ConflictError) -async def conflict_error_handler(_, exc: ConflictError): - return JSONResponse( - status_code=409, - content=exc_to_content(exc), - ) - - -@app.exception_handler(ValueError) -async def value_error_handler(_, exc: ValueError): - return JSONResponse( - status_code=400, - content=exc_to_content(exc), - ) - -@app.exception_handler(TypeError) -async def type_error_handler(_, exc: ValueError): - return JSONResponse( - status_code=400, - content=exc_to_content(exc), - ) - - -@app.exception_handler(KeyError) -async def key_error_handler(_, exc: KeyError): - return JSONResponse( - status_code=404, - content=exc_to_content(exc), - ) - -@app.exception_handler(IndexError) -async def index_error_handler(_, exc: IndexError): - return JSONResponse( - status_code=404, - content=exc_to_content(exc), - ) @router.get("/projects",tags=["Project"]) def get_projects() -> list[str]: diff --git a/registry/purview-registry/registry/purview_registry.py b/registry/purview-registry/registry/purview_registry.py index 06d7bd8d1..9f5f47560 100644 --- a/registry/purview-registry/registry/purview_registry.py +++ b/registry/purview-registry/registry/purview_registry.py @@ -1,6 +1,8 @@ import copy +from http.client import CONFLICT, HTTPException import itertools -from typing import Optional, Tuple, Union +from typing import Any, Optional, Tuple, Union +from urllib.error import HTTPError from uuid import UUID from azure.identity import DefaultAzureCredential @@ -9,7 +11,7 @@ from pyapacheatlas.core import (AtlasEntity, AtlasProcess, PurviewClient) from pyapacheatlas.core.typedef import (AtlasAttributeDef,Cardinality,EntityTypeDef) -from pyapacheatlas.core.util import GuidTracker, AtlasException +from pyapacheatlas.core.util import GuidTracker from pyhocon import ConfigFactory from registry.interface import Registry @@ -21,10 +23,6 @@ TYPEDEF_ARRAY_ANCHOR=f"array" TYPEDEF_ARRAY_DERIVED_FEATURE=f"array" TYPEDEF_ARRAY_ANCHOR_FEATURE=f"array" - -class ConflictError(Exception): - pass - class PurviewRegistry(Registry): def __init__(self,azure_purview_name: str, registry_delimiter: str = "__", credential=None,register_types = True): self.registry_delimiter = registry_delimiter @@ -570,22 +568,18 @@ def _register_feathr_feature_types(self): def _upload_entity_batch(self, entity_batch:list[AtlasEntity]): # we only support entity creation, update is not supported. # setting lastModifiedTS ==0 will ensure this, if another entity with ts>=1 exist - # upload function will fail with 412 Precondition fail. + # upload funtion will fail with 412 Precondition fail. for entity in entity_batch: entity.lastModifiedTS="0" - try: - results = self.purview_client.upload_entities( - batch=entity) - if results: - dict = {x.guid: x for x in entity_batch} - for k, v in results['guidAssignments'].items(): - dict[k].guid = v - else: - raise RuntimeError("Feature registration failed.", results) - except AtlasException as e: - if "PreConditionCheckFailed" in e.args[0]: - raise ConflictError(f"Entity {entity.guid}, {entity.typeName} -- {entity.qualifiedName} already exists in Purview. Please use a new name.") - + results = self.purview_client.upload_entities( + batch=entity) + if results: + dict = {x.guid: x for x in entity_batch} + for k, v in results['guidAssignments'].items(): + dict[k].guid = v + else: + raise RuntimeError("Feature registration failed.", results) + def _generate_fully_qualified_name(self, segments): return self.registry_delimiter.join(segments) From 6d1e7a698334ced9612b2f9ae645968fefbc1808 Mon Sep 17 00:00:00 2001 From: Jinghui Mo Date: Wed, 5 Oct 2022 16:15:17 -0400 Subject: [PATCH 03/68] Improve Avro GenericRecord and SpecificRecord based row-level extractor performance (#723) * 1. In SparkRowExtractor.scala, add new extractor method which can be extended to do batch preprocess source dataframe into RDD[IndexRecord]. 2. In FeatureTransformation.scala, add logic to extract features from RDD[IndexedRecord]. 3. Improve some error messages. --- .../feathr/common/AnchorExtractor.scala | 8 +- .../feathr/common/CanConvertToAvroRDD.scala | 20 ++ .../feathr/common/SparkRowExtractor.scala | 6 +- .../SimpleConfigurableAnchorExtractor.scala | 7 +- .../keyExtractor/MVELSourceKeyExtractor.scala | 7 +- .../offline/config/FeathrConfigLoader.scala | 2 +- .../StreamingFeatureGenerator.scala | 4 +- .../offline/job/FeatureTransformation.scala | 259 +++++++++++++++++- .../DataFrameBasedRowEvaluator.scala | 27 +- .../DefaultValueSubstituter.scala | 2 +- .../transformation/FDSConversionUtils.scala | 2 +- .../util/FeatureValueTypeValidator.scala | 24 +- .../offline/util/FeaturizedDatasetUtils.scala | 2 +- .../util/TestFeatureValueTypeValidator.scala | 4 +- 14 files changed, 318 insertions(+), 56 deletions(-) create mode 100644 src/main/scala/com/linkedin/feathr/common/CanConvertToAvroRDD.scala diff --git a/src/main/scala/com/linkedin/feathr/common/AnchorExtractor.scala b/src/main/scala/com/linkedin/feathr/common/AnchorExtractor.scala index 2e38e4d04..185c9d2d6 100644 --- a/src/main/scala/com/linkedin/feathr/common/AnchorExtractor.scala +++ b/src/main/scala/com/linkedin/feathr/common/AnchorExtractor.scala @@ -1,7 +1,5 @@ package com.linkedin.feathr.common -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema - /** * Provides feature values based on some "raw" data element * @@ -39,12 +37,14 @@ trait AnchorExtractor[T] extends AnchorExtractorBase[T] with SparkRowExtractor { * @param datum input row * @return list of feature keys */ - def getKeyFromRow(datum: GenericRowWithSchema): Seq[String] = getKey(datum.asInstanceOf[T]) + def getKeyFromRow(datum: Any): Seq[String] = getKey(datum.asInstanceOf[T]) /** * Get the feature value from the row * @param datum input row * @return A map of feature name to feature value */ - def getFeaturesFromRow(datum: GenericRowWithSchema): Map[String, FeatureValue] = getFeatures(datum.asInstanceOf[T]) + def getFeaturesFromRow(datum: Any): Map[String, FeatureValue] = getFeatures(datum.asInstanceOf[T]) + + override def toString: String = getClass.getSimpleName } diff --git a/src/main/scala/com/linkedin/feathr/common/CanConvertToAvroRDD.scala b/src/main/scala/com/linkedin/feathr/common/CanConvertToAvroRDD.scala new file mode 100644 index 000000000..7051a308c --- /dev/null +++ b/src/main/scala/com/linkedin/feathr/common/CanConvertToAvroRDD.scala @@ -0,0 +1,20 @@ +package com.linkedin.feathr.common + +import org.apache.avro.generic.IndexedRecord +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.DataFrame + +/** + * If an AnchorExtractor only works on a Avro record, it should extends + * this trait, and use convertToAvroRdd to do a one-time batch conversion of DataFrame to RDD of their choice. + * convertToAvroRdd will be called by Feathr engine before calling getKeyFromRow() and getFeaturesFromRow() in AnchorExtractor. + */ +trait CanConvertToAvroRDD { + + /** + * One time batch converting the input data source into a RDD[IndexedRecord] for feature extraction later + * @param df input data source + * @return batch preprocessed dataframe, as RDD[IndexedRecord] + */ + def convertToAvroRdd(df: DataFrame) : RDD[IndexedRecord] +} diff --git a/src/main/scala/com/linkedin/feathr/common/SparkRowExtractor.scala b/src/main/scala/com/linkedin/feathr/common/SparkRowExtractor.scala index 04e715e8c..ad088ac0a 100644 --- a/src/main/scala/com/linkedin/feathr/common/SparkRowExtractor.scala +++ b/src/main/scala/com/linkedin/feathr/common/SparkRowExtractor.scala @@ -1,7 +1,5 @@ package com.linkedin.feathr.common -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema - /** * An extractor trait that provides APIs to transform a Spark GenericRowWithSchema into feature values */ @@ -12,12 +10,12 @@ trait SparkRowExtractor { * @param datum input row * @return list of feature keys */ - def getKeyFromRow(datum: GenericRowWithSchema): Seq[String] + def getKeyFromRow(datum: Any): Seq[String] /** * Get the feature value from the row * @param datum input row * @return A map of feature name to feature value */ - def getFeaturesFromRow(datum: GenericRowWithSchema): Map[String, FeatureValue] + def getFeaturesFromRow(datum: Any): Map[String, FeatureValue] } \ No newline at end of file diff --git a/src/main/scala/com/linkedin/feathr/offline/anchored/anchorExtractor/SimpleConfigurableAnchorExtractor.scala b/src/main/scala/com/linkedin/feathr/offline/anchored/anchorExtractor/SimpleConfigurableAnchorExtractor.scala index 59f5bfbe7..edb2e2c06 100644 --- a/src/main/scala/com/linkedin/feathr/offline/anchored/anchorExtractor/SimpleConfigurableAnchorExtractor.scala +++ b/src/main/scala/com/linkedin/feathr/offline/anchored/anchorExtractor/SimpleConfigurableAnchorExtractor.scala @@ -10,7 +10,6 @@ import com.linkedin.feathr.offline.mvel.plugins.FeathrExpressionExecutionContext import com.linkedin.feathr.offline.mvel.{MvelContext, MvelUtils} import com.linkedin.feathr.offline.util.FeatureValueTypeValidator import org.apache.log4j.Logger -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.types._ import org.mvel2.MVEL @@ -66,7 +65,7 @@ private[offline] class SimpleConfigurableAnchorExtractor( @JsonProperty("key") k * @param datum input row * @return list of feature keys */ - override def getKeyFromRow(datum: GenericRowWithSchema): Seq[String] = { + override def getKeyFromRow(datum: Any): Seq[String] = { getKey(datum.asInstanceOf[Any]) } @@ -107,7 +106,7 @@ private[offline] class SimpleConfigurableAnchorExtractor( @JsonProperty("key") k * @param row input row * @return A map of feature name to feature value */ - override def getFeaturesFromRow(row: GenericRowWithSchema) = { + override def getFeaturesFromRow(row: Any) = { getFeatures(row.asInstanceOf[Any]) } @@ -147,7 +146,7 @@ private[offline] class SimpleConfigurableAnchorExtractor( @JsonProperty("key") k featureTypeConfigs(featureRefStr) } val featureValue = offline.FeatureValue.fromTypeConfig(value, featureTypeConfig) - FeatureValueTypeValidator.validate(featureValue, featureTypeConfigs(featureRefStr)) + FeatureValueTypeValidator.validate(featureRefStr, featureValue, featureTypeConfigs(featureRefStr) ) (featureRefStr, featureValue) } diff --git a/src/main/scala/com/linkedin/feathr/offline/anchored/keyExtractor/MVELSourceKeyExtractor.scala b/src/main/scala/com/linkedin/feathr/offline/anchored/keyExtractor/MVELSourceKeyExtractor.scala index 209ac89e1..bf5108e8b 100644 --- a/src/main/scala/com/linkedin/feathr/offline/anchored/keyExtractor/MVELSourceKeyExtractor.scala +++ b/src/main/scala/com/linkedin/feathr/offline/anchored/keyExtractor/MVELSourceKeyExtractor.scala @@ -43,7 +43,7 @@ private[feathr] class MVELSourceKeyExtractor(val anchorExtractorV1: AnchorExtrac .toDF() } - def getKey(datum: GenericRowWithSchema): Seq[String] = { + def getKey(datum: Any): Seq[String] = { anchorExtractorV1.getKeyFromRow(datum) } @@ -55,7 +55,7 @@ private[feathr] class MVELSourceKeyExtractor(val anchorExtractorV1: AnchorExtrac */ override def getKeyColumnNames(datum: Option[Any]): Seq[String] = { if (datum.isDefined) { - val size = getKey(datum.get.asInstanceOf[GenericRowWithSchema]).size + val size = getKey(datum.get).size (1 to size).map(JOIN_KEY_PREFIX + _) } else { // return empty join key to signal empty dataset @@ -86,5 +86,6 @@ private[feathr] class MVELSourceKeyExtractor(val anchorExtractorV1: AnchorExtrac // this helps to reduce the number of joins // to the observation data // The default toString does not work, because toString of each object have different values - override def toString: String = getClass.getSimpleName + " with keyExprs:" + keyExprs.mkString(" key:") + override def toString: String = getClass.getSimpleName + " with keyExprs:" + keyExprs.mkString(" key:") + + "anchorExtractor:" + anchorExtractorV1.toString } diff --git a/src/main/scala/com/linkedin/feathr/offline/config/FeathrConfigLoader.scala b/src/main/scala/com/linkedin/feathr/offline/config/FeathrConfigLoader.scala index 1faf0d814..e2ec6e588 100644 --- a/src/main/scala/com/linkedin/feathr/offline/config/FeathrConfigLoader.scala +++ b/src/main/scala/com/linkedin/feathr/offline/config/FeathrConfigLoader.scala @@ -327,7 +327,7 @@ private[offline] class AnchorLoader extends JsonDeserializer[FeatureAnchor] { case Some(tType) => offline.FeatureValue.fromTypeConfig(rawValue, tType) case None => offline.FeatureValue(rawValue, featureType, key) } - FeatureValueTypeValidator.validate(featureValue, featureTypeConfig) + FeatureValueTypeValidator.validate(featureValue, featureTypeConfig, key) (key, featureValue) } .toMap diff --git a/src/main/scala/com/linkedin/feathr/offline/generation/StreamingFeatureGenerator.scala b/src/main/scala/com/linkedin/feathr/offline/generation/StreamingFeatureGenerator.scala index 99436b93c..126128323 100644 --- a/src/main/scala/com/linkedin/feathr/offline/generation/StreamingFeatureGenerator.scala +++ b/src/main/scala/com/linkedin/feathr/offline/generation/StreamingFeatureGenerator.scala @@ -6,7 +6,7 @@ import com.linkedin.feathr.common.JoiningFeatureParams import com.linkedin.feathr.offline.config.location.KafkaEndpoint import com.linkedin.feathr.offline.generation.outputProcessor.PushToRedisOutputProcessor.TABLE_PARAM_CONFIG_NAME import com.linkedin.feathr.offline.generation.outputProcessor.RedisOutputUtils -import com.linkedin.feathr.offline.job.FeatureTransformation.getFeatureJoinKey +import com.linkedin.feathr.offline.job.FeatureTransformation.getFeatureKeyColumnNames import com.linkedin.feathr.offline.job.{FeatureGenSpec, FeatureTransformation} import com.linkedin.feathr.offline.logical.FeatureGroups import com.linkedin.feathr.offline.source.accessor.DataPathHandler @@ -111,7 +111,7 @@ class StreamingFeatureGenerator(dataPathHandlers: List[DataPathHandler]) { // Apply feature transformation val transformedResult = DataFrameBasedSqlEvaluator.transform(anchor.featureAnchor.extractor.asInstanceOf[SimpleAnchorExtractorSpark], withKeyColumnDF, featureNamePrefixPairs, anchor.featureAnchor.featureTypeConfigs) - val outputJoinKeyColumnNames = getFeatureJoinKey(keyExtractor, withKeyColumnDF) + val outputJoinKeyColumnNames = getFeatureKeyColumnNames(keyExtractor, withKeyColumnDF) val selectedColumns = outputJoinKeyColumnNames ++ anchor.selectedFeatures.filter(keyTaggedFeatures.map(_.featureName).contains(_)) val cleanedDF = transformedResult.df.select(selectedColumns.head, selectedColumns.tail:_*) val keyColumnNames = FeatureTransformation.getStandardizedKeyNames(outputJoinKeyColumnNames.size) diff --git a/src/main/scala/com/linkedin/feathr/offline/job/FeatureTransformation.scala b/src/main/scala/com/linkedin/feathr/offline/job/FeatureTransformation.scala index 94de8e645..7b106572b 100644 --- a/src/main/scala/com/linkedin/feathr/offline/job/FeatureTransformation.scala +++ b/src/main/scala/com/linkedin/feathr/offline/job/FeatureTransformation.scala @@ -1,7 +1,9 @@ package com.linkedin.feathr.offline.job -import com.linkedin.feathr.common._ import com.linkedin.feathr.common.exception.{ErrorLabel, FeathrException, FeathrFeatureTransformationException} +import com.linkedin.feathr.common.tensor.TensorData +import com.linkedin.feathr.common.types.FeatureType +import com.linkedin.feathr.common.{AnchorExtractorBase, _} import com.linkedin.feathr.offline.anchored.anchorExtractor.{SQLConfigurableAnchorExtractor, SimpleConfigurableAnchorExtractor, TimeWindowConfigurableAnchorExtractor} import com.linkedin.feathr.offline.anchored.feature.{FeatureAnchor, FeatureAnchorWithSource} import com.linkedin.feathr.offline.anchored.keyExtractor.MVELSourceKeyExtractor @@ -22,6 +24,7 @@ import com.linkedin.feathr.offline.{FeatureDataFrame, JoinKeys} import com.linkedin.feathr.sparkcommon.{SimpleAnchorExtractorSpark, SourceKeyExtractor} import com.linkedin.feathr.swj.aggregate.AggregationType import com.linkedin.feathr.{common, offline} +import org.apache.avro.generic.IndexedRecord import org.apache.log4j.Logger import org.apache.spark.rdd.RDD import org.apache.spark.sql.functions._ @@ -41,6 +44,16 @@ import scala.concurrent.{Await, ExecutionContext, Future} */ private[offline] case class AnchorFeatureGroups(anchorsWithSameSource: Seq[FeatureAnchorWithSource], requestedFeatures: Seq[String]) +/** + * Context info needed in feature transformation + * @param featureAnchorWithSource feature annchor with its source + * @param featureNamePrefixPairs map of feature name to its prefix + * @param transformer transformer of anchor + */ +private[offline] case class TransformInfo(featureAnchorWithSource: FeatureAnchorWithSource, + featureNamePrefixPairs: Seq[(FeatureName, FeatureName)], + transformer: AnchorExtractorBase[IndexedRecord]) + /** * Represent the transformed result of an anchor extractor after evaluating its features * @param featureNameAndPrefixPairs pairs of feature name and feature name prefix @@ -75,7 +88,27 @@ private[offline] object FeatureTransformation { // feature name, column prefix type FeatureNameAndColumnPrefix = (String, String) - def getFeatureJoinKey(sourceKeyExtractor: SourceKeyExtractor, withKeyColumnDF: DataFrame, featureExtractor: Option[AnyRef] = None): Seq[String] = { + /** + * Extract feature key column names from the input feature RDD using the sourceKeyExtractor. + * @param sourceKeyExtractor key extractor that knows what are the key column in a feature RDD. + * @param withKeyColumnRDD RDD that contains the key columns. + * @return feature key column names + */ + def getFeatureKeyColumnNamesRdd(sourceKeyExtractor: SourceKeyExtractor, withKeyColumnRDD: RDD[_]): Seq[String] = { + if (withKeyColumnRDD.isEmpty) { + sourceKeyExtractor.getKeyColumnNames(None) + } else { + sourceKeyExtractor.getKeyColumnNames(Some(withKeyColumnRDD.first())) + } + } + + /** + * Extract feature key column names from the input feature DataFrame using the sourceKeyExtractor. + * @param sourceKeyExtractor key extractor that knows what are the key column in a feature RDD. + * @param withKeyColumnDF DataFrame that contains the key columns. + * @return feature key column names + */ + def getFeatureKeyColumnNames(sourceKeyExtractor: SourceKeyExtractor, withKeyColumnDF: DataFrame): Seq[String] = { if (withKeyColumnDF.head(1).isEmpty) { sourceKeyExtractor.getKeyColumnNames(None) } else { @@ -306,7 +339,8 @@ private[offline] object FeatureTransformation { } val withKeyColumnDF = keyExtractor.appendKeyColumns(sourceDF) - val outputJoinKeyColumnNames = getFeatureJoinKey(keyExtractor, withKeyColumnDF, Some(anchorFeatureGroup.anchorsWithSameSource.head.featureAnchor.extractor)) + + val outputJoinKeyColumnNames = getFeatureKeyColumnNames(keyExtractor, withKeyColumnDF) val filteredFactData = applyBloomFilter((keyExtractor, withKeyColumnDF), bloomFilter) // 1. apply all transformations on the dataframe in sequential order @@ -457,10 +491,21 @@ private[offline] object FeatureTransformation { val keyExtractor = anchorsWithSameSource.head._1.featureAnchor.sourceKeyExtractor val featureAnchorWithSource = anchorsWithSameSource.keys.toSeq val selectedFeatures = anchorsWithSameSource.flatMap(_._2.featureNames).toSeq - - val sourceDF = featureGroupingFactors.source - val transformedResults: Seq[KeyedTransformedResult] = transformMultiAnchorsOnSingleDataFrame(sourceDF, + val isAvroRddBasedExtractor = featureAnchorWithSource + .map(_.featureAnchor.extractor) + .filter(extractor => extractor.isInstanceOf[CanConvertToAvroRDD] + ).nonEmpty + val transformedResults: Seq[KeyedTransformedResult] = if (isAvroRddBasedExtractor) { + // If there are features are defined using AVRO record based extractor, run RDD based feature transformation + val sourceAccessor = featureGroupingFactors.source + val sourceRdd = sourceAccessor.asInstanceOf[NonTimeBasedDataSourceAccessor].get() + val featureTypeConfigs = featureAnchorWithSource.flatMap(featureAnchor => featureAnchor.featureAnchor.featureTypeConfigs).toMap + Seq(transformFeaturesOnAvroRecord(sourceRdd, keyExtractor, featureAnchorWithSource, bloomFilter, selectedFeatures, featureTypeConfigs)) + } else { + val sourceDF = featureGroupingFactors.source + transformFeaturesOnDataFrameRow(sourceDF, keyExtractor, featureAnchorWithSource, bloomFilter, selectedFeatures, incrementalAggContext, mvelContext) + } val res = transformedResults .map { transformedResultWithKey => @@ -673,6 +718,204 @@ private[offline] object FeatureTransformation { } } + + /** + * Apply a bloomfilter to a RDD + * + * @param keyExtractor key extractor to extract the key values from the RDD + * @param rdd RDD to filter + * @param bloomFilter bloomfilter used to filter out unwanted row in the RDD based on key columns + * @return filtered RDD + */ + + private def applyBloomFilterRdd(keyExtractor: SourceKeyExtractor, rdd: RDD[IndexedRecord], bloomFilter: Option[BloomFilter]): RDD[IndexedRecord] = { + bloomFilter match { + case None => + // no bloom filter, use data as it + rdd + case Some(filter) => + // get the list of join key columns or expression + keyExtractor match { + case extractor: MVELSourceKeyExtractor => + // get the list of join key columns or expression + val keyColumnsList = if (rdd.isEmpty) { + extractor.getKeyColumnNames(None) + } else { + extractor.getKeyColumnNames(Some(rdd.first)) + } + if (!keyColumnsList.isEmpty) { + val filtered = rdd.filter { record: Any => + val keyVals = extractor.getKey(record) + // if key is not in observation, skip it + if (keyVals != null && keyVals.count(_ == null) == 0) { + filter.mightContainString(SourceUtils.generateFilterKeyString(keyVals)) + } else { + false + } + } + filtered + } else { + // expand feature for seq join does not have right key, so we allow empty here + rdd + } + case _ => throw new FeathrFeatureTransformationException(ErrorLabel.FEATHR_USER_ERROR, "No source key extractor found") + } + } + } + + /** + * Transform features defined in a group of anchors based on same source + * This is for the AVRO record based extractors + * + * @param rdd source that requested features are defined on + * @param keyExtractor key extractor to apply on source rdd + * @param featureAnchorWithSources feature anchors defined on source rdd to be evaluated + * @param bloomFilter bloomfilter to apply on source rdd + * @param requestedFeatureNames requested features + * @param featureTypeConfigs user specified feature types + * @return TransformedResultWithKey The output feature DataFrame conforms to FDS format + */ + private def transformFeaturesOnAvroRecord(df: DataFrame, + keyExtractor: SourceKeyExtractor, + featureAnchorWithSources: Seq[FeatureAnchorWithSource], + bloomFilter: Option[BloomFilter], + requestedFeatureNames: Seq[FeatureName], + featureTypeConfigs: Map[String, FeatureTypeConfig] = Map()): KeyedTransformedResult = { + if (!keyExtractor.isInstanceOf[MVELSourceKeyExtractor]) { + throw new FeathrException(ErrorLabel.FEATHR_ERROR, s"Error processing requested Feature :${requestedFeatureNames}. " + + s"Key extractor ${keyExtractor} must extends MVELSourceKeyExtractor.") + } + val extractor = keyExtractor.asInstanceOf[MVELSourceKeyExtractor] + if (!extractor.anchorExtractorV1.isInstanceOf[CanConvertToAvroRDD]) { + throw new FeathrException(ErrorLabel.FEATHR_ERROR, s"Error processing requested Feature :${requestedFeatureNames}. " + + s"isLowLevelRddExtractor() should return true and convertToAvroRdd should be implemented.") + } + val rdd = extractor.anchorExtractorV1.asInstanceOf[CanConvertToAvroRDD].convertToAvroRdd(df) + val filteredFactData = applyBloomFilterRdd(keyExtractor, rdd, bloomFilter) + + // Build a sequence of 3-tuple of (FeatureAnchorWithSource, featureNamePrefixPairs, AnchorExtractorBase) + val transformInfo = featureAnchorWithSources map { featureAnchorWithSource => + val extractor = featureAnchorWithSource.featureAnchor.extractor + extractor match { + case transformer: AnchorExtractorBase[IndexedRecord] => + // We no longer need prefix for the simplicity of the implementation, instead if there's a feature name + // and source data field clash, we will throw exception and ask user to rename the feature. + val featureNamePrefix = "" + val featureNames = featureAnchorWithSource.selectedFeatures.filter(requestedFeatureNames.contains) + val featureNamePrefixPairs = featureNames.map((_, featureNamePrefix)) + TransformInfo(featureAnchorWithSource, featureNamePrefixPairs, transformer) + + case _ => + throw new FeathrFeatureTransformationException(ErrorLabel.FEATHR_USER_ERROR, s"Unsupported transformer $extractor for features: $requestedFeatureNames") + } + } + + // to avoid name conflict between feature names and the raw data field names + val sourceKeyExtractors = transformInfo.map(_.featureAnchorWithSource.featureAnchor.sourceKeyExtractor) + assert(sourceKeyExtractors.map(_.toString).distinct.size == 1) + + val transformers = transformInfo map (_.transformer) + + /* + * Transform the given RDD by applying extractors to each row to create an RDD[Row] where each Row + * represents keys and feature values + */ + val spark = SparkSession.builder().getOrCreate() + val userProvidedFeatureTypes = transformInfo.flatMap(_.featureAnchorWithSource.featureAnchor.getFeatureTypes.getOrElse(Map.empty[String, FeatureTypes])).toMap + val FeatureTypeInferenceContext(featureTypeAccumulators) = + FeatureTransformation.getTypeInferenceContext(spark, userProvidedFeatureTypes, requestedFeatureNames) + val transformedRdd = filteredFactData map { record => + val (keys, featureValuesWithType) = transformAvroRecord(requestedFeatureNames, sourceKeyExtractors, transformers, record, featureTypeConfigs) + requestedFeatureNames.zip(featureValuesWithType).foreach { + case (featureRef, (_, featureType)) => + if (featureTypeAccumulators(featureRef).isZero && featureType != null) { + // This is lazy evaluated + featureTypeAccumulators(featureRef).add(FeatureTypes.valueOf(featureType.getBasicType.toString)) + } + } + // Create a row by merging a row created from keys and a row created from term-vectors/tensors + Row.merge(Row.fromSeq(keys), Row.fromSeq(featureValuesWithType.map(_._1))) + } + + // Create a DataFrame from the above obtained RDD + val keyNames = getFeatureKeyColumnNamesRdd(sourceKeyExtractors.head, filteredFactData) + val (outputSchema, inferredFeatureTypeConfigs) = { + val allFeatureTypeConfigs = featureAnchorWithSources.flatMap(featureAnchorWithSource => featureAnchorWithSource.featureAnchor.featureTypeConfigs).toMap + val inferredFeatureTypes = inferFeatureTypes(featureTypeAccumulators, transformedRdd, requestedFeatureNames) + val inferredFeatureTypeConfigs = inferredFeatureTypes.map(x => x._1 -> new FeatureTypeConfig(x._2)) + val mergedFeatureTypeConfig = inferredFeatureTypeConfigs ++ allFeatureTypeConfigs + val colPrefix = "" + val featureTensorTypeInfo = getFDSSchemaFields(requestedFeatureNames, mergedFeatureTypeConfig, colPrefix) + val structFields = keyNames.foldRight(List.empty[StructField]) { + case (colName, acc) => + StructField(colName, StringType) :: acc + } + val outputSchema = StructType(StructType(structFields ++ featureTensorTypeInfo)) + (outputSchema, mergedFeatureTypeConfig) + } + val transformedDF = spark.createDataFrame(transformedRdd, outputSchema) + + val featureFormat = FeatureColumnFormat.FDS_TENSOR + val featureColumnFormats = requestedFeatureNames.map(name => name -> featureFormat).toMap + val transformedInfo = TransformedResult(transformInfo.flatMap(_.featureNamePrefixPairs), transformedDF, featureColumnFormats, inferredFeatureTypeConfigs) + KeyedTransformedResult(keyNames, transformedInfo) + } + + /** + * Apply a keyExtractor and feature transformer on a Record to extractor feature values. + * @param requestedFeatureNames requested feature names in the output. Extractors may produce more features than requested. + * @param sourceKeyExtractors extractor to extract the key from the record + * @param transformers transform to produce the feature value from the record + * @param record avro record to work on + * @param featureTypeConfigs user defined feature types + * @return tuple of (feature join key, sequence of (feature value, feature type) in the order of requestedFeatureNames) + */ + private def transformAvroRecord( + requestedFeatureNames: Seq[FeatureName], + sourceKeyExtractors: Seq[SourceKeyExtractor], + transformers: Seq[AnchorExtractorBase[IndexedRecord]], + record: IndexedRecord, + featureTypeConfigs: Map[String, FeatureTypeConfig] = Map()): (Seq[String], Seq[(Any, FeatureType)]) = { + val keys = sourceKeyExtractors.head match { + case mvelSourceKeyExtractor: MVELSourceKeyExtractor => mvelSourceKeyExtractor.getKey(record) + case _ => throw new FeathrFeatureTransformationException(ErrorLabel.FEATHR_USER_ERROR, s"${sourceKeyExtractors.head} is not a valid extractor on RDD") + } + + /* + * For the given row, apply all extractors to extract feature values. If requested as tensors, each feature value + * contains a tensor else a term-vector. + */ + val features = transformers map { + case extractor: AnchorExtractor[IndexedRecord] => + val features = extractor.getFeatures(record) + FeatureValueTypeValidator.validate(features, featureTypeConfigs) + features + case extractor => + throw new FeathrFeatureTransformationException( + ErrorLabel.FEATHR_USER_ERROR, + s"Invalid extractor $extractor for features:" + + s"$requestedFeatureNames requested as tensors") + } reduce (_ ++ _) + if (logger.isTraceEnabled) { + logger.trace(s"Extracted features: $features") + } + + /* + * Retain feature values for only the requested features, and represent each feature value as + * a tensor, as specified. + */ + val featureValuesWithType = requestedFeatureNames map { name => + features.get(name) map { + case featureValue => + val tensorData: TensorData = featureValue.getAsTensorData() + val featureType: FeatureType = featureValue.getFeatureType() + val row = FeaturizedDatasetUtils.tensorToFDSDataFrameRow(tensorData) + (row, featureType) + } getOrElse ((null, null)) // return null if no feature value present + } + (keys, featureValuesWithType) + } + /** * Helper function to be used by groupFeatures. Given a collection of feature anchors which also contains information about grouping * criteria and extractor type per feature anchor, returns a map of FeatureGroupingCriteria to @@ -851,7 +1094,7 @@ private[offline] object FeatureTransformation { * others use direct aggregation * */ - private def transformMultiAnchorsOnSingleDataFrame( + private def transformFeaturesOnDataFrameRow( source: DataSourceAccessor, keyExtractor: SourceKeyExtractor, anchorsWithSameSource: Seq[FeatureAnchorWithSource], @@ -878,7 +1121,7 @@ private[offline] object FeatureTransformation { val incrAggCtx = incrementalAggContext.get val preAggDFs = incrAggCtx.previousSnapshotMap.collect { case (featureName, df) if requestedFeatures.exists(df.columns.contains) => df }.toSeq.distinct // join each previous aggregation dataframe sequentially - val groupKeys = getFeatureJoinKey(keyExtractor, preAggDFs.head) + val groupKeys = getFeatureKeyColumnNames(keyExtractor, preAggDFs.head) val keyColumnNames = getStandardizedKeyNames(groupKeys.size) val firstPreAgg = preAggDFs.head val joinedPreAggDFs = preAggDFs diff --git a/src/main/scala/com/linkedin/feathr/offline/transformation/DataFrameBasedRowEvaluator.scala b/src/main/scala/com/linkedin/feathr/offline/transformation/DataFrameBasedRowEvaluator.scala index d242372bf..cc6cba1c7 100644 --- a/src/main/scala/com/linkedin/feathr/offline/transformation/DataFrameBasedRowEvaluator.scala +++ b/src/main/scala/com/linkedin/feathr/offline/transformation/DataFrameBasedRowEvaluator.scala @@ -74,19 +74,20 @@ private[offline] object DataFrameBasedRowEvaluator { val featureTypes = featureTypeConfigs.mapValues(_.getFeatureType) val FeatureTypeInferenceContext(featureTypeAccumulators) = FeatureTransformation.getTypeInferenceContext(spark, featureTypes, featureRefStrs) + val transformedRdd = inputDF.rdd.map(row => { - // in some cases, the input dataframe row here only have Row and does not have schema attached, - // while MVEL only works with GenericRowWithSchema, create it manually - val rowWithSchema = if (row.isInstanceOf[GenericRowWithSchema]) { - row.asInstanceOf[GenericRowWithSchema] - } else { - new GenericRowWithSchema(row.toSeq.toArray, inputSchema) - } - if (rowExtractor.isInstanceOf[SimpleConfigurableAnchorExtractor]) { - rowExtractor.asInstanceOf[SimpleConfigurableAnchorExtractor].mvelContext = mvelContext - } - val result = rowExtractor.getFeaturesFromRow(rowWithSchema) - val featureValues = featureRefStrs map { + // in some cases, the input dataframe row here only have Row and does not have schema attached, + // while MVEL only works with GenericRowWithSchema, create it manually + val rowWithSchema = if (row.isInstanceOf[GenericRowWithSchema]) { + row.asInstanceOf[GenericRowWithSchema] + } else { + new GenericRowWithSchema(row.toSeq.toArray, inputSchema) + } + if (rowExtractor.isInstanceOf[SimpleConfigurableAnchorExtractor]) { + rowExtractor.asInstanceOf[SimpleConfigurableAnchorExtractor].mvelContext = mvelContext + } + val result = rowExtractor.getFeaturesFromRow(rowWithSchema) + val featureValues = featureRefStrs map { featureRef => if (result.contains(featureRef)) { val featureValue = result(featureRef) @@ -95,7 +96,7 @@ private[offline] object DataFrameBasedRowEvaluator { featureTypeAccumulators(featureRef).add(FeatureTypes.valueOf(rowFeatureType.toString)) } val tensorData: TensorData = featureValue.getAsTensorData() - FeaturizedDatasetUtils.tensorToDataFrameRow(tensorData) + FeaturizedDatasetUtils.tensorToFDSDataFrameRow(tensorData) } else null } Row.merge(row, Row.fromSeq(featureValues)) diff --git a/src/main/scala/com/linkedin/feathr/offline/transformation/DefaultValueSubstituter.scala b/src/main/scala/com/linkedin/feathr/offline/transformation/DefaultValueSubstituter.scala index 366967cc2..1b67d9558 100644 --- a/src/main/scala/com/linkedin/feathr/offline/transformation/DefaultValueSubstituter.scala +++ b/src/main/scala/com/linkedin/feathr/offline/transformation/DefaultValueSubstituter.scala @@ -112,7 +112,7 @@ private[offline] object DataFrameDefaultValueSubstituter extends DataFrameDefaul // For tensor default, since we don't have type, so we need to use expr to construct the default column val schema = field.dataType val tensorData = defaultFeatureValue.getAsTensorData - val ts = FeaturizedDatasetUtils.tensorToDataFrameRow(tensorData) + val ts = FeaturizedDatasetUtils.tensorToFDSDataFrameRow(tensorData) val fdsTensorDefaultUDF = getFDSTensorDefaultUDF(schema, ts) ss.udf.register("tz_udf", fdsTensorDefaultUDF) expr(s"tz_udf($featureColumnName)") diff --git a/src/main/scala/com/linkedin/feathr/offline/transformation/FDSConversionUtils.scala b/src/main/scala/com/linkedin/feathr/offline/transformation/FDSConversionUtils.scala index 824f48fe3..96a10a67c 100644 --- a/src/main/scala/com/linkedin/feathr/offline/transformation/FDSConversionUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/transformation/FDSConversionUtils.scala @@ -37,7 +37,7 @@ private[offline] object FDSConversionUtils { // convert the "raw" input data into a FDS column a specific dataType rawFeatureValue match { case tensorData: TensorData => - FeaturizedDatasetUtils.tensorToDataFrameRow(tensorData, Some(targetDataType)) + FeaturizedDatasetUtils.tensorToFDSDataFrameRow(tensorData, Some(targetDataType)) case _ => targetDataType match { // Scalar tensor diff --git a/src/main/scala/com/linkedin/feathr/offline/util/FeatureValueTypeValidator.scala b/src/main/scala/com/linkedin/feathr/offline/util/FeatureValueTypeValidator.scala index ee06f3acd..aec0b1aea 100644 --- a/src/main/scala/com/linkedin/feathr/offline/util/FeatureValueTypeValidator.scala +++ b/src/main/scala/com/linkedin/feathr/offline/util/FeatureValueTypeValidator.scala @@ -16,7 +16,7 @@ private[offline] object FeatureValueTypeValidator { features.foreach { case (key, value) => featureTypeConfigs.get(key).foreach( - featureTypeConfig => FeatureValueTypeValidator.validate(value, featureTypeConfig)) + featureTypeConfig => FeatureValueTypeValidator.validate(key, value, featureTypeConfig)) } } @@ -27,9 +27,9 @@ private[offline] object FeatureValueTypeValidator { * @param featureValue value extracted from data * @param featureTypeConfig user-defined config, optional */ - def validate(featureValue: FeatureValue, featureTypeConfig: Option[FeatureTypeConfig]): Unit = { + def validate(featureValue: FeatureValue, featureTypeConfig: Option[FeatureTypeConfig], featureName: String): Unit = { featureTypeConfig match { - case Some(f) => validate(featureValue, f) + case Some(f) => validate(featureName, featureValue, f) case None => } } @@ -41,31 +41,31 @@ private[offline] object FeatureValueTypeValidator { * @param featureValue value extracted from data * @param featureTypeConfig user-defined config */ - def validate(featureValue: FeatureValue, featureTypeConfig: FeatureTypeConfig): Unit = { + def validate(featureName: String, featureValue: FeatureValue, featureTypeConfig: FeatureTypeConfig): Unit = { val configFeatureTypes = featureTypeConfig.getFeatureType val valueBasicType = featureValue.getFeatureType.getBasicType if (configFeatureTypes != FeatureTypes.UNSPECIFIED) { if (valueBasicType != FeatureType.BasicType.TENSOR || configFeatureTypes != FeatureTypes.TENSOR) { if (configFeatureTypes != FeatureTypes.valueOf(valueBasicType.name)) { - throw new FeathrException(ErrorLabel.FEATHR_USER_ERROR, "The FeatureValue type: " + valueBasicType - + " is not consistent with the type specified in the Feathr config: ." + configFeatureTypes); + throw new FeathrException(ErrorLabel.FEATHR_USER_ERROR, "The FeatureValue type of : " + featureName + + " is " + valueBasicType + ", which is not consistent with the type specified in the Feathr config: ." + configFeatureTypes); } } else if (featureTypeConfig.getTensorType != null) { val configTensorType = featureTypeConfig.getTensorType val valueTensorType = featureValue.getAsTypedTensor.getType if (configTensorType.getValueType != null && configTensorType.getValueType != valueTensorType.getValueType) { - throw new FeathrException(ErrorLabel.FEATHR_USER_ERROR, "The tensor value type: " + valueTensorType - + " is not consistent with the type specified in the Feathr config: ." + configTensorType); + throw new FeathrException(ErrorLabel.FEATHR_USER_ERROR, "The tensor value type of :" + featureName + + " is " + valueTensorType + ", which is not consistent with the type specified in the Feathr config: ." + configTensorType); } if (configTensorType.getTensorCategory != null && configTensorType.getTensorCategory != valueTensorType.getTensorCategory) { - throw new FeathrException(ErrorLabel.FEATHR_USER_ERROR, "The tensor category type: " + valueTensorType - + " is not consistent with the type specified in the Feathr config: ." + configTensorType); + throw new FeathrException(ErrorLabel.FEATHR_USER_ERROR, "The tensor category type of : " + featureName + " is " + + valueTensorType + ", which is not consistent with the type specified in the Feathr config: ." + configTensorType); } if (configTensorType.getDimensionTypes != null && configTensorType.getDimensionTypes != valueTensorType.getDimensionTypes) { - throw new FeathrException(ErrorLabel.FEATHR_USER_ERROR, "The tensor dimension type: " + valueTensorType - + " is not consistent with the type specified in the Feathr config: ." + configTensorType); + throw new FeathrException(ErrorLabel.FEATHR_USER_ERROR, "The tensor dimension type of : " + featureName + " is " + + valueTensorType + ", which is not consistent with the type specified in the Feathr config: ." + configTensorType); } } } diff --git a/src/main/scala/com/linkedin/feathr/offline/util/FeaturizedDatasetUtils.scala b/src/main/scala/com/linkedin/feathr/offline/util/FeaturizedDatasetUtils.scala index d672cf5f5..534881f7a 100644 --- a/src/main/scala/com/linkedin/feathr/offline/util/FeaturizedDatasetUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/util/FeaturizedDatasetUtils.scala @@ -157,7 +157,7 @@ private[offline] object FeaturizedDatasetUtils { * @return the Quince-FDS struct or primitive */ - def tensorToDataFrameRow(tensor: TensorData, targetDataType: Option[DataType] = None): Any = { + def tensorToFDSDataFrameRow(tensor: TensorData, targetDataType: Option[DataType] = None): Any = { tensor match { case null => null case _ => diff --git a/src/test/scala/com/linkedin/feathr/offline/util/TestFeatureValueTypeValidator.scala b/src/test/scala/com/linkedin/feathr/offline/util/TestFeatureValueTypeValidator.scala index 1e9bae9b7..bda25b1cc 100644 --- a/src/test/scala/com/linkedin/feathr/offline/util/TestFeatureValueTypeValidator.scala +++ b/src/test/scala/com/linkedin/feathr/offline/util/TestFeatureValueTypeValidator.scala @@ -45,7 +45,7 @@ class TestFeatureValueTypeValidator extends TestFeathr { new FeatureValue(value, valueFeatureType.asInstanceOf[FeatureTypes]); } val featureTypeConfig = new FeatureTypeConfig(configFeatureTypes.asInstanceOf[FeatureTypes], configTensorType.asInstanceOf[TensorType], null) - FeatureValueTypeValidator.validate(featureValue, featureTypeConfig) + FeatureValueTypeValidator.validate("", featureValue, featureTypeConfig) } @DataProvider(name = "failTestCases") @@ -75,7 +75,7 @@ class TestFeatureValueTypeValidator extends TestFeathr { new FeatureValue(value, valueFeatureType.asInstanceOf[FeatureTypes]); } val featureTypeConfig = new FeatureTypeConfig(configFeatureTypes.asInstanceOf[FeatureTypes], configTensorType.asInstanceOf[TensorType], null) - FeatureValueTypeValidator.validate(featureValue, featureTypeConfig) + FeatureValueTypeValidator.validate("", featureValue, featureTypeConfig) } From 5fc3730907d6f4acdcec09e7fa7542d6dfc734a1 Mon Sep 17 00:00:00 2001 From: Jinghui Mo Date: Fri, 7 Oct 2022 20:02:36 -0400 Subject: [PATCH 04/68] Save lookup feature definition to HOCON files (#732) --- feathr_project/feathr/definition/lookup_feature.py | 6 ++++-- feathr_project/test/test_lookup_feature.py | 2 ++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/feathr_project/feathr/definition/lookup_feature.py b/feathr_project/feathr/definition/lookup_feature.py index 647df37ce..2f1b80ccd 100644 --- a/feathr_project/feathr/definition/lookup_feature.py +++ b/feathr_project/feathr/definition/lookup_feature.py @@ -4,12 +4,13 @@ from jinja2 import Template from feathr.definition.dtype import FeatureType +from feathr.definition.feature_derivations import DerivedFeature from feathr.definition.feature import FeatureBase from feathr.definition.transformation import RowTransformation from feathr.definition.typed_key import DUMMY_KEY, TypedKey from feathr.definition.aggregation import Aggregation -class LookupFeature(FeatureBase): +class LookupFeature(DerivedFeature): """A lookup feature is a feature defined on top of two other features, i.e. using the feature value of the base feature as key, to lookup the feature value from the expansion feature. e.g. a lookup feature user_purchased_item_avg_price could be key-ed by user_id, and computed by: base feature is user_purchased_item_ids. For a given user_id, it returns the item ids purchased by the user. @@ -36,7 +37,8 @@ def __init__(self, key: Optional[Union[TypedKey, List[TypedKey]]] = [DUMMY_KEY], registry_tags: Optional[Dict[str, str]] = None, ): - super(LookupFeature, self).__init__(name, feature_type, key=key, registry_tags=registry_tags) + super(LookupFeature, self).__init__(name, feature_type, input_features=[base_feature, expansion_feature], + transform="", key=key, registry_tags=registry_tags) self.base_feature = base_feature self.expansion_feature = expansion_feature self.aggregation = aggregation diff --git a/feathr_project/test/test_lookup_feature.py b/feathr_project/test/test_lookup_feature.py index ffdb9a686..82fe385a7 100644 --- a/feathr_project/test/test_lookup_feature.py +++ b/feathr_project/test/test_lookup_feature.py @@ -1,6 +1,7 @@ from feathr import Aggregation from feathr import Feature from feathr import LookupFeature +from feathr import DerivedFeature from feathr import FLOAT, FLOAT_VECTOR, ValueType, INT32_VECTOR from feathr import TypedKey @@ -39,4 +40,5 @@ def test_single_key_lookup_feature_to_config(): } }""" assert_config_equals(lookup_feature.to_feature_config(), lookup_feature_config) + assert(isinstance(lookup_feature, DerivedFeature)) \ No newline at end of file From 356f74b409c9e8e17a2eba51a6c21266ed465afe Mon Sep 17 00:00:00 2001 From: Jun Ki Min <42475935+loomlike@users.noreply.github.com> Date: Fri, 7 Oct 2022 17:03:32 -0700 Subject: [PATCH 05/68] Fix function string parsing (#725) * Add version. Fix function string parsing Signed-off-by: Jun Ki Min <42475935+loomlike@users.noreply.github.com> * Add unit test Signed-off-by: Jun Ki Min <42475935+loomlike@users.noreply.github.com> * Add comments Signed-off-by: Jun Ki Min <42475935+loomlike@users.noreply.github.com> Signed-off-by: Jun Ki Min <42475935+loomlike@users.noreply.github.com> --- feathr_project/feathr/__init__.py | 5 +++ .../udf/_preprocessing_pyudf_manager.py | 45 +++++++++++-------- .../udf/test_preprocessing_pyudf_manager.py | 15 +++++++ 3 files changed, 47 insertions(+), 18 deletions(-) create mode 100644 feathr_project/test/unit/udf/test_preprocessing_pyudf_manager.py diff --git a/feathr_project/feathr/__init__.py b/feathr_project/feathr/__init__.py index fae0cb60c..9b0cf0a49 100644 --- a/feathr_project/feathr/__init__.py +++ b/feathr_project/feathr/__init__.py @@ -1,3 +1,5 @@ +import pkg_resources + from .client import FeathrClient from .spark_provider.feathr_configurations import SparkExecutionConfiguration from .definition.feature_derivations import * @@ -74,3 +76,6 @@ 'FeaturePrinter', 'SparkExecutionConfiguration', ] + + +__version__ = pkg_resources.require("feathr")[0].version diff --git a/feathr_project/feathr/udf/_preprocessing_pyudf_manager.py b/feathr_project/feathr/udf/_preprocessing_pyudf_manager.py index ca7114343..55756ba3d 100644 --- a/feathr_project/feathr/udf/_preprocessing_pyudf_manager.py +++ b/feathr_project/feathr/udf/_preprocessing_pyudf_manager.py @@ -1,12 +1,15 @@ +import ast import inspect import os +import pickle from pathlib import Path from typing import List, Optional, Union -import pickle -from feathr.definition.anchor import FeatureAnchor + from jinja2 import Template + +from feathr.definition.anchor import FeatureAnchor from feathr.definition.source import HdfsSource -import ast + # Some metadata that are only needed by Feathr FEATHR_PYSPARK_METADATA = 'generated_feathr_pyspark_metadata' @@ -42,7 +45,7 @@ def build_anchor_preprocessing_metadata(anchor_list: List[FeatureAnchor], local_ # delete the file if it already exists to avoid caching previous results for f in [client_udf_repo_path, metadata_path, pyspark_driver_path]: if os.path.exists(f): - os.remove(f) + os.remove(f) for anchor in anchor_list: # only support batch source preprocessing for now. @@ -73,23 +76,29 @@ def build_anchor_preprocessing_metadata(anchor_list: List[FeatureAnchor], local_ with open(feathr_pyspark_metadata_abs_path, 'wb') as file: pickle.dump(features_with_preprocessing, file) + @staticmethod - def _parse_function_str_for_name(source: str) -> str: - """ - Use AST to parse the functions and get the name out. + def _parse_function_str_for_name(fn_str: str) -> str: + """Use AST to parse the function string and get the name out. + + Args: + fn_str: Function code in string. + + Returns: + Name of the function. """ - if source is None: + if not fn_str: return None - tree = ast.parse(source) + + tree = ast.parse(fn_str) + + # tree.body contains a list of function definition objects parsed from the input string. + # Currently, we only accept a single function. if len(tree.body) != 1 or not isinstance(tree.body[0], ast.FunctionDef): - raise ValueError('provided code fragment is not a single function') - code = compile(source=tree, filename='custom.py',mode= 'exec') - # https://docs.python.org/3/library/inspect.html see the inspect module for more details - # tuple of names other than arguments and function locals. Assume there will be only one function, so will return the first as the name - for ele in code.co_consts: - # find the first object, that is the str, this will be the name of the function - if isinstance(ele, str): - return ele + raise ValueError("provided code fragment is not a single function") + + # Get the function name from the function definition. + return tree.body[0].name @staticmethod @@ -174,7 +183,7 @@ def prepare_pyspark_udf_files(feature_names: List[str], local_workspace_dir): client_udf_repo_path = os.path.join(local_workspace_dir, FEATHR_CLIENT_UDF_FILE_NAME) # write pyspark_driver_template_abs_path and then client_udf_repo_path filenames = [pyspark_driver_template_abs_path, client_udf_repo_path] - + with open(pyspark_driver_path, 'w') as outfile: for fname in filenames: with open(fname) as infile: diff --git a/feathr_project/test/unit/udf/test_preprocessing_pyudf_manager.py b/feathr_project/test/unit/udf/test_preprocessing_pyudf_manager.py new file mode 100644 index 000000000..1daa87632 --- /dev/null +++ b/feathr_project/test/unit/udf/test_preprocessing_pyudf_manager.py @@ -0,0 +1,15 @@ +import pytest + +from feathr.udf._preprocessing_pyudf_manager import _PreprocessingPyudfManager + + +@pytest.mark.parametrize( + "fn_name, fn_str", + [ + ("fn_without_type_hint", "def fn_without_type_hint(a):\n return a + 10\n"), + ("fn_with_type_hint", "def fn_with_type_hint(a: int) -> int:\n return a + 10\n"), + ("fn_with_complex_type_hint", "def fn_with_complex_type_hint(a: Union[int, float]) -> Union[int, float]:\n return a + 10\n"), + ] +) +def test__parse_function_str_for_name(fn_name, fn_str): + assert fn_name == _PreprocessingPyudfManager._parse_function_str_for_name(fn_str) From b433039d9c416863a5ee8758a2e885fa7af5aae2 Mon Sep 17 00:00:00 2001 From: Enya-Yx <108409954+enya-yx@users.noreply.github.com> Date: Tue, 11 Oct 2022 07:19:25 +0800 Subject: [PATCH 06/68] Apply a same credential within each sample (#718) Co-authored-by: enya-yx --- docs/samples/fraud_detection_demo.ipynb | 2 +- docs/samples/product_recommendation_demo.ipynb | 2 +- docs/samples/product_recommendation_demo_advanced.ipynb | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/samples/fraud_detection_demo.ipynb b/docs/samples/fraud_detection_demo.ipynb index 88c672160..0f35bc3bb 100644 --- a/docs/samples/fraud_detection_demo.ipynb +++ b/docs/samples/fraud_detection_demo.ipynb @@ -284,7 +284,7 @@ }, "outputs": [], "source": [ - "client = FeathrClient(config_path=tmp.name)" + "client = FeathrClient(config_path=tmp.name, credential=credential)" ] }, { diff --git a/docs/samples/product_recommendation_demo.ipynb b/docs/samples/product_recommendation_demo.ipynb index aa7699eb5..8636bb992 100644 --- a/docs/samples/product_recommendation_demo.ipynb +++ b/docs/samples/product_recommendation_demo.ipynb @@ -284,7 +284,7 @@ "metadata": {}, "outputs": [], "source": [ - "feathr_client = FeathrClient(config_path=tmp.name)" + "feathr_client = FeathrClient(config_path=tmp.name, credential=credential)" ] }, { diff --git a/docs/samples/product_recommendation_demo_advanced.ipynb b/docs/samples/product_recommendation_demo_advanced.ipynb index fff2a1cd5..de3eb2654 100644 --- a/docs/samples/product_recommendation_demo_advanced.ipynb +++ b/docs/samples/product_recommendation_demo_advanced.ipynb @@ -420,7 +420,7 @@ }, "outputs": [], "source": [ - "client = FeathrClient(config_path=tmp.name)" + "client = FeathrClient(config_path=tmp.name, credential=credential)" ] }, { From a3255976b8e9ffe993b11e1115e29704972cc1c0 Mon Sep 17 00:00:00 2001 From: Enya-Yx <108409954+enya-yx@users.noreply.github.com> Date: Tue, 11 Oct 2022 13:11:31 +0800 Subject: [PATCH 07/68] Enable incremental for HDFS sink (#695) * Enable incremental for HDFS sink * Add docstring * Add docs * minor fix * minor changes * quick fix Co-authored-by: enya-yx --- docs/concepts/materializing-features.md | 12 +++++++++++ .../definition/_materialization_utils.py | 3 +++ .../definition/materialization_settings.py | 5 ++++- feathr_project/feathr/definition/sink.py | 20 ++++++++++++++++--- .../test/test_feature_materialization.py | 5 +++++ 5 files changed, 41 insertions(+), 4 deletions(-) diff --git a/docs/concepts/materializing-features.md b/docs/concepts/materializing-features.md index 28d824525..eacf980ff 100644 --- a/docs/concepts/materializing-features.md +++ b/docs/concepts/materializing-features.md @@ -31,6 +31,18 @@ More reference on the APIs: In the above example, we define a Redis table called `nycTaxiDemoFeature` and materialize two features called `f_location_avg_fare` and `f_location_max_fare` to Redis. +## Incremental Aggregation +Use incremental aggregation will significantly expedite the WindowAggTransformation feature calculation. +For example, aggregation sum of a feature F within a 180-day window at day T can be expressed as: F(T) = F(T - 1)+DirectAgg(T-1)-DirectAgg(T - 181). +Once a SNAPSHOT of the first day is generated, the calculation for the following days can leverage it. + +A storeName is required if incremental aggregated is enabled. There could be multiple output Datasets, and each of them need to be stored in a separate folder. The storeName is used as the folder name to create under the base "path". + +Incremental aggregation is enabled by default when using HdfsSink. + +More reference on the APIs: +- [HdfsSink API doc](https://feathr.readthedocs.io/en/latest/feathr.html#feathr.HdfsSink) + ## Feature Backfill It is also possible to backfill the features till a particular time, like below. If the `BackfillTime` part is not specified, it's by default to `now()` (i.e. if not specified, it's equivalent to `BackfillTime(start=now, end=now, step=timedelta(days=1))`). diff --git a/feathr_project/feathr/definition/_materialization_utils.py b/feathr_project/feathr/definition/_materialization_utils.py index ef066b104..b49f7dced 100644 --- a/feathr_project/feathr/definition/_materialization_utils.py +++ b/feathr_project/feathr/definition/_materialization_utils.py @@ -10,6 +10,9 @@ def _to_materialization_config(settings: MaterializationSettings): endTime: "{{ settings.backfill_time.end.strftime('%Y-%m-%d %H:%M:%S') }}" endTimeFormat: "yyyy-MM-dd HH:mm:ss" resolution: DAILY + {% if settings.has_hdfs_sink == True %} + enableIncremental = true + {% endif %} output:[ {% for sink in settings.sinks %} {{sink.to_feature_config()}} diff --git a/feathr_project/feathr/definition/materialization_settings.py b/feathr_project/feathr/definition/materialization_settings.py index 8cdc2fc71..27b644139 100644 --- a/feathr_project/feathr/definition/materialization_settings.py +++ b/feathr_project/feathr/definition/materialization_settings.py @@ -32,7 +32,10 @@ def __init__(self, name: str, sinks: List[Sink], feature_names: List[str], backf now = datetime.now() self.backfill_time = backfill_time if backfill_time else BackfillTime(start=now, end=now, step=timedelta(days=1)) for sink in sinks: - if isinstance(sink, RedisSink): + if isinstance(sink, HdfsSink): + self.has_hdfs_sink = True + sink.aggregation_features = feature_names + elif isinstance(sink, RedisSink): sink.aggregation_features = feature_names self.sinks = sinks self.feature_names = feature_names diff --git a/feathr_project/feathr/definition/sink.py b/feathr_project/feathr/definition/sink.py index a23718a44..71c406561 100644 --- a/feathr_project/feathr/definition/sink.py +++ b/feathr_project/feathr/definition/sink.py @@ -103,25 +103,35 @@ def to_argument(self): class HdfsSink(Sink): """Offline Hadoop HDFS-compatible(HDFS, delta lake, Azure blog storage etc) sink that is used to store feature data. - The result is in AVRO format. + The result is in AVRO format. + + Incremental aggregation is enabled by default when using HdfsSink. Use incremental aggregation will significantly expedite the WindowAggTransformation feature calculation. + For example, aggregation sum of a feature F within a 180-day window at day T can be expressed as: F(T) = F(T - 1)+DirectAgg(T-1)-DirectAgg(T - 181). + Once a SNAPSHOT of the first day is generated, the calculation for the following days can leverage it. Attributes: output_path: output path + store_name: the folder name under the base "path". Used especially for the current dataset to support 'Incremental' aggregation. + """ - def __init__(self, output_path: str) -> None: + def __init__(self, output_path: str, store_name: Optional[str]="df0") -> None: self.output_path = output_path - + self.store_name = store_name # Sample generated HOCON config: # operational: { # name: testFeatureGen # endTime: 2019-05-01 # endTimeFormat: "yyyy-MM-dd" # resolution: DAILY + # enableIncremental = true # output:[ # { # name: HDFS + # outputFormat: RAW_DATA # params: { # path: "/user/featureGen/hdfsResult/" + # features: [mockdata_a_ct_gen, mockdata_a_sample_gen] + # storeName: "yyyy/MM/dd" # } # } # ] @@ -132,11 +142,15 @@ def to_feature_config(self) -> str: tm = Template(""" { name: HDFS + outputFormat: RAW_DATA params: { path: "{{sink.output_path}}" {% if sink.aggregation_features %} features: [{{','.join(sink.aggregation_features)}}] {% endif %} + {% if sink.store_name %} + storeName: "{{sink.store_name}}" + {% endif %} } } """) diff --git a/feathr_project/test/test_feature_materialization.py b/feathr_project/test/test_feature_materialization.py index 62b84d367..edd9bb537 100644 --- a/feathr_project/test/test_feature_materialization.py +++ b/feathr_project/test/test_feature_materialization.py @@ -61,12 +61,17 @@ def test_feature_materialization_offline_config(): endTime: "2020-05-20 00:00:00" endTimeFormat: "yyyy-MM-dd HH:mm:ss" resolution: DAILY + enableIncremental = true output:[ { name: HDFS + outputFormat: RAW_DATA params: { path: "abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/demo_data/output/hdfs_test.avro" + features: [f_location_avg_fare,f_location_max_fare] + storeName: "df0" } + } ] } From bb679390b30ae270c214c799d07b2a26595f6d14 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BE=90=E8=BE=B0?= Date: Wed, 12 Oct 2022 00:16:15 +0800 Subject: [PATCH 08/68] #492 fix, fail only if different sources have same name (#733) --- feathr_project/feathr/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/feathr_project/feathr/client.py b/feathr_project/feathr/client.py index f21d37d23..759a33365 100644 --- a/feathr_project/feathr/client.py +++ b/feathr_project/feathr/client.py @@ -216,7 +216,7 @@ def build_features(self, anchor_list: List[FeatureAnchor] = [], derived_feature_ f"definitions. Anchor name of {anchor} is already defined in {anchor_names[anchor.name]}") else: anchor_names[anchor.name] = anchor - if anchor.source.name in source_names: + if anchor.source.name in source_names and (anchor.source is not source_names[anchor.source.name]): raise RuntimeError(f"Source name should be unique but there are duplicate source names in your source " f"definitions. Source name of {anchor.source} is already defined in {source_names[anchor.source.name]}") else: From 4f76e19cdbb2472423c93e44d0c3c843b9765e57 Mon Sep 17 00:00:00 2001 From: Enya-Yx <108409954+enya-yx@users.noreply.github.com> Date: Wed, 12 Oct 2022 10:55:15 +0800 Subject: [PATCH 09/68] Remove unused credentials and deprecated purview settings (#708) * Remove unused credentials and deprecated purview settings --- docs/how-to-guides/azure-deployment-cli.md | 1 - docs/how-to-guides/feathr-configuration-and-env.md | 6 +++--- docs/quickstart_synapse.md | 6 ------ docs/samples/customer360/Customer360.ipynb | 11 +---------- .../databricks_quickstart_nyc_taxi_driver.ipynb | 8 +------- docs/samples/product_recommendation_demo.ipynb | 2 +- .../product_recommendation_demo_advanced.ipynb | 2 +- 7 files changed, 7 insertions(+), 29 deletions(-) diff --git a/docs/how-to-guides/azure-deployment-cli.md b/docs/how-to-guides/azure-deployment-cli.md index 3762f7b3f..70067b148 100644 --- a/docs/how-to-guides/azure-deployment-cli.md +++ b/docs/how-to-guides/azure-deployment-cli.md @@ -117,7 +117,6 @@ echo "AZURE_TENANT_ID: $sp_tenantid" echo "AZURE_CLIENT_SECRET: $sp_password" This will give three variables: AZURE_CLIENT_ID, AZURE_TENANT_ID and AZURE_CLIENT_SECRET. You will need them later. ``` - Note: **You should save AZURE_CLIENT_SECRET because you will only see it once here** ## Create a storage account diff --git a/docs/how-to-guides/feathr-configuration-and-env.md b/docs/how-to-guides/feathr-configuration-and-env.md index fd32fb2f6..e4d393a43 100644 --- a/docs/how-to-guides/feathr-configuration-and-env.md +++ b/docs/how-to-guides/feathr-configuration-and-env.md @@ -77,9 +77,9 @@ Feathr will get the configurations in the following order: | ONLINE_STORE__REDIS__SSL_ENABLED | Whether SSL is enabled to access Redis cluster. | Required if using Redis as online store. | | REDIS_PASSWORD | Password for the Redis cluster. | Required if using Redis as online store. | | FEATURE_REGISTRY__API_ENDPOINT | Specifies registry endpoint. | Required if using registry service. | -| FEATURE_REGISTRY__PURVIEW__PURVIEW_NAME | Configure the name of the purview endpoint. | Required if using Purview directly without registry service. Deprecate soon, see [here](#deprecation) for more details.| -| FEATURE_REGISTRY__PURVIEW__DELIMITER | See [here](#FEATURE_REGISTRY__PURVIEW__DELIMITER) for more details. | Required if using Purview directly without registry service. Deprecate soon, see [here](#deprecation) for more details.| -| FEATURE_REGISTRY__PURVIEW__TYPE_SYSTEM_INITIALIZATION | Controls whether the type system (think this as the "schema" for the registry) will be initialized or not. Usually this is only required to be set to `True` to initialize schema, and then you can set it to `False` to shorten the initialization time. | Required if using Purview directly without registry service. Deprecate soon, see [here](#deprecation) for more details.| +| FEATURE_REGISTRY__PURVIEW__PURVIEW_NAME (Deprecated Soon) | Configure the name of the purview endpoint. | Required if using Purview directly without registry service. Deprecate soon, see [here](#deprecation) for more details.| +| FEATURE_REGISTRY__PURVIEW__DELIMITER (Deprecated Soon) | See [here](#FEATURE_REGISTRY__PURVIEW__DELIMITER) for more details. | Required if using Purview directly without registry service. Deprecate soon, see [here](#deprecation) for more details.| +| FEATURE_REGISTRY__PURVIEW__TYPE_SYSTEM_INITIALIZATION (Deprecated Soon)| Controls whether the type system (think this as the "schema" for the registry) will be initialized or not. Usually this is only required to be set to `True` to initialize schema, and then you can set it to `False` to shorten the initialization time. | Required if using Purview directly without registry service. Deprecate soon, see [here](#deprecation) for more details.| # Explanation for selected configurations diff --git a/docs/quickstart_synapse.md b/docs/quickstart_synapse.md index 5dee17931..0a66a96bb 100644 --- a/docs/quickstart_synapse.md +++ b/docs/quickstart_synapse.md @@ -61,9 +61,6 @@ project_config: # Redis password for your online store - "REDIS_PASSWORD" # Client IDs and Client Secret for the service principal. Read the getting started docs on how to get those information. - - "AZURE_CLIENT_ID" - - "AZURE_TENANT_ID" - - "AZURE_CLIENT_SECRET" offline_store: --- @@ -98,9 +95,6 @@ These values can also be retrieved by using cloud key value store, such as [Azur ```python import os os.environ['REDIS_PASSWORD'] = '' -os.environ['AZURE_CLIENT_ID'] = '' -os.environ['AZURE_TENANT_ID'] = '' -os.environ['AZURE_CLIENT_SECRET'] = '' ``` Please refer to [A note on using azure key vault to store credentials](https://github.com/feathr-ai/feathr/blob/41e7496b38c43af6d7f8f1de842f657b27840f6d/docs/how-to-guides/feathr-configuration-and-env.md#a-note-on-using-azure-key-vault-to-store-credentials) for more details. diff --git a/docs/samples/customer360/Customer360.ipynb b/docs/samples/customer360/Customer360.ipynb index 4b202e13a..db042011b 100644 --- a/docs/samples/customer360/Customer360.ipynb +++ b/docs/samples/customer360/Customer360.ipynb @@ -192,9 +192,6 @@ " project_name: 'customer360'\n", " required_environment_variables:\n", " - 'REDIS_PASSWORD'\n", - " - 'AZURE_CLIENT_ID'\n", - " - 'AZURE_TENANT_ID'\n", - " - 'AZURE_CLIENT_SECRET'\n", " - 'ADLS_ACCOUNT'\n", " - 'ADLS_KEY'\n", " - 'WASB_ACCOUNT'\n", @@ -239,10 +236,7 @@ " port: 6380\n", " ssl_enabled: True\n", "feature_registry:\n", - " purview:\n", - " type_system_initialization: true\n", - " purview_name: ''\n", - " delimiter: '__'\n", + " api_endpoint: \"https://.azurewebsites.net/api/v1\"\n", "\"\"\"\n", "# write this configuration string to a temporary location and load it to Feathr\n", "tmp = tempfile.NamedTemporaryFile(mode='w', delete=False)\n", @@ -331,9 +325,6 @@ "source": [ "import os\n", "os.environ['REDIS_PASSWORD'] = ''\n", - "os.environ['AZURE_CLIENT_ID'] = ''\n", - "os.environ['AZURE_TENANT_ID'] = ''\n", - "os.environ['AZURE_CLIENT_SECRET'] = ''\n", "os.environ['ADLS_ACCOUNT'] = ''\n", "os.environ['ADLS_KEY'] = ''\n", "os.environ['WASB_ACCOUNT'] = \"\"\n", diff --git a/docs/samples/databricks/databricks_quickstart_nyc_taxi_driver.ipynb b/docs/samples/databricks/databricks_quickstart_nyc_taxi_driver.ipynb index 82aaf3832..52790f884 100644 --- a/docs/samples/databricks/databricks_quickstart_nyc_taxi_driver.ipynb +++ b/docs/samples/databricks/databricks_quickstart_nyc_taxi_driver.ipynb @@ -332,9 +332,6 @@ " project_name: 'feathr_getting_started2'\n", " required_environment_variables:\n", " - 'REDIS_PASSWORD'\n", - " - 'AZURE_CLIENT_ID'\n", - " - 'AZURE_TENANT_ID'\n", - " - 'AZURE_CLIENT_SECRET'\n", "offline_store:\n", " adls:\n", " adls_enabled: true\n", @@ -364,10 +361,7 @@ " port: 6380\n", " ssl_enabled: True\n", "feature_registry:\n", - " purview:\n", - " type_system_initialization: true\n", - " purview_name: ''\n", - " delimiter: '__'\n", + " api_endpoint: \"https://.azurewebsites.net/api/v1\"\n", "\"\"\"\n", "tmp = tempfile.NamedTemporaryFile(mode='w', delete=False)\n", "with open(tmp.name, \"w\") as text_file:\n", diff --git a/docs/samples/product_recommendation_demo.ipynb b/docs/samples/product_recommendation_demo.ipynb index 8636bb992..9ae3ee34d 100644 --- a/docs/samples/product_recommendation_demo.ipynb +++ b/docs/samples/product_recommendation_demo.ipynb @@ -260,7 +260,7 @@ "\n", "You should setup the environment variables in order to run this sample. More environment variables can be set by referring to [feathr_config.yaml](https://github.com/linkedin/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml) and use that as the source of truth. It also has more explanations on the meaning of each variable.\n", "\n", - "To run this notebook, for Azure users, you need AZURE_CLIENT_ID, AZURE_TENANT_ID, AZURE_CLIENT_SECRET and REDIS_PASSWORD.\n", + "To run this notebook, for Azure users, you need REDIS_PASSWORD.\n", "To run this notebook, for Databricks useres, you need DATABRICKS_WORKSPACE_TOKEN_VALUE and REDIS_PASSWORD." ] }, diff --git a/docs/samples/product_recommendation_demo_advanced.ipynb b/docs/samples/product_recommendation_demo_advanced.ipynb index de3eb2654..e4c5917a5 100644 --- a/docs/samples/product_recommendation_demo_advanced.ipynb +++ b/docs/samples/product_recommendation_demo_advanced.ipynb @@ -389,7 +389,7 @@ "\n", "You should setup the environment variables in order to run this sample. More environment variables can be set by referring to [feathr_config.yaml](https://github.com/linkedin/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml) and use that as the source of truth. It also has more explanations on the meaning of each variable.\n", "\n", - "To run this notebook, for Azure users, you need AZURE_CLIENT_ID, AZURE_TENANT_ID, AZURE_CLIENT_SECRET and REDIS_PASSWORD.\n", + "To run this notebook, for Azure users, you need REDIS_PASSWORD.\n", "To run this notebook, for Databricks useres, you need DATABRICKS_WORKSPACE_TOKEN_VALUE and REDIS_PASSWORD." ] }, From 18d776d3a35edd23e1d8db2bd643c5d88bb09f5f Mon Sep 17 00:00:00 2001 From: Blair Chen Date: Wed, 12 Oct 2022 11:11:39 +0800 Subject: [PATCH 10/68] Revoke token submitted by mistaken (#730) --- .../test_user_workspace/feathr_config_registry_purview.yaml | 2 +- .../feathr_config_registry_purview_rbac.yaml | 2 +- .../test/test_user_workspace/feathr_config_registry_sql.yaml | 2 +- .../test_user_workspace/feathr_config_registry_sql_rbac.yaml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml index afe923163..f716da0b4 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml @@ -28,7 +28,7 @@ spark_config: feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' - workspace_token_value: 'dapid8ddd83000dc2863763b7d47f0e8f3db' + workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml index fb88972f7..c842bc702 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml @@ -28,7 +28,7 @@ spark_config: feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' - workspace_token_value: 'dapid8ddd83000dc2863763b7d47f0e8f3db' + workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml index 486eed1e4..dcb73d827 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml @@ -28,7 +28,7 @@ spark_config: feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' - workspace_token_value: 'dapid8ddd83000dc2863763b7d47f0e8f3db' + workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml index 4ad7d35db..29c6889e8 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml @@ -28,7 +28,7 @@ spark_config: feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' - workspace_token_value: 'dapid8ddd83000dc2863763b7d47f0e8f3db' + workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" From 9f446bf287387ee20effc1cb7e9b39869f7451e8 Mon Sep 17 00:00:00 2001 From: Hangfei Lin Date: Tue, 11 Oct 2022 20:35:32 -0700 Subject: [PATCH 11/68] Update product_recommendation_demo.ipynb --- docs/samples/product_recommendation_demo.ipynb | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/samples/product_recommendation_demo.ipynb b/docs/samples/product_recommendation_demo.ipynb index 9ae3ee34d..44febb062 100644 --- a/docs/samples/product_recommendation_demo.ipynb +++ b/docs/samples/product_recommendation_demo.ipynb @@ -21,10 +21,10 @@ "\n", "First step is to provision required cloud resources if you want to use Feathr. Feathr provides a python based client to interact with cloud resources.\n", "\n", - "Please follow the steps [here](https://feathr-ai.github.io/feathr/how-to-guides/azure-deployment-arm.html) to provision required cloud resources. Due to the complexity of the possible cloud environment, it is almost impossible to create a script that works for all the use cases. Because of this, [azure_resource_provision.sh](https://github.com/linkedin/feathr/blob/main/docs/how-to-guides/azure_resource_provision.sh) is a full end to end command line to create all the required resources, and you can tailor the script as needed, while [the companion documentation](https://feathr-ai.github.io/feathr/how-to-guides/azure-deployment-cli.html) can be used as a complete guide for using that shell script. \n", + "Please follow the steps [here](https://feathr-ai.github.io/feathr/how-to-guides/azure-deployment-arm.html) to provision required cloud resources. Due to the complexity of the possible cloud environment, it is almost impossible to create a script that works for all the use cases. Because of this, [azure_resource_provision.sh](https://github.com/feathr-ai/feathr/blob/main/docs/how-to-guides/azure_resource_provision.sh) is a full end to end command line to create all the required resources, and you can tailor the script as needed, while [the companion documentation](https://feathr-ai.github.io/feathr/how-to-guides/azure-deployment-cli.html) can be used as a complete guide for using that shell script. \n", "\n", "\n", - "![Architecture](https://github.com/linkedin/feathr/blob/main/docs/images/architecture.png?raw=true)" + "![Architecture](https://github.com/feathr-ai/feathr/blob/main/docs/images/architecture.png?raw=true)" ] }, { @@ -190,7 +190,7 @@ "\n", "In the first step (Provision cloud resources), you should have provisioned all the required cloud resources. If you use Feathr CLI to create a workspace, you should have a folder with a file called `feathr_config.yaml` in it with all the required configurations. Otherwise, update the configuration below.\n", "\n", - "The code below will write this configuration string to a temporary location and load it to Feathr. Please still refer to [feathr_config.yaml](https://github.com/linkedin/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml) and use that as the source of truth. It should also have more explanations on the meaning of each variable." + "The code below will write this configuration string to a temporary location and load it to Feathr. Please still refer to [feathr_config.yaml](https://github.com/feathr-ai/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml) and use that as the source of truth. It should also have more explanations on the meaning of each variable." ] }, { @@ -201,7 +201,7 @@ "source": [ "import tempfile\n", "yaml_config = \"\"\"\n", - "# Please refer to https://github.com/linkedin/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml for explanations on the meaning of each field.\n", + "# Please refer to https://github.com/feathr-ai/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml for explanations on the meaning of each field.\n", "api_version: 1\n", "project_config:\n", " project_name: 'feathr_getting_started'\n", @@ -258,7 +258,7 @@ "source": [ "## Prerequisite: Setup necessary environment variables (Skip this step if using the above Quick Start Template)\n", "\n", - "You should setup the environment variables in order to run this sample. More environment variables can be set by referring to [feathr_config.yaml](https://github.com/linkedin/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml) and use that as the source of truth. It also has more explanations on the meaning of each variable.\n", + "You should setup the environment variables in order to run this sample. More environment variables can be set by referring to [feathr_config.yaml](https://github.com/feathr-ai/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml) and use that as the source of truth. It also has more explanations on the meaning of each variable.\n", "\n", "To run this notebook, for Azure users, you need REDIS_PASSWORD.\n", "To run this notebook, for Databricks useres, you need DATABRICKS_WORKSPACE_TOKEN_VALUE and REDIS_PASSWORD." @@ -339,7 +339,7 @@ " After a bit of data exploration, we want to create a training dataset like this:\n", "\n", " \n", - "![Feature Flow](https://github.com/linkedin/feathr/blob/main/docs/images/product_recommendation.jpg?raw=true)" + "![Feature Flow](https://github.com/feathr-ai/feathr/blob/main/docs/images/product_recommendation.jpg?raw=true)" ] }, { @@ -360,7 +360,7 @@ "1. Feature source: what source data that this feature is based on\n", "2. Transformation: what transformation is used to transform the source data into feature. Transformation can be optional when you just want to take a column out from the source data.\n", "\n", - "(For more details on feature definition, please refer to the [Feathr Feature Definition Guide](https://github.com/linkedin/feathr/blob/main/docs/concepts/feature-definition.md))" + "(For more details on feature definition, please refer to the [Feathr Feature Definition Guide](https://github.com/feathr-ai/feathr/blob/main/docs/concepts/feature-definition.md))" ] }, { @@ -569,7 +569,7 @@ "To create a training dataset using Feathr, we need to provide a feature join settings to specify\n", "what features and how these features should be joined to the observation data. \n", "\n", - "(To learn more on this topic, please refer to [Point-in-time Correctness](https://github.com/linkedin/feathr/blob/main/docs/concepts/point-in-time-join.md))" + "(To learn more on this topic, please refer to [Point-in-time Correctness](https://github.com/feathr-ai/feathr/blob/main/docs/concepts/point-in-time-join.md))" ] }, { @@ -701,7 +701,7 @@ "![img](../images/online_inference.jpg)\n", "\n", "\n", - "In this section, we will focus on materialize features to online store. For materialization to offline store, you can check out our [user guide](https://github.com/linkedin/feathr/blob/main/docs/concepts/materializing-features.md#materializing-features-to-offline-store).\n", + "In this section, we will focus on materialize features to online store. For materialization to offline store, you can check out our [user guide](https://github.com/feathr-ai/feathr/blob/main/docs/concepts/materializing-features.md#materializing-features-to-offline-store).\n", "\n", "We can push the computed features to the online store like below:" ] From 39c14ca77e629adf111778ae70c720084d38f5ef Mon Sep 17 00:00:00 2001 From: Enya-Yx <108409954+enya-yx@users.noreply.github.com> Date: Wed, 12 Oct 2022 23:02:44 +0800 Subject: [PATCH 12/68] Fix synapse errors not print out issue (#734) Co-authored-by: enya-yx --- feathr_project/feathr/spark_provider/_synapse_submission.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/feathr_project/feathr/spark_provider/_synapse_submission.py b/feathr_project/feathr/spark_provider/_synapse_submission.py index b72acdc42..010444715 100644 --- a/feathr_project/feathr/spark_provider/_synapse_submission.py +++ b/feathr_project/feathr/spark_provider/_synapse_submission.py @@ -325,7 +325,7 @@ def create_spark_batch_job(self, job_name, main_file, class_name=None, def get_driver_log(self, job_id) -> str: # @see: https://docs.microsoft.com/en-us/azure/synapse-analytics/spark/connect-monitor-azure-synapse-spark-application-level-metrics app_id = self.get_spark_batch_job(job_id).app_id - url = "%s/sparkhistory/api/v1/sparkpools/%s/livyid/%s/applications/%s/driverlog/stdout/?isDownload=true" % (self._synapse_dev_url, self._spark_pool_name, job_id, app_id) + url = "%s/sparkhistory/api/v1/sparkpools/%s/livyid/%s/applications/%s/driverlog/stderr/?isDownload=true" % (self._synapse_dev_url, self._spark_pool_name, job_id, app_id) token = self._credential.get_token("https://dev.azuresynapse.net/.default").token req = urllib.request.Request(url=url, headers={"authorization": "Bearer %s" % token}) resp = urllib.request.urlopen(req) From c075dc2d89a0ade329ea07e759decdd8a021d242 Mon Sep 17 00:00:00 2001 From: Jun Ki Min <42475935+loomlike@users.noreply.github.com> Date: Wed, 12 Oct 2022 10:28:03 -0700 Subject: [PATCH 13/68] Spark config passing bug fix for local spark submission (#729) * Fix local spark output file-format bug Signed-off-by: Jun Ki Min <42475935+loomlike@users.noreply.github.com> * Add dev dependencies. Add unit-test for local spark job launcher Signed-off-by: Jun Ki Min <42475935+loomlike@users.noreply.github.com> * Fix local spark submission unused param error Signed-off-by: Jun Ki Min <42475935+loomlike@users.noreply.github.com> Signed-off-by: Jun Ki Min <42475935+loomlike@users.noreply.github.com> --- feathr_project/feathr/client.py | 53 +++-- feathr_project/feathr/spark_provider/_abc.py | 4 +- .../spark_provider/_localspark_submission.py | 197 +++++++++--------- feathr_project/pyproject.toml | 13 +- feathr_project/setup.py | 14 +- .../test_localspark_submission.py | 51 +++++ 6 files changed, 204 insertions(+), 128 deletions(-) create mode 100644 feathr_project/test/unit/spark_provider/test_localspark_submission.py diff --git a/feathr_project/feathr/client.py b/feathr_project/feathr/client.py index 759a33365..1f77d61e0 100644 --- a/feathr_project/feathr/client.py +++ b/feathr_project/feathr/client.py @@ -1,39 +1,36 @@ import base64 +import copy import logging import os import tempfile from typing import Dict, List, Union -from feathr.definition.feature import FeatureBase -import copy -import redis from azure.identity import DefaultAzureCredential from jinja2 import Template from pyhocon import ConfigFactory -from feathr.definition.sink import Sink -from feathr.registry.feature_registry import default_registry_client - -from feathr.spark_provider._databricks_submission import _FeathrDatabricksJobLauncher -from feathr.spark_provider._synapse_submission import _FeathrSynapseJobLauncher -from feathr.spark_provider._localspark_submission import _FeathrDLocalSparkJobLauncher +import redis -from feathr.definition._materialization_utils import _to_materialization_config -from feathr.udf._preprocessing_pyudf_manager import _PreprocessingPyudfManager from feathr.constants import * -from feathr.spark_provider.feathr_configurations import SparkExecutionConfiguration +from feathr.definition._materialization_utils import _to_materialization_config +from feathr.definition.anchor import FeatureAnchor +from feathr.definition.feature import FeatureBase from feathr.definition.feature_derivations import DerivedFeature from feathr.definition.materialization_settings import MaterializationSettings from feathr.definition.monitoring_settings import MonitoringSettings -from feathr.protobuf.featureValue_pb2 import FeatureValue from feathr.definition.query_feature_list import FeatureQuery from feathr.definition.settings import ObservationSettings -from feathr.definition.feature_derivations import DerivedFeature -from feathr.definition.anchor import FeatureAnchor +from feathr.definition.sink import Sink +from feathr.protobuf.featureValue_pb2 import FeatureValue +from feathr.registry.feature_registry import default_registry_client +from feathr.spark_provider._databricks_submission import _FeathrDatabricksJobLauncher +from feathr.spark_provider._localspark_submission import _FeathrLocalSparkJobLauncher +from feathr.spark_provider._synapse_submission import _FeathrSynapseJobLauncher from feathr.spark_provider.feathr_configurations import SparkExecutionConfiguration +from feathr.udf._preprocessing_pyudf_manager import _PreprocessingPyudfManager from feathr.utils._envvariableutil import _EnvVaraibleUtil from feathr.utils._file_utils import write_to_file from feathr.utils.feature_printer import FeaturePrinter -from feathr.utils.spark_job_params import FeatureJoinJobParams, FeatureGenerationJobParams +from feathr.utils.spark_job_params import FeatureGenerationJobParams, FeatureJoinJobParams class FeathrClient(object): @@ -161,7 +158,7 @@ def __init__(self, config_path:str = "./feathr_config.yaml", local_workspace_dir self._FEATHR_JOB_JAR_PATH = \ self.envutils.get_environment_variable_with_default( 'spark_config', 'local', 'feathr_runtime_location') - self.feathr_spark_launcher = _FeathrDLocalSparkJobLauncher( + self.feathr_spark_launcher = _FeathrLocalSparkJobLauncher( workspace_path = self.envutils.get_environment_variable_with_default('spark_config', 'local', 'workspace'), master = self.envutils.get_environment_variable_with_default('spark_config', 'local', 'master') ) @@ -354,7 +351,7 @@ def _decode_proto(self, feature_list): else: typed_result.append(raw_feature) return typed_result - + def delete_feature_from_redis(self, feature_table, key, feature_name) -> None: """ Delete feature from Redis @@ -364,7 +361,7 @@ def delete_feature_from_redis(self, feature_table, key, feature_name) -> None: key: the key of the entity feature_name: feature name to be deleted """ - + redis_key = self._construct_redis_key(feature_table, key) if self.redis_client.hexists(redis_key, feature_name): self.redis_client.delete(redis_key, feature_name) @@ -575,20 +572,20 @@ def monitor_features(self, settings: MonitoringSettings, execution_configuration def _get_feature_key(self, feature_name: str): features = [] if 'derived_feature_list' in dir(self): - features += self.derived_feature_list + features += self.derived_feature_list if 'anchor_list' in dir(self): for anchor in self.anchor_list: - features += anchor.features + features += anchor.features for feature in features: if feature.name == feature_name: keys = feature.key - return set(key.key_column for key in keys) + return set(key.key_column for key in keys) self.logger.warning(f"Invalid feature name: {feature_name}. Please call FeathrClient.build_features() first in order to materialize the features.") return None - + # Validation on feature keys: # Features within a set of aggregation or planned to be merged should have same keys - # The param "allow_empty_key" shows if empty keys are acceptable + # The param "allow_empty_key" shows if empty keys are acceptable def _valid_materialize_keys(self, features: List[str], allow_empty_key=False): keys = None for feature in features: @@ -611,7 +608,7 @@ def _valid_materialize_keys(self, features: List[str], allow_empty_key=False): self.logger.error(f"Inconsistent feature keys. Current keys are {str(keys)}") return False return True - + def materialize_features(self, settings: MaterializationSettings, execution_configurations: Union[SparkExecutionConfiguration ,Dict[str,str]] = {}, verbose: bool = False): """Materialize feature data @@ -622,7 +619,7 @@ def materialize_features(self, settings: MaterializationSettings, execution_conf feature_list = settings.feature_names if len(feature_list) > 0 and not self._valid_materialize_keys(feature_list): raise RuntimeError(f"Invalid materialization features: {feature_list}, since they have different keys. Currently Feathr only supports materializing features of the same keys.") - + # Collect secrets from sinks secrets = [] for sink in settings.sinks: @@ -632,7 +629,7 @@ def materialize_features(self, settings: MaterializationSettings, execution_conf # produce materialization config for end in settings.get_backfill_cutoff_time(): settings.backfill_time.end = end - config = _to_materialization_config(settings) + config = _to_materialization_config(settings) config_file_name = "feature_gen_conf/auto_gen_config_{}.conf".format(end.timestamp()) config_file_path = os.path.join(self.local_workspace_dir, config_file_name) write_to_file(content=config, full_file_name=config_file_path) @@ -854,7 +851,7 @@ def get_features_from_registry(self, project_name: str) -> Dict[str, FeatureBase feature_dict[feature.name] = feature for feature in registry_derived_feature_list: feature_dict[feature.name] = feature - return feature_dict + return feature_dict def _reshape_config_str(self, config_str:str): if self.spark_runtime == 'local': diff --git a/feathr_project/feathr/spark_provider/_abc.py b/feathr_project/feathr/spark_provider/_abc.py index 2644f82fe..c91fdf5c1 100644 --- a/feathr_project/feathr/spark_provider/_abc.py +++ b/feathr_project/feathr/spark_provider/_abc.py @@ -1,6 +1,6 @@ from abc import ABC, abstractmethod +from typing import Dict, List, Optional, Tuple -from typing import Any, Dict, List, Optional, Tuple class SparkJobLauncher(ABC): """This is the abstract class for all the spark launchers. All the Spark launcher should implement those interfaces @@ -15,7 +15,6 @@ def upload_or_get_cloud_path(self, local_path_or_http_path: str): """ pass - @abstractmethod def submit_feathr_job(self, job_name: str, main_jar_path: str, main_class_name: str, arguments: List[str], reference_files_path: List[str], job_tags: Dict[str, str] = None, @@ -33,6 +32,7 @@ def submit_feathr_job(self, job_name: str, main_jar_path: str, main_class_name: properties (Dict[str, str]): Additional System Properties for the spark job """ pass + @abstractmethod def wait_for_completion(self, timeout_seconds: Optional[float]) -> bool: """Returns true if the job completed successfully diff --git a/feathr_project/feathr/spark_provider/_localspark_submission.py b/feathr_project/feathr/spark_provider/_localspark_submission.py index 3b24fd513..afed9683d 100644 --- a/feathr_project/feathr/spark_provider/_localspark_submission.py +++ b/feathr_project/feathr/spark_provider/_localspark_submission.py @@ -1,129 +1,125 @@ -import time from datetime import datetime import json import os from pathlib import Path -from typing import Dict, List, Optional +from shlex import split +from subprocess import STDOUT, Popen +import time +from typing import Any, Dict, List, Optional -from feathr.spark_provider._abc import SparkJobLauncher from loguru import logger - from pyspark import * -from subprocess import TimeoutExpired, STDOUT, Popen -from shlex import split from feathr.constants import FEATHR_MAVEN_ARTIFACT +from feathr.spark_provider._abc import SparkJobLauncher +class _FeathrLocalSparkJobLauncher(SparkJobLauncher): + """Class to interact with local Spark. This class is not intended to be used in Production environments. + It is intended to be used for testing and development purposes. No authentication is required to use this class. -class _FeathrDLocalSparkJobLauncher(SparkJobLauncher): - """Class to interact with local Spark - This class is not intended to be used in Production environments. - It is intended to be used for testing and development purposes. - No authentication is required to use this class. - Args: - workspace_path (str): Path to the workspace + Args: + workspace_path (str): Path to the workspace """ + def __init__( self, workspace_path: str, master: str = None, - debug_folder:str = "debug", - clean_up:bool = True, - retry:int = 3, - retry_sec:int = 5, + debug_folder: str = "debug", + clean_up: bool = True, + retry: int = 3, + retry_sec: int = 5, ): - """Initialize the Local Spark job launcher - """ - self.workspace_path = workspace_path, + """Initialize the Local Spark job launcher""" + self.workspace_path = (workspace_path,) self.debug_folder = debug_folder self.spark_job_num = 0 self.clean_up = clean_up self.retry = retry self.retry_sec = retry_sec self.packages = self._get_default_package() - self.master = master + self.master = master or "local[*]" def upload_or_get_cloud_path(self, local_path_or_http_path: str): """For Local Spark Case, no need to upload to cloud workspace.""" return local_path_or_http_path - def submit_feathr_job(self, job_name: str, main_jar_path: str = None, main_class_name: str = None, arguments: List[str] = None, - python_files: List[str]= None, configuration: Dict[str, str] = {}, properties: Dict[str, str] = {}, reference_files_path: List[str] = None, job_tags: Dict[str, str] = None): - """ - Submits the Feathr job to local spark, using subprocess args. - - reference files: put everything there and the function will automatically categorize them based on the - extension name to either the "files" argument in the Livy API, or the "jars" argument in the Livy API. The - path can be local path and this function will automatically upload the function to the corresponding azure - storage - - Also, note that the Spark application will automatically run on YARN cluster mode. You cannot change it if + def submit_feathr_job( + self, + job_name: str, + main_jar_path: str, + main_class_name: str, + arguments: List[str] = None, + python_files: List[str] = None, + configuration: Dict[str, str] = {}, + properties: Dict[str, str] = {}, + **_, + ) -> Any: + """Submits the Feathr job to local spark, using subprocess args. + Note that the Spark application will automatically run on YARN cluster mode. You cannot change it if you are running with Azure Synapse. Args: - job_name (str): name of the job - main_jar_path (str): main file paths, usually your main jar file - main_class_name (str): name of your main class - arguments (str): all the arguments you want to pass into the spark job - configuration (Dict[str, str]): Additional configs for the spark job - python_files (List[str]): required .zip, .egg, or .py files of spark job - properties (Dict[str, str]): Additional System Properties for the spark job - job_tags (str): not used in local spark mode - reference_files_path (str): not used in local spark mode + job_name: name of the job + main_jar_path: main file paths, usually your main jar file + main_class_name: name of your main class + arguments: all the arguments you want to pass into the spark job + python_files: required .zip, .egg, or .py files of spark job + configuration: Additional configs for the spark job + properties: System properties configuration + **_: Not used arguments in local spark mode, such as reference_files_path and job_tags """ - logger.warning(f"Local Spark Mode only support basic params right now and should be used only for testing purpose.") - self.cmd_file, self.log_path = self._get_debug_file_name(self.debug_folder, prefix = job_name) - args = self._init_args(master = self.master, job_name=job_name) + logger.warning( + f"Local Spark Mode only support basic params right now and should be used only for testing purpose." + ) + self.cmd_file, self.log_path = self._get_debug_file_name(self.debug_folder, prefix=job_name) - if properties: - arguments.extend(["--system-properties", json.dumps(properties)]) + # Get conf and package arguments + cfg = configuration.copy() if configuration else {} + maven_dependency = f"{cfg.pop('spark.jars.packages', self.packages)},{FEATHR_MAVEN_ARTIFACT}" + spark_args = self._init_args(job_name=job_name, confs=cfg) - if configuration: - cfg = configuration.copy() # We don't want to mess up input parameters - else: - cfg = {} - if not main_jar_path: # We don't have the main jar, use Maven - # Add Maven dependency to the job configuration - if "spark.jars.packages" in cfg: - cfg["spark.jars.packages"] = ",".join( - [cfg["spark.jars.packages"], FEATHR_MAVEN_ARTIFACT]) - else: - cfg["spark.jars.packages"] = ",".join([self.packages, FEATHR_MAVEN_ARTIFACT]) - if not python_files: # This is a JAR job # Azure Synapse/Livy doesn't allow JAR job starts from Maven directly, we must have a jar file uploaded. # so we have to use a dummy jar as the main file. logger.info(f"Main JAR file is not set, using default package '{FEATHR_MAVEN_ARTIFACT}' from Maven") # Use the no-op jar as the main file - # This is a dummy jar which contains only one `org.example.Noop` class with one empty `main` function which does nothing + # This is a dummy jar which contains only one `org.example.Noop` class with one empty `main` function + # which does nothing current_dir = Path(__file__).parent.resolve() main_jar_path = os.path.join(current_dir, "noop-1.0.jar") - args.extend(["--packages", cfg["spark.jars.packages"],"--class", main_class_name, main_jar_path]) + spark_args.extend(["--packages", maven_dependency, "--class", main_class_name, main_jar_path]) else: - args.extend(["--packages", cfg["spark.jars.packages"]]) - # This is a PySpark job, no more things to + spark_args.extend(["--packages", maven_dependency]) + # This is a PySpark job, no more things to if python_files.__len__() > 1: - args.extend(["--py-files", ",".join(python_files[1:])]) + spark_args.extend(["--py-files", ",".join(python_files[1:])]) print(python_files) - args.append(python_files[0]) + spark_args.append(python_files[0]) else: - args.extend(["--class", main_class_name, main_jar_path]) + spark_args.extend(["--class", main_class_name, main_jar_path]) + + if arguments: + spark_args.extend(arguments) - cmd = " ".join(args) + " " + " ".join(arguments) + if properties: + spark_args.extend(["--system-properties", json.dumps(properties)]) + + cmd = " ".join(spark_args) - log_append = open(f"{self.log_path}_{self.spark_job_num}.txt" , "a") + log_append = open(f"{self.log_path}_{self.spark_job_num}.txt", "a") proc = Popen(split(cmd), shell=False, stdout=log_append, stderr=STDOUT) logger.info(f"Detail job stdout and stderr are in {self.log_path}.") self.spark_job_num += 1 with open(self.cmd_file, "a") as c: - c.write(" ".join(proc.args)) - c.write("\n") + c.write(" ".join(proc.args)) + c.write("\n") self.latest_spark_proc = proc @@ -132,9 +128,8 @@ def submit_feathr_job(self, job_name: str, main_jar_path: str = None, main_clas return proc def wait_for_completion(self, timeout_seconds: Optional[float] = 500) -> bool: - """ - this function track local spark job commands and process status. - files will be write into `debug` folder under your workspace. + """This function track local spark job commands and process status. + Files will be write into `debug` folder under your workspace. """ logger.info(f"{self.spark_job_num} local spark job(s) in this Launcher, only the latest will be monitored.") logger.info(f"Please check auto generated spark command in {self.cmd_file} and detail logs in {self.log_path}.") @@ -143,12 +138,15 @@ def wait_for_completion(self, timeout_seconds: Optional[float] = 500) -> bool: start_time = time.time() retry = self.retry - log_read = open(f"{self.log_path}_{self.spark_job_num-1}.txt" , "r") + log_read = open(f"{self.log_path}_{self.spark_job_num-1}.txt", "r") while proc.poll() is None and (((timeout_seconds is None) or (time.time() - start_time < timeout_seconds))): time.sleep(1) try: if retry < 1: - logger.warning(f"Spark job has hang for {self.retry * self.retry_sec} seconds. latest msg is {last_line}. please check {log_read.name}") + logger.warning( + f"Spark job has hang for {self.retry * self.retry_sec} seconds. latest msg is {last_line}. \ + Please check {log_read.name}" + ) if self.clean_up: self._clean_up() proc.wait() @@ -168,22 +166,28 @@ def wait_for_completion(self, timeout_seconds: Optional[float] = 500) -> bool: retry -= 1 job_duration = time.time() - start_time - log_read.close() + log_read.close() if proc.returncode == None: - logger.warning(f"Spark job with pid {self.latest_spark_proc.pid} not completed after {timeout_seconds} sec time out setting, please check.") + logger.warning( + f"Spark job with pid {self.latest_spark_proc.pid} not completed after {timeout_seconds} sec \ + time out setting. Please check." + ) if self.clean_up: self._clean_up() proc.wait() return True elif proc.returncode == 1: - logger.warning(f"Spark job with pid {self.latest_spark_proc.pid} is not successful, please check.") + logger.warning(f"Spark job with pid {self.latest_spark_proc.pid} is not successful. Please check.") return False else: - logger.info(f"Spark job with pid {self.latest_spark_proc.pid} finished in: {int(job_duration)} seconds with returncode {proc.returncode}") + logger.info( + f"Spark job with pid {self.latest_spark_proc.pid} finished in: {int(job_duration)} seconds \ + with returncode {proc.returncode}" + ) return True - def _clean_up(self, proc:Popen = None): + def _clean_up(self, proc: Popen = None): logger.warning(f"Terminate the spark job due to as clean_up is set to True.") if not proc: self.latest_spark_proc.terminate() @@ -194,30 +198,35 @@ def get_status(self) -> str: """Get the status of the job, only a placeholder for local spark""" return self.latest_spark_proc.returncode - def _init_args(self, master:str, job_name:str): - if master is None: - master = "local[*]" - logger.info(f"Spark job: {job_name} is running on local spark with master: {master}.") + def _init_args(self, job_name: str, confs: Dict[str, str]) -> List[str]: + logger.info(f"Spark job: {job_name} is running on local spark with master: {self.master}.") args = [ "spark-submit", - "--master",master, - "--name",job_name, - "--conf", "spark.hadoop.fs.wasbs.impl=org.apache.hadoop.fs.azure.NativeAzureFileSystem", - "--conf", "spark.hadoop.fs.wasbs=org.apache.hadoop.fs.azure.NativeAzureFileSystem", + "--master", + self.master, + "--name", + job_name, + "--conf", + "spark.hadoop.fs.wasbs.impl=org.apache.hadoop.fs.azure.NativeAzureFileSystem", + "--conf", + "spark.hadoop.fs.wasbs=org.apache.hadoop.fs.azure.NativeAzureFileSystem", ] + + for k, v in confs.items(): + args.extend(["--conf", f"{k}={v}"]) + return args - def _get_debug_file_name(self, debug_folder: str = "debug", prefix:str = None): - """ - auto generated command will be write into cmd file - spark job output will be write into log path with job number as suffix + def _get_debug_file_name(self, debug_folder: str = "debug", prefix: str = None): + """Auto generated command will be write into cmd file. + Spark job output will be write into log path with job number as suffix. """ prefix += datetime.now().strftime("%Y%m%d%H%M%S") debug_path = os.path.join(debug_folder, prefix) print(debug_path) if not os.path.exists(debug_path): - os.makedirs(debug_path) + os.makedirs(debug_path) cmd_file = os.path.join(debug_path, f"command.sh") log_path = os.path.join(debug_path, f"log") @@ -227,7 +236,7 @@ def _get_debug_file_name(self, debug_folder: str = "debug", prefix:str = None): def _get_default_package(self): # default packages of Feathr Core, requires manual update when new dependency introduced or package updated. # TODO: automate this process, e.g. read from pom.xml - # TODO: dynamical modularization: add package only when it's used in the job, e.g. data source dependencies. + # TODO: dynamical modularization: add package only when it's used in the job, e.g. data source dependencies. packages = [] packages.append("org.apache.spark:spark-avro_2.12:3.3.0") packages.append("com.microsoft.sqlserver:mssql-jdbc:10.2.0.jre8") @@ -236,7 +245,7 @@ def _get_default_package(self): packages.append("com.fasterxml.jackson.core:jackson-databind:2.12.6.1") packages.append("org.apache.hadoop:hadoop-mapreduce-client-core:2.7.7") packages.append("org.apache.hadoop:hadoop-common:2.7.7") - packages.append("org.apache.hadoop:hadoop-azure:3.2.0") + packages.append("org.apache.hadoop:hadoop-azure:3.2.0") packages.append("org.apache.avro:avro:1.8.2,org.apache.xbean:xbean-asm6-shaded:4.10") packages.append("org.apache.spark:spark-sql-kafka-0-10_2.12:3.1.3") packages.append("com.microsoft.azure:azure-eventhubs-spark_2.12:2.3.21") diff --git a/feathr_project/pyproject.toml b/feathr_project/pyproject.toml index f8d897579..693233dc2 100644 --- a/feathr_project/pyproject.toml +++ b/feathr_project/pyproject.toml @@ -1,6 +1,17 @@ +[tool.black] +line-length = 120 +target_version = ['py38'] + +[tool.isort] +profile = "black" +line_length = 120 +known_first_party = ['feathr'] +force_sort_within_sections = true +multi_line_output = 3 + [build-system] requires = [ "setuptools", "wheel" ] -build-backend = "setuptools.build_meta" \ No newline at end of file +build-backend = "setuptools.build_meta" diff --git a/feathr_project/setup.py b/feathr_project/setup.py index e937f19c4..ce7ec14d6 100644 --- a/feathr_project/setup.py +++ b/feathr_project/setup.py @@ -20,7 +20,7 @@ include_package_data=True, # consider install_requires=[ - 'click<=8.1.3', + "click<=8.1.3", "py4j<=0.10.9.7", "loguru<=0.6.0", "pandas<=1.5.0", @@ -54,9 +54,17 @@ "azure-core<=1.22.1", "typing_extensions>=4.2.0" ], - tests_require=[ - 'pytest', + tests_require=[ # TODO: This has been depricated + "pytest", ], + extras_require=dict( + dev=[ + "black>=22.1.0", # formatter + "isort", # sort import statements + "pytest>=7", + "pytest-mock>=3.8.1", + ], + ), entry_points={ 'console_scripts': ['feathr=feathrcli.cli:cli'] }, diff --git a/feathr_project/test/unit/spark_provider/test_localspark_submission.py b/feathr_project/test/unit/spark_provider/test_localspark_submission.py new file mode 100644 index 000000000..9a9d7238b --- /dev/null +++ b/feathr_project/test/unit/spark_provider/test_localspark_submission.py @@ -0,0 +1,51 @@ +from typing import Dict +from unittest.mock import MagicMock + +import pytest +from pytest_mock import MockerFixture + +from feathr.spark_provider._localspark_submission import _FeathrLocalSparkJobLauncher + + +@pytest.fixture(scope="function") +def local_spark_job_launcher(tmp_path) -> _FeathrLocalSparkJobLauncher: + return _FeathrLocalSparkJobLauncher( + workspace_path=str(tmp_path), + debug_folder=str(tmp_path), + ) + + +def test__local_spark_job_launcher__submit_feathr_job( + mocker: MockerFixture, + local_spark_job_launcher: _FeathrLocalSparkJobLauncher, +): + # Mock necessary components + local_spark_job_launcher._init_args = MagicMock(return_value=[]) + mocked_proc = MagicMock() + mocked_proc.args = [] + mocked_proc.pid = 0 + + mocked_spark_proc = mocker.patch("feathr.spark_provider._localspark_submission.Popen", return_value=mocked_proc) + + local_spark_job_launcher.submit_feathr_job( + job_name="unit-test", + main_jar_path="", + main_class_name="", + ) + + # Assert if the mocked spark process has called once + mocked_spark_proc.assert_called_once() + + +@pytest.mark.parametrize( + "confs", [{}, {"spark.feathr.outputFormat": "parquet"}] +) +def test__local_spark_job_launcher__init_args( + local_spark_job_launcher: _FeathrLocalSparkJobLauncher, + confs: Dict[str, str], +): + spark_args = local_spark_job_launcher._init_args(job_name=None, confs=confs) + + # Assert if spark_args contains confs at the end + for k, v in confs.items(): + assert spark_args[-1] == f"{k}={v}" From d771c3c0574dc6d04a3dc969c0e93ba5283206c4 Mon Sep 17 00:00:00 2001 From: Yihui Guo Date: Thu, 13 Oct 2022 22:37:33 +0800 Subject: [PATCH 14/68] Fix direct purview client missing transformation (#736) --- .../feathr/registry/_feature_registry_purview.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/feathr_project/feathr/registry/_feature_registry_purview.py b/feathr_project/feathr/registry/_feature_registry_purview.py index 4ef05a690..c564e8796 100644 --- a/feathr_project/feathr/registry/_feature_registry_purview.py +++ b/feathr_project/feathr/registry/_feature_registry_purview.py @@ -1368,11 +1368,11 @@ def _get_transformation_from_dict(self, input: Dict) -> FeatureType: if 'transformExpr' in input: # it's ExpressionTransformation return ExpressionTransformation(input['transformExpr']) - elif 'def_expr' in input: - agg_expr=input['def_expr'] if 'def_expr' in input else None - agg_func=input['agg_func']if 'agg_func' in input else None + elif 'def_expr' in input or 'defExpr' in input: + agg_expr=input['def_expr'] if 'def_expr' in input else (input['defExpr'] if 'defExpr' in input else None) + agg_func=input['agg_func']if 'agg_func' in input else (input['aggFunc'] if 'aggFunc' in input else None) window=input['window']if 'window' in input else None - group_by=input['group_by']if 'group_by' in input else None + group_by=input['group_by']if 'group_by' in input else (input['groupBy'] if 'groupBy' in input else None) filter=input['filter']if 'filter' in input else None limit=input['limit']if 'limit' in input else None return WindowAggTransformation(agg_expr, agg_func, window, group_by, filter, limit) From f677a174353530dbd15835d00fe3671d1858bf77 Mon Sep 17 00:00:00 2001 From: Jinghui Mo Date: Thu, 13 Oct 2022 12:33:22 -0400 Subject: [PATCH 15/68] Revert "Derived feature bugfix (#121)" (#731) This reverts commit fa645f386002211286b5bfe3f9c72ba987be71cc. --- .../feathr/definition/feature_derivations.py | 2 +- feathr_project/test/test_derived_features.py | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/feathr_project/feathr/definition/feature_derivations.py b/feathr_project/feathr/definition/feature_derivations.py index 84583654f..9205685ce 100644 --- a/feathr_project/feathr/definition/feature_derivations.py +++ b/feathr_project/feathr/definition/feature_derivations.py @@ -58,7 +58,7 @@ def to_feature_config(self) -> str: } {% endfor %} } - definition: {{derived_feature.transform.to_feature_config(False)}} + definition.sqlExpr: {{derived_feature.transform.to_feature_config(False)}} {{derived_feature.feature_type.to_feature_config()}} } """) diff --git a/feathr_project/test/test_derived_features.py b/feathr_project/test/test_derived_features.py index f879553d7..ee10cd285 100644 --- a/feathr_project/test/test_derived_features.py +++ b/feathr_project/test/test_derived_features.py @@ -26,7 +26,7 @@ def test_single_key_derived_feature_to_config(): inputs: { user_embedding: {key: [user_id], feature: user_embedding} } - definition: "if_else(user_embedding, user_embedding, [])" + definition.sqlExpr: "if_else(user_embedding, user_embedding, [])" type: { type: TENSOR tensorCategory: DENSE @@ -58,7 +58,7 @@ def test_multikey_derived_feature_to_config(): user_embedding: {key: [user_id], feature: user_embedding} item_embedding: {key: [item_id], feature: item_embedding} } - definition: "similarity(user_embedding,item_embedding)" + definition.sqlExpr: "similarity(user_embedding,item_embedding)" type: { type: TENSOR tensorCategory: DENSE @@ -88,7 +88,7 @@ def test_derived_feature_to_config_with_alias(): viewer_embedding: {key: [viewer], feature: user_embedding} viewee_embedding: {key: [viewee], feature: user_embedding} } - definition: "distance(viewer_embedding, viewee_embedding)" + definition.sqlExpr: "distance(viewer_embedding, viewee_embedding)" type: { type: TENSOR tensorCategory: DENSE @@ -129,7 +129,7 @@ def test_multi_key_derived_feature_to_config_with_alias(): viewer_viewee_distance: {key: [viewer, viewee], feature: viewer_viewee_distance} viewee_viewer_distance: {key: [viewee, viewer], feature: viewer_viewee_distance} } - definition: "viewer_viewee_distance + viewee_viewer_distance" + definition.sqlExpr: "viewer_viewee_distance + viewee_viewer_distance" type: { type: TENSOR tensorCategory: DENSE @@ -159,7 +159,7 @@ def test_derived_feature_on_multikey_anchored_feature_to_config(): inputs: { user_embedding: {key: [viewer, viewee], feature: user_embedding} } - definition: "if_else(user_embedding, user_embedding, [])" + definition.sqlExpr: "if_else(user_embedding, user_embedding, [])" type: { type: TENSOR tensorCategory: DENSE From 616d76e96ed24e45e3059de883d6aee56ff1ea20 Mon Sep 17 00:00:00 2001 From: Jinghui Mo Date: Thu, 13 Oct 2022 16:34:52 -0400 Subject: [PATCH 16/68] Support SWA with groupBy to 1d tensor conversion (#748) * Support SWA with groupby to 1d tensor conversion --- .../transformation/FDSConversionUtils.scala | 11 ++++++++--- .../offline/util/TestFDSConversionUtil.scala | 17 ++++++++++++----- 2 files changed, 20 insertions(+), 8 deletions(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/transformation/FDSConversionUtils.scala b/src/main/scala/com/linkedin/feathr/offline/transformation/FDSConversionUtils.scala index 96a10a67c..25d96af11 100644 --- a/src/main/scala/com/linkedin/feathr/offline/transformation/FDSConversionUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/transformation/FDSConversionUtils.scala @@ -2,14 +2,13 @@ package com.linkedin.feathr.offline.transformation import com.linkedin.feathr.common.exception.{ErrorLabel, FeathrException} import com.linkedin.feathr.common.tensor.TensorData - -import java.util import com.linkedin.feathr.common.util.CoercionUtils import com.linkedin.feathr.offline.util.FeaturizedDatasetUtils import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.types._ +import java.util import scala.collection.JavaConverters._ import scala.collection.convert.Wrappers.JMapWrapper import scala.collection.mutable @@ -253,7 +252,13 @@ private[offline] object FDSConversionUtils { case values: util.ArrayList[Any] => values.asScala.toArray case values: mutable.WrappedArray[Any] => - values.asInstanceOf[mutable.WrappedArray[Any]].toArray + if (values.nonEmpty && values(0).isInstanceOf[GenericRowWithSchema]) { + // Assuming the result is returned by SWA feature with groupBy, hence keeping only the + // feature value as an array and dropping the index info. + values.asInstanceOf[mutable.WrappedArray[GenericRowWithSchema]].map(v => v.get(v.size - 1)).toArray + } else { + values.toArray + } case values: List[Any] => values.toArray case mapValues: Map[Integer, Any] => diff --git a/src/test/scala/com/linkedin/feathr/offline/util/TestFDSConversionUtil.scala b/src/test/scala/com/linkedin/feathr/offline/util/TestFDSConversionUtil.scala index 3ab94e616..f3b75024e 100644 --- a/src/test/scala/com/linkedin/feathr/offline/util/TestFDSConversionUtil.scala +++ b/src/test/scala/com/linkedin/feathr/offline/util/TestFDSConversionUtil.scala @@ -3,18 +3,17 @@ package com.linkedin.feathr.offline.util import com.linkedin.feathr.common.TensorUtils import com.linkedin.feathr.common.tensor.{TensorType, Tensors} import com.linkedin.feathr.common.types.PrimitiveType - -import java.util -import java.util.Collections import com.linkedin.feathr.offline.AssertFeatureUtils import com.linkedin.feathr.offline.transformation.FDSConversionUtils import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.catalyst.expressions.{GenericRow, GenericRowWithSchema} import org.apache.spark.sql.types._ import org.scalatest.testng.TestNGSuite import org.testng.Assert.{assertEquals, assertTrue} import org.testng.annotations.{DataProvider, Test} +import java.util +import java.util.Collections import scala.collection.mutable class TestFDSConversionUtil extends TestNGSuite { @@ -141,10 +140,18 @@ class TestFDSConversionUtil extends TestNGSuite { @DataProvider def dataForTestConvertRawValueTo1DFDSDenseTensorRowTz(): Array[Array[Any]] = { + val eleType = StructType( + StructField("group", IntegerType, false) :: + StructField("value", IntegerType, false) :: Nil + ) + val row1 = new GenericRowWithSchema(Array(1, 3), eleType) + val row2 = new GenericRowWithSchema(Array(2, 4), eleType) Array( Array(mutable.WrappedArray.make(Array(2.0f, 6.0f)), util.Arrays.asList(2.0f, 6.0f).toArray), Array(Array(1.1).toList, util.Arrays.asList(1.1).toArray), - Array(Map("a" -> 1.1), util.Arrays.asList(1.1).toArray) + Array(Map("a" -> 1.1), util.Arrays.asList(1.1).toArray), + // Simulate raw value return by SWA feature with groupBy + Array(mutable.WrappedArray.make(Array(row1, row2)), util.Arrays.asList(3, 4).toArray) ) } @Test(dataProvider = "dataForTestConvertRawValueTo1DFDSDenseTensorRowTz") From 8d7d41269a376f890917d2c1cd7ac86bfdc87102 Mon Sep 17 00:00:00 2001 From: Richin Jain Date: Fri, 14 Oct 2022 15:34:45 -0700 Subject: [PATCH 17/68] Rijai/armfix (#742) * Adding DevSkim linter to Github actions * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Making ARM instructions for Owner role and AAD App more clear * Removing devskim file * Reverting the changes to docker file to match with feathr/main --- docs/how-to-guides/azure-deployment-arm.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/how-to-guides/azure-deployment-arm.md b/docs/how-to-guides/azure-deployment-arm.md index 0d833abf0..7bc9a926f 100644 --- a/docs/how-to-guides/azure-deployment-arm.md +++ b/docs/how-to-guides/azure-deployment-arm.md @@ -6,7 +6,7 @@ parent: How-to Guides # Azure Resource Provisioning -The provided Azure Resource Manager (ARM) template deploys the following resources, please make sure you have enough quota in the subscription and region you are deploying this in. You can view your quota and make request on Azure [portal](https://ms.portal.azure.com/#view/Microsoft_Azure_Capacity/QuotaMenuBlade/~/overview) +The provided Azure Resource Manager (ARM) template deploys the following resources, please make sure you have enough quota in the subscription and region you are deploying this in. You can view your quota and make request on Azure [portal](https://ms.portal.azure.com/#view/Microsoft_Azure_Capacity/QuotaMenuBlade/~/overview). 1. Azure Storage account 2. Azure Purview (metadata store if you selected Azure-Purview as registry backend) @@ -17,9 +17,9 @@ The provided Azure Resource Manager (ARM) template deploys the following resourc 7. Azure Event Hub 8. Azure Redis -Please note, you need to have **owner access** in the resource group you are deploying this in. Owner access is required to assign role to managed identity within ARM template so it can access key vault and store secrets. If you don't have such permission, you might want to contact your IT admin to see if they can do that. +### Please Note: you need to have the **Owner Role** in the resource group you are deploying this in. Owner access is required to assign role to managed identity within the ARM template so it can access key vault and store secrets. It is also required by the permission section in our sample notebooks. If you don't have such permission, you might want to contact your IT admin to see if they can do that. -Although we recommend end users deploy the resources using the ARM template, we understand that in many situations where users want to reuse existing resources instead of creating new resources; or users have many other permission issues. See [Manually connecting existing resources](#manually-connecting-existing-resources) for more details. +Although we recommend end users deploy the resources using the ARM template, we understand that in many situations where users want to reuse existing resources instead of creating new resources; or users may have permission issues. See [Manually connecting existing resources](#manually-connecting-existing-resources) for more details. ## Architecture @@ -34,10 +34,12 @@ Feathr has native cloud integration and getting started with Feathr is very stra The very first step is to create an Azure Active Directory (AAD) application to enable authentication on the Feathr UI (which gets created as part of the deployment script). Currently it is not possible to create one through ARM template but you can easily create one by running the following CLI commands in the [Cloud Shell](https://shell.azure.com/bash). +### Please make note of the Client ID and Tenant ID for the AAD app, you will need it in the ARM template deployment section. + ```bash # This is the prefix you want to name your resources with, make a note of it, you will need it during deployment. # Note: please keep the `resourcePrefix` short (less than 15 chars), since some of the Azure resources need the full name to be less than 24 characters. Only lowercase alphanumeric characters are allowed for resource prefix. -resource_prefix="userprefix1" +resource_prefix="yourprefix" # Please don't change this name, a corresponding webapp with same name gets created in subsequent steps. sitename="${resource_prefix}webapp" From 4bdcd574fde47ff3072a83b507d92a3fe612de49 Mon Sep 17 00:00:00 2001 From: Yuqing Wei Date: Sat, 15 Oct 2022 06:52:43 +0800 Subject: [PATCH 18/68] bump version to 0.8.2 (#722) Signed-off-by: Yuqing Wei Signed-off-by: Yuqing Wei --- build.sbt | 2 +- docs/how-to-guides/azure_resource_provision.json | 2 +- feathr_project/feathr/constants.py | 2 +- feathr_project/setup.py | 2 +- feathr_project/test/test_user_workspace/feathr_config.yaml | 4 ++-- .../test_user_workspace/feathr_config_registry_purview.yaml | 4 ++-- .../feathr_config_registry_purview_rbac.yaml | 4 ++-- .../test/test_user_workspace/feathr_config_registry_sql.yaml | 4 ++-- .../test_user_workspace/feathr_config_registry_sql_rbac.yaml | 4 ++-- 9 files changed, 14 insertions(+), 14 deletions(-) diff --git a/build.sbt b/build.sbt index 2919ddae6..d85f99016 100644 --- a/build.sbt +++ b/build.sbt @@ -2,7 +2,7 @@ import sbt.Keys.publishLocalConfiguration ThisBuild / resolvers += Resolver.mavenLocal ThisBuild / scalaVersion := "2.12.15" -ThisBuild / version := "0.8.0" +ThisBuild / version := "0.8.2" ThisBuild / organization := "com.linkedin.feathr" ThisBuild / organizationName := "linkedin" val sparkVersion = "3.1.3" diff --git a/docs/how-to-guides/azure_resource_provision.json b/docs/how-to-guides/azure_resource_provision.json index 03d175052..6ab77a419 100644 --- a/docs/how-to-guides/azure_resource_provision.json +++ b/docs/how-to-guides/azure_resource_provision.json @@ -111,7 +111,7 @@ "destinationBacpacBlobUrl": "[concat('https://',variables('dlsName'),'.blob.core.windows.net/',variables('dlsFsName'),'/',variables('bacpacBlobName'))]", "bacpacDeploymentScriptName": "CopyBacpacFile", "bacpacDbExtensionName": "registryRbacDbImport", - "preBuiltdockerImage": "feathrfeaturestore/feathr-registry:releases-v0.8.0" + "preBuiltdockerImage": "feathrfeaturestore/feathr-registry:releases-v0.8.2" }, "functions": [], "resources": [ diff --git a/feathr_project/feathr/constants.py b/feathr_project/feathr/constants.py index 6686f14ac..c4cbad7ff 100644 --- a/feathr_project/feathr/constants.py +++ b/feathr_project/feathr/constants.py @@ -28,7 +28,7 @@ TYPEDEF_ARRAY_DERIVED_FEATURE=f"array" TYPEDEF_ARRAY_ANCHOR_FEATURE=f"array" -FEATHR_MAVEN_ARTIFACT="com.linkedin.feathr:feathr_2.12:0.8.0" +FEATHR_MAVEN_ARTIFACT="com.linkedin.feathr:feathr_2.12:0.8.2" JOIN_CLASS_NAME="com.linkedin.feathr.offline.job.FeatureJoinJob" GEN_CLASS_NAME="com.linkedin.feathr.offline.job.FeatureGenJob" \ No newline at end of file diff --git a/feathr_project/setup.py b/feathr_project/setup.py index ce7ec14d6..7c3e10178 100644 --- a/feathr_project/setup.py +++ b/feathr_project/setup.py @@ -7,7 +7,7 @@ setup( name='feathr', - version='0.8.0', + version='0.8.2', long_description=long_description, long_description_content_type="text/markdown", author_email="feathr-technical-discuss@lists.lfaidata.foundation", diff --git a/feathr_project/test/test_user_workspace/feathr_config.yaml b/feathr_project/test/test_user_workspace/feathr_config.yaml index e67c803ef..b0f2b259c 100644 --- a/feathr_project/test/test_user_workspace/feathr_config.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config.yaml @@ -82,7 +82,7 @@ spark_config: # Feathr Job configuration. Support local paths, path start with http(s)://, and paths start with abfs(s):// # this is the default location so end users don't have to compile the runtime again. # feathr_runtime_location: wasbs://public@azurefeathrstorage.blob.core.windows.net/feathr-assembly-LATEST.jar - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" databricks: # workspace instance workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' @@ -93,7 +93,7 @@ spark_config: # Feathr Job location. Support local paths, path start with http(s)://, and paths start with dbfs:/ work_dir: 'dbfs:/feathr_getting_started' # this is the default location so end users don't have to compile the runtime again. - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" online_store: redis: diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml index f716da0b4..003e424eb 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml @@ -25,13 +25,13 @@ spark_config: workspace_dir: 'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_test_workspace' executor_size: 'Small' executor_num: 1 - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" online_store: redis: diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml index c842bc702..0722a34d8 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml @@ -25,13 +25,13 @@ spark_config: workspace_dir: 'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_test_workspace' executor_size: 'Small' executor_num: 1 - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" online_store: redis: diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml index dcb73d827..507b096cb 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml @@ -25,13 +25,13 @@ spark_config: workspace_dir: 'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_test_workspace' executor_size: 'Small' executor_num: 1 - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" online_store: redis: diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml index 29c6889e8..9403cb190 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml @@ -25,13 +25,13 @@ spark_config: workspace_dir: 'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_test_workspace' executor_size: 'Small' executor_num: 1 - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" online_store: redis: From 3c407c3aec9f3f435f177854a664b49de5d167ff Mon Sep 17 00:00:00 2001 From: Chang Yong Lik <51813538+ahlag@users.noreply.github.com> Date: Sat, 15 Oct 2022 08:17:42 +0900 Subject: [PATCH 19/68] Added latest deltalake version (#735) * Added latest deltalake version * Changed == to <= for deltalake installation * Changed <= to >= --- feathr_project/setup.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/feathr_project/setup.py b/feathr_project/setup.py index 7c3e10178..47d105061 100644 --- a/feathr_project/setup.py +++ b/feathr_project/setup.py @@ -35,8 +35,7 @@ "pyarrow<=9.0.0", "pyspark>=3.1.2", "python-snappy<=0.6.1", - # fixing https://github.com/feathr-ai/feathr/issues/687 - "deltalake<=0.5.8", + "deltalake>=0.6.2", "graphlib_backport<=1.0.3", "protobuf==3.*", "confluent-kafka<=1.9.2", From 1465f6472094da5e710c832c3d4691e694644d50 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BE=90=E8=BE=B0?= Date: Sat, 15 Oct 2022 13:15:57 +0800 Subject: [PATCH 20/68] #474 Disable local mode (#738) --- .../com/linkedin/feathr/offline/util/SourceUtils.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala index e9d3a2bf1..a70c11fd0 100644 --- a/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala +++ b/src/main/scala/com/linkedin/feathr/offline/util/SourceUtils.scala @@ -652,11 +652,7 @@ private[offline] object SourceUtils { ss.read.format("csv").option("header", "true").option("delimiter", csvDelimiterOption).load(inputData.inputPath) } case _ => { - if (ss.sparkContext.isLocal){ - getLocalDF(ss, inputData.inputPath, dataLoaderHandlers) - } else { - loadAsDataFrame(ss, SimplePath(inputData.inputPath),dataLoaderHandlers) - } + loadAsDataFrame(ss, SimplePath(inputData.inputPath),dataLoaderHandlers) } } } From d59ea4bdad6606683470883e686a56262c5f10b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BE=90=E8=BE=B0?= Date: Sat, 15 Oct 2022 13:40:58 +0800 Subject: [PATCH 21/68] Allow recreating entities for PurView registry (#691) * Allow recreating entities for PurView registry * Use constants --- .../registry/_feature_registry_purview.py | 46 ++++++++++++++++ .../registry/purview_registry.py | 54 +++++++++++++++---- 2 files changed, 91 insertions(+), 9 deletions(-) diff --git a/feathr_project/feathr/registry/_feature_registry_purview.py b/feathr_project/feathr/registry/_feature_registry_purview.py index c564e8796..a6b2c17e4 100644 --- a/feathr_project/feathr/registry/_feature_registry_purview.py +++ b/feathr_project/feathr/registry/_feature_registry_purview.py @@ -3,6 +3,7 @@ import inspect import itertools import os +import re import sys import ast import types @@ -44,6 +45,25 @@ from feathr.constants import * +def _to_snake(d, level: int = 0): + """ + Convert `string`, `list[string]`, or all keys in a `dict` into snake case + The maximum length of input string or list is 100, or it will be truncated before being processed, for dict, the exception will be thrown if it has more than 100 keys. + the maximum nested level is 10, otherwise the exception will be thrown + """ + if level >= 10: + raise ValueError("Too many nested levels") + if isinstance(d, str): + d = d[:100] + return re.sub(r'(? 100: + raise ValueError("Dict has too many keys") + return {_to_snake(a, level + 1): _to_snake(b, level + 1) if isinstance(b, (dict, list)) else b for a, b in d.items()} + + class _PurviewRegistry(FeathrRegistry): """ Initializes the feature registry, doing the following: @@ -720,6 +740,32 @@ def upload_single_entity_to_purview(self,entity:Union[AtlasEntity,AtlasProcess]) The entity itself will also be modified, fill the GUID with real GUID in Purview. In order to avoid having concurrency issue, and provide clear guidance, this method only allows entity uploading once at a time. ''' + try: + """ + Try to find existing entity/process first, if found, return the existing entity's GUID + """ + id = self.get_entity_id(entity.qualifiedName) + response = self.purview_client.get_entity(id)['entities'][0] + j = entity.to_json() + if j["typeName"] == response["typeName"]: + if j["typeName"] == "Process": + if response["attributes"]["qualifiedName"] != j["attributes"]["qualifiedName"]: + raise RuntimeError("The requested entity %s conflicts with the existing entity in PurView" % j["attributes"]["qualifiedName"]) + else: + if "type" in response['attributes'] and response["typeName"] in (TYPEDEF_ANCHOR_FEATURE, TYPEDEF_DERIVED_FEATURE): + conf = ConfigFactory.parse_string(response['attributes']['type']) + response['attributes']['type'] = dict(conf) + keys = set([_to_snake(key) for key in j["attributes"].keys()]) - set(["qualified_name"]) + keys.add("qualifiedName") + for k in keys: + if response["attributes"][k] != j["attributes"][k]: + raise RuntimeError("The requested entity %s conflicts with the existing entity in PurView" % j["attributes"]["qualifiedName"]) + return response["guid"] + else: + raise RuntimeError("The requested entity %s conflicts with the existing entity in PurView" % j["attributes"]["qualifiedName"]) + except AtlasException as e: + pass + try: entity.lastModifiedTS="0" result = self.purview_client.upload_entities([entity]) diff --git a/registry/purview-registry/registry/purview_registry.py b/registry/purview-registry/registry/purview_registry.py index 9f5f47560..15a650167 100644 --- a/registry/purview-registry/registry/purview_registry.py +++ b/registry/purview-registry/registry/purview_registry.py @@ -5,6 +5,9 @@ from urllib.error import HTTPError from uuid import UUID +from registry.models import to_snake +from pyapacheatlas.core.util import AtlasException + from azure.identity import DefaultAzureCredential from loguru import logger from pyapacheatlas.auth.azcredential import AzCredentialWrapper @@ -20,6 +23,9 @@ Label_BelongsTo = "BELONGSTO" Label_Consumes = "CONSUMES" Label_Produces = "PRODUCES" +TYPEDEF_DERIVED_FEATURE="feathr_derived_feature_v1" +TYPEDEF_ANCHOR_FEATURE="feathr_anchor_feature_v1" + TYPEDEF_ARRAY_ANCHOR=f"array" TYPEDEF_ARRAY_DERIVED_FEATURE=f"array" TYPEDEF_ARRAY_ANCHOR_FEATURE=f"array" @@ -568,17 +574,47 @@ def _register_feathr_feature_types(self): def _upload_entity_batch(self, entity_batch:list[AtlasEntity]): # we only support entity creation, update is not supported. # setting lastModifiedTS ==0 will ensure this, if another entity with ts>=1 exist - # upload funtion will fail with 412 Precondition fail. + # upload function will fail with 412 Precondition fail. for entity in entity_batch: - entity.lastModifiedTS="0" - results = self.purview_client.upload_entities( - batch=entity) - if results: - dict = {x.guid: x for x in entity_batch} - for k, v in results['guidAssignments'].items(): - dict[k].guid = v + self._upload_single_entity(entity) + + def _upload_single_entity(self, entity:AtlasEntity): + try: + """ + Try to find existing entity/process first, if found, return the existing entity's GUID + """ + id = self.get_entity_id(entity.qualifiedName) + response = self.purview_client.get_entity(id)['entities'][0] + j = entity.to_json() + if j["typeName"] == response["typeName"]: + if j["typeName"] == "Process": + if response["attributes"]["qualifiedName"] != j["attributes"]["qualifiedName"]: + raise RuntimeError("The requested entity %s conflicts with the existing entity in PurView" % j["attributes"]["qualifiedName"]) + else: + if "type" in response['attributes'] and response["typeName"] in (TYPEDEF_ANCHOR_FEATURE, TYPEDEF_DERIVED_FEATURE): + conf = ConfigFactory.parse_string(response['attributes']['type']) + response['attributes']['type'] = dict(conf) + keys = set([to_snake(key) for key in j["attributes"].keys()]) - set(["qualified_name"]) + keys.add("qualifiedName") + for k in keys: + if response["attributes"][k] != j["attributes"][k]: + raise RuntimeError("The requested entity %s conflicts with the existing entity in PurView" % j["attributes"]["qualifiedName"]) + entity.guid = response["guid"] + return else: - raise RuntimeError("Feature registration failed.", results) + raise RuntimeError("The requested entity %s conflicts with the existing entity in PurView" % j["attributes"]["qualifiedName"]) + except AtlasException as e: + pass + + entity.lastModifiedTS="0" + results = self.purview_client.upload_entities( + batch=entity) + if results: + d = {x.guid: x for x in [entity]} + for k, v in results['guidAssignments'].items(): + d[k].guid = v + else: + raise RuntimeError("Feature registration failed.", results) def _generate_fully_qualified_name(self, segments): return self.registry_delimiter.join(segments) From b6cff14ecbf77a147a53653056e9efdd3f1fb6a9 Mon Sep 17 00:00:00 2001 From: Richin Jain Date: Sun, 16 Oct 2022 19:24:52 -0700 Subject: [PATCH 22/68] Adding DevSkim linter to Github actions (#657) * Adding DevSkim linter to Github actions * Ignoring .git and test folder --- .github/workflows/devskim-security-linter.yml | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) create mode 100644 .github/workflows/devskim-security-linter.yml diff --git a/.github/workflows/devskim-security-linter.yml b/.github/workflows/devskim-security-linter.yml new file mode 100644 index 000000000..e7e33b7ab --- /dev/null +++ b/.github/workflows/devskim-security-linter.yml @@ -0,0 +1,37 @@ +# This workflow uses actions that are not certified by GitHub. +# They are provided by a third-party (Microsoft) and are governed by +# separate terms of service, privacy policy, and support +# documentation. +# For more details about Devskim, visit https://github.com/marketplace/actions/devskim + +name: DevSkim + +on: + push: + branches: [ "main" ] + pull_request: + branches: [ "main" ] + schedule: + - cron: '00 4 * * *' + +jobs: + lint: + name: DevSkim + runs-on: ubuntu-20.04 + permissions: + actions: read + contents: read + security-events: write + steps: + - name: Checkout code + uses: actions/checkout@v3 + + - name: Run DevSkim scanner + uses: microsoft/DevSkim-Action@v1 + with: + ignore-globs: "**/.git/**,**/test/**" + + - name: Upload DevSkim scan results to GitHub Security tab + uses: github/codeql-action/upload-sarif@v2 + with: + sarif_file: devskim-results.sarif From 3d12944e952bf3dd40bb7ee56f40b32fd31f14d8 Mon Sep 17 00:00:00 2001 From: Boli Guan Date: Mon, 17 Oct 2022 16:26:29 +0800 Subject: [PATCH 23/68] Fix icons in UI cannot auto scale (#737) (#744) * Fix icons in UI cannot auto scale (#737) * Fix home.css code style issue --- ui/.vscode/settings.json | 9 +- ui/src/pages/home/home.css | 23 ++++ ui/src/pages/home/home.tsx | 210 +++++++++++-------------------------- 3 files changed, 93 insertions(+), 149 deletions(-) create mode 100644 ui/src/pages/home/home.css diff --git a/ui/.vscode/settings.json b/ui/.vscode/settings.json index 24fe97ae7..c8e624dc0 100644 --- a/ui/.vscode/settings.json +++ b/ui/.vscode/settings.json @@ -4,5 +4,12 @@ }, "editor.defaultFormatter": "esbenp.prettier-vscode", "editor.formatOnSave": true, - "eslint.workingDirectories": [{ "mode": "auto" }] + "eslint.workingDirectories": [ + { + "mode": "auto" + } + ], + "[css]": { + "editor.defaultFormatter": "esbenp.prettier-vscode" + } } diff --git a/ui/src/pages/home/home.css b/ui/src/pages/home/home.css new file mode 100644 index 000000000..5c4a3b8a8 --- /dev/null +++ b/ui/src/pages/home/home.css @@ -0,0 +1,23 @@ +.home .ant-card { + box-shadow: 5px 8px 15px 5px rgba(208, 216, 243, 0.6); + border-radius: 8px; +} + +.home .card-meta { + display: flex; +} + +.home .card-meta .ant-card-meta-avatar { + max-width: 80px; + flex-basis: 30%; + box-sizing: border-box; +} + +.home .card-meta .ant-card-meta-avatar > span { + width: 100%; +} + +.home .card-meta .ant-card-meta-avatar svg { + width: 100%; + height: auto; +} diff --git a/ui/src/pages/home/home.tsx b/ui/src/pages/home/home.tsx index 240ab8901..88732ffeb 100644 --- a/ui/src/pages/home/home.tsx +++ b/ui/src/pages/home/home.tsx @@ -8,17 +8,42 @@ import { ProjectOutlined, } from "@ant-design/icons"; +import "./home.css"; + const { Title } = Typography; +const { Meta } = Card; + +const features = [ + { + icon: , + title: "Projects", + link: "/projects", + linkText: "See all", + }, + { + icon: , + title: "Sources", + link: "/dataSources", + linkText: "See all", + }, + { + icon: , + title: "Features", + link: "/features", + linkText: "See all", + }, + { + icon: , + title: "Monitoring", + link: "/monitoring", + linkText: "See all", + }, +]; const Home = () => { return ( -
- +
+ Welcome to Feathr Feature Store You can use Feathr UI to search features, identify data sources, track @@ -33,141 +58,36 @@ const Home = () => { - - - - - - + {features.map((item) => { + return ( + + + + {item.title} + + } + description={{item.linkText}} /> - - - - - Projects - - - - - - See all - - - - - - - - - - - - - - - - - Sources - - - - - - See all - - - - - - - - - - - - - - - - - Features - - - - - - See all - - - - - - - - - - - - - - - - - Monitoring - - - - - - See all - - - - - - - + + + ); + })} - - - + + + Need help to get started? Explore the following resources to get started with Feathr:
    @@ -237,14 +157,8 @@ const Home = () => {

    - - + + Recent Activity Under construction From 3070a863d69f767658ca4adc819f6a3aac71ea18 Mon Sep 17 00:00:00 2001 From: Enya-Yx <108409954+enya-yx@users.noreply.github.com> Date: Mon, 17 Oct 2022 16:30:25 +0800 Subject: [PATCH 24/68] Expose 'timePartitionPattern' in Python API [ WIP ] (#714) * Expose 'timePartitionPattern' * add test case * Add test cases and docstring * delete local files * quick fix Co-authored-by: enya-yx Co-authored-by: enya-yx --- feathr_project/feathr/definition/source.py | 21 ++++++- .../spark_provider/_synapse_submission.py | 2 + feathr_project/test/test_azure_spark_e2e.py | 36 +++++++++++- feathr_project/test/test_feature_anchor.py | 58 ++++++++++++++++++- feathr_project/test/test_fixture.py | 28 +++++++++ 5 files changed, 142 insertions(+), 3 deletions(-) diff --git a/feathr_project/feathr/definition/source.py b/feathr_project/feathr/definition/source.py index b9721a1a5..3ca8925df 100644 --- a/feathr_project/feathr/definition/source.py +++ b/feathr_project/feathr/definition/source.py @@ -100,14 +100,30 @@ class HdfsSource(Source): - `epoch` (seconds since epoch), for example `1647737463` - `epoch_millis` (milliseconds since epoch), for example `1647737517761` - Any date formats supported by [SimpleDateFormat](https://docs.oracle.com/javase/8/docs/api/java/text/SimpleDateFormat.html). + registry_tags: A dict of (str, str) that you can pass to feature registry for better organization. For example, you can use {"deprecated": "true"} to indicate this source is deprecated, etc. + time_partition_pattern(Optional[str]): Format of the time partitioned feature data. e.g. yyyy/MM/DD. All formats supported in dateTimeFormatter. + config: + timeSnapshotHdfsSource: + { + location: + { + path: "/data/somePath/daily" + } + timePartitionPattern: "yyyy/MM/dd" + } + Given the above HDFS path: /data/somePath/daily, + then the expectation is that the following sub directorie(s) should exist: + /data/somePath/daily/{yyyy}/{MM}/{dd} + """ - def __init__(self, name: str, path: str, preprocessing: Optional[Callable] = None, event_timestamp_column: Optional[str] = None, timestamp_format: Optional[str] = "epoch", registry_tags: Optional[Dict[str, str]] = None) -> None: + def __init__(self, name: str, path: str, preprocessing: Optional[Callable] = None, event_timestamp_column: Optional[str] = None, timestamp_format: Optional[str] = "epoch", registry_tags: Optional[Dict[str, str]] = None, time_partition_pattern: Optional[str] = None) -> None: super().__init__(name, event_timestamp_column, timestamp_format, registry_tags=registry_tags) self.path = path self.preprocessing = preprocessing + self.time_partition_pattern = time_partition_pattern if path.startswith("http"): logger.warning( "Your input path {} starts with http, which is not supported. Consider using paths starting with wasb[s]/abfs[s]/s3.", path) @@ -116,6 +132,9 @@ def to_feature_config(self) -> str: tm = Template(""" {{source.name}}: { location: {path: "{{source.path}}"} + {% if source.time_partition_pattern %} + timePartitionPattern: "{{source.time_partition_pattern}}" + {% endif %} {% if source.event_timestamp_column %} timeWindowParameters: { timestampColumn: "{{source.event_timestamp_column}}" diff --git a/feathr_project/feathr/spark_provider/_synapse_submission.py b/feathr_project/feathr/spark_provider/_synapse_submission.py index 010444715..5b55a3b06 100644 --- a/feathr_project/feathr/spark_provider/_synapse_submission.py +++ b/feathr_project/feathr/spark_provider/_synapse_submission.py @@ -432,6 +432,8 @@ def download_file(self, target_adls_directory: str, local_dir_cache: str): # returns the paths to all the files in the target director in ADLS # get all the paths that are not under a directory + test_paths = self.file_system_client.get_paths( + path=parse_result.path, recursive=False) result_paths = [basename(file_path.name) for file_path in self.file_system_client.get_paths( path=parse_result.path, recursive=False) if not file_path.is_directory] diff --git a/feathr_project/test/test_azure_spark_e2e.py b/feathr_project/test/test_azure_spark_e2e.py index d2aa0b032..9e452d2d0 100644 --- a/feathr_project/test/test_azure_spark_e2e.py +++ b/feathr_project/test/test_azure_spark_e2e.py @@ -20,7 +20,7 @@ from feathr import ValueType from feathr.utils.job_utils import get_result_df from feathrcli.cli import init -from test_fixture import (basic_test_setup, get_online_test_table_name) +from test_fixture import (basic_test_setup, get_online_test_table_name, time_partition_pattern_test_setup) from test_utils.constants import Constants # make sure you have run the upload feature script before running these tests @@ -58,6 +58,40 @@ def test_feathr_materialize_to_offline(): res_df = get_result_df(client, "avro", output_path + "/df0/daily/2020/05/20") assert res_df.shape[0] > 0 +def test_feathr_materialize_with_time_partition_pattern(): + """ + Test FeathrClient() using HdfsSource with 'timePartitionPattern'. + """ + test_workspace_dir = Path( + __file__).parent.resolve() / "test_user_workspace" + # os.chdir(test_workspace_dir) + + client: FeathrClient = time_partition_pattern_test_setup(os.path.join(test_workspace_dir, "feathr_config.yaml")) + + backfill_time = BackfillTime(start=datetime( + 2020, 5, 20), end=datetime(2020, 5, 20), step=timedelta(days=1)) + + now = datetime.now() + if client.spark_runtime == 'databricks': + output_path = ''.join(['dbfs:/feathrazure_cijob_materialize_offline_','_', str(now.minute), '_', str(now.second), ""]) + else: + output_path = ''.join(['abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/demo_data/feathrazure_cijob_materialize_offline_','_', str(now.minute), '_', str(now.second), ""]) + offline_sink = HdfsSink(output_path=output_path) + settings = MaterializationSettings("nycTaxiTable", + sinks=[offline_sink], + feature_names=[ + "f_location_avg_fare", "f_location_max_fare"], + backfill_time=backfill_time) + client.materialize_features(settings) + # assuming the job can successfully run; otherwise it will throw exception + client.wait_job_to_finish(timeout_sec=Constants.SPARK_JOB_TIMEOUT_SECONDS) + + # download result and just assert the returned result is not empty + # by default, it will write to a folder appended with date + res_df = get_result_df(client, "avro", output_path + "/df0/daily/2020/05/20") + assert res_df.shape[0] > 0 + + def test_feathr_online_store_agg_features(): """ Test FeathrClient() get_online_features and batch_get can get data correctly. diff --git a/feathr_project/test/test_feature_anchor.py b/feathr_project/test/test_feature_anchor.py index d5e6701b8..1fabeec67 100644 --- a/feathr_project/test/test_feature_anchor.py +++ b/feathr_project/test/test_feature_anchor.py @@ -167,4 +167,60 @@ def test_agg_anchor_to_config(): } } """ - assert ''.join(agg_anchor.to_feature_config().split()) == ''.join(expected_agg_feature_config.split()) \ No newline at end of file + assert ''.join(agg_anchor.to_feature_config().split()) == ''.join(expected_agg_feature_config.split()) + +def test_time_partition_to_config(): + batch_source = HdfsSource(name="testTimePartitionSource", + path="abfss://public@azurefeathrstorage.blob.core.windows.net/sample_data/time_partition_pattern", + time_partition_pattern="yyyy/MM/dd" + ) + key = TypedKey(key_column="key0", + key_column_type=ValueType.INT32) + agg_features = [ + Feature(name="f_loc_avg", + key=[key], + feature_type=FLOAT, + transform="f_location_avg_fare"), + Feature(name="f_loc_max", + feature_type=FLOAT, + key=[key], + transform="f_location_max_fare"), + ] + agg_anchor = FeatureAnchor(name="testTimePartitionFeaturesSource", + source=batch_source, + features=agg_features) + expected_time_partition_config = """ + anchors: { + testTimePartitionFeatures: { + source: testTimePartitionSource + key.sqlExpr: [key0] + features: { + f_loc_avg: { + def.sqlExpr: "f_location_avg_fare" + type: { + type: TENSOR + tensorCategory: DENSE + dimensionType: [] + valType: FLOAT + } + } + f_loc_max: { + def.sqlExpr: "f_location_max_fare" + type: { + type: TENSOR + tensorCategory: DENSE + dimensionType: [] + valType: FLOAT + } + } + } + } + } + sources: { + testTimePartitionSource: { + location: {path: "abfss://public@azurefeathrstorage.blob.core.windows.net/sample_data/time_partition_pattern"} + timePartitionPattern: "yyyy/MM/dd" + } + } + """ + assert ''.join(agg_anchor.to_feature_config().split()) == ''.join(expected_time_partition_config.split()) \ No newline at end of file diff --git a/feathr_project/test/test_fixture.py b/feathr_project/test/test_fixture.py index c048eff7c..637e74202 100644 --- a/feathr_project/test/test_fixture.py +++ b/feathr_project/test/test_fixture.py @@ -88,6 +88,34 @@ def basic_test_setup(config_path: str): return client +def time_partition_pattern_test_setup(config_path: str): + now = datetime.now() + # set workspace folder by time; make sure we don't have write conflict if there are many CI tests running + os.environ['SPARK_CONFIG__DATABRICKS__WORK_DIR'] = ''.join(['dbfs:/feathrazure_cijob','_', str(now.minute), '_', str(now.second), '_', str(now.microsecond)]) + os.environ['SPARK_CONFIG__AZURE_SYNAPSE__WORKSPACE_DIR'] = ''.join(['abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_github_ci','_', str(now.minute), '_', str(now.second) ,'_', str(now.microsecond)]) + client = FeathrClient(config_path=config_path) + batch_source = HdfsSource(name="testTimePartitionSource", + path="wasbs://public@azurefeathrstorage.blob.core.windows.net/sample_data/time_partition_pattern/daily", + time_partition_pattern="yyyy/MM/dd" + ) + key = TypedKey(key_column="key0", + key_column_type=ValueType.INT32) + agg_features = [ + Feature(name="f_loc_avg", + key=[key], + feature_type=FLOAT, + transform="f_location_avg_fare"), + Feature(name="f_loc_max", + feature_type=FLOAT, + key=[key], + transform="f_location_max_fare"), + ] + + agg_anchor = FeatureAnchor(name="testTimePartitionFeatures", + source=batch_source, + features=agg_features) + client.build_features(anchor_list=[agg_anchor]) + return client def snowflake_test_setup(config_path: str): now = datetime.now() From 83b79c9f894b734ff0596bfcbb343d430a904338 Mon Sep 17 00:00:00 2001 From: Richin Jain Date: Mon, 17 Oct 2022 01:32:25 -0700 Subject: [PATCH 25/68] Setting up component governance pipeline (#655) [skip ci] --- azure-pipelines.yml | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 azure-pipelines.yml diff --git a/azure-pipelines.yml b/azure-pipelines.yml new file mode 100644 index 000000000..e5808aff2 --- /dev/null +++ b/azure-pipelines.yml @@ -0,0 +1,15 @@ +# Component Governance Pipeline +# Runs the Feathr code through Component Governance Detection tool and publishes the result under compliance tab. + +trigger: +- main + +pool: + vmImage: ubuntu-latest + +steps: +- task: ComponentGovernanceComponentDetection@0 + inputs: + scanType: 'Register' + verbosity: 'Verbose' + alertWarningLevel: 'High' \ No newline at end of file From b03689862857f6bae9bea7b8d148b4349bd8b6d6 Mon Sep 17 00:00:00 2001 From: Xiaoyong Zhu Date: Mon, 17 Oct 2022 18:41:46 -0700 Subject: [PATCH 26/68] Add docs to explain on feature materialization behavior (#688) * Update materializing-features.md * Update materializing-features.md --- docs/concepts/materializing-features.md | 50 +++++++++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/docs/concepts/materializing-features.md b/docs/concepts/materializing-features.md index eacf980ff..3c31124e2 100644 --- a/docs/concepts/materializing-features.md +++ b/docs/concepts/materializing-features.md @@ -161,3 +161,53 @@ More reference on the APIs: - [MaterializationSettings API](https://feathr.readthedocs.io/en/latest/feathr.html#feathr.MaterializationSettings) - [HdfsSink API](https://feathr.readthedocs.io/en/latest/feathr.html#feathr.HdfsSource) + +## Expected behavior on Feature Materialization + +When end users materialize features to a sink, what is the expected behavior? + +It seems to be a straightforward question, but actually it is not. Basically when end users want to materialize a feature, Feathr is expecting that: For a certain entity key (say a user_id), there will be multiple features (say user_total_gift_card_balance, and user_purchase_in_last_week). So two checks will be performed: + +1. Those features should have the same entity key (say a user_id). You cannot materialize features for two entity keys in the same materialization job (although you can do it in different jobs), for example materializing `uer_total_purchase` and `product_sold_in_last_week` in the same Feathr materialization job. +2. Those features should all be "aggregated" feature. I.e. they should be a feature which has a type of `WindowAggTransformation`, such as `product_sold_in_last_week`, or `user_latest_total_gift_card_balance`. + +The first constraint is pretty straightforward to explain - since when Feathr materializes certain features, they are used to describe certain aspects of a given entity such as user. Describing `product_sold_in_last_week` would not make sense for users. + +The second constraint is a bit more interesting. For example, you have defined `user_total_gift_card_balance` and it has different value for the same user across different time, say the corresponding value is 40,30,20,20 for the last 4 days, like below. +Original data: + +| UserId | user_total_gift_card_balance | Date | +| ------ | ---------------------------- | ---------- | +| 1 | 40 | 2022/01/01 | +| 1 | 30 | 2022/01/02 | +| 1 | 20 | 2022/01/03 | +| 1 | 20 | 2022/01/04 | +| 2 | 40 | 2022/01/01 | +| 2 | 30 | 2022/01/02 | +| 2 | 20 | 2022/01/03 | +| 2 | 20 | 2022/01/04 | +| 3 | 40 | 2022/01/01 | +| 3 | 30 | 2022/01/02 | +| 3 | 20 | 2022/01/03 | +| 3 | 20 | 2022/01/04 | + +However, the materialized features have no dates associated with them. I.e. the materialized result should be something like this: + +| UserId | user_total_gift_card_balance | +| ------ | ---------------------------- | +| 1 | ? | +| 2 | ? | +| 3 | ? | + +When you ask Feathr to "materialize" `user_total_gift_card_balance` for you, there's only one value that can be materialized, since the materialized feature does not have a date associated with them. So the problem is - for a given `user_id`, only one `user_total_gift_card_balance` can be its feature. Which value you are choosing out of the 4 values? A random value? The latest value? + +It might be natural to think that "we should materialize the latest feature", and that behavior, by definition, is an "aggregation" operation, since we have 4 values for a given `user_id` but we are only materializing and using one of them. In that case, Feathr asks you to explicitly say that you want to materialize the latest feature (i.e. by using [Point-in-time Join](./point-in-time-join.md)) + +```python +feature = Feature(name="user_total_gift_card_balance", + key=UserId, + feature_type=FLOAT, + transform=WindowAggTransformation(agg_expr="gift_card_balance", + agg_func="LATEST", + window="7d")) +``` \ No newline at end of file From 5030eee7979973b979045773d0c7c0f429ac937d Mon Sep 17 00:00:00 2001 From: Enya-Yx <108409954+enya-yx@users.noreply.github.com> Date: Tue, 18 Oct 2022 15:08:39 +0800 Subject: [PATCH 27/68] Fix protobuf version (#711) * Fix protobuf version * quick fix Co-authored-by: enya-yx --- feathr_project/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/feathr_project/setup.py b/feathr_project/setup.py index 47d105061..f36ba18b0 100644 --- a/feathr_project/setup.py +++ b/feathr_project/setup.py @@ -37,7 +37,7 @@ "python-snappy<=0.6.1", "deltalake>=0.6.2", "graphlib_backport<=1.0.3", - "protobuf==3.*", + "protobuf<=3.19.4,>=3.0.0", "confluent-kafka<=1.9.2", "databricks-cli<=0.17.3", "avro<=1.11.1", From aad580dc580364664a3eefac3321523213313934 Mon Sep 17 00:00:00 2001 From: Enya-Yx <108409954+enya-yx@users.noreply.github.com> Date: Tue, 18 Oct 2022 22:45:15 +0800 Subject: [PATCH 28/68] Add some notes based on on-call issues (#753) * Add some notes based on on-call issues * quick fix Co-authored-by: enya-yx --- docs/concepts/feature-registry.md | 2 ++ docs/how-to-guides/feathr-configuration-and-env.md | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/concepts/feature-registry.md b/docs/concepts/feature-registry.md index 9bc00b275..112fa1c4b 100644 --- a/docs/concepts/feature-registry.md +++ b/docs/concepts/feature-registry.md @@ -74,6 +74,8 @@ client.register_features() all_features = client.list_registered_features(project_name=client.project_name) ``` +Please avoid applying a same name to different features under a certain project. Since it will be treated as updating an exsiting project which is not supported by feathr and will cause errors. + ### Reuse Features from Existing Registry The feature producers can just let the feature consumers know which features exist so the feature consumers can reuse them. For feature consumers, they can reuse existing features from the registry. The whole project can be retrieved to local environment by calling this API `client.get_features_from_registry` with a project name. This encourage feature reuse across organizations. For example, end users of a feature just need to read all feature definitions from the existing projects, then use a few features from the projects and join those features with a new dataset you have. diff --git a/docs/how-to-guides/feathr-configuration-and-env.md b/docs/how-to-guides/feathr-configuration-and-env.md index e4d393a43..e5c6b761a 100644 --- a/docs/how-to-guides/feathr-configuration-and-env.md +++ b/docs/how-to-guides/feathr-configuration-and-env.md @@ -64,7 +64,7 @@ Feathr will get the configurations in the following order: | SPARK_CONFIG__SPARK_RESULT_OUTPUT_PARTS | Configure number of parts for the spark output for feature generation job | Required | | SPARK_CONFIG__AZURE_SYNAPSE__DEV_URL | Dev URL to the synapse cluster. Usually it's something like `https://yourclustername.dev.azuresynapse.net` | Required if using Azure Synapse | | SPARK_CONFIG__AZURE_SYNAPSE__POOL_NAME | name of the spark pool that you are going to use | Required if using Azure Synapse | -| SPARK_CONFIG__AZURE_SYNAPSE__WORKSPACE_DIR | A location that Synapse has access to. This workspace dir stores all the required configuration files and the jar resources. All the feature definitions will be uploaded here | Required if using Azure Synapse | +| SPARK_CONFIG__AZURE_SYNAPSE__WORKSPACE_DIR | A location that Synapse has access to. This workspace dir stores all the required configuration files and the jar resources. All the feature definitions will be uploaded here. Suggest to use an empty dir for a new spark job to avoid conflicts. | Required if using Azure Synapse | | SPARK_CONFIG__AZURE_SYNAPSE__EXECUTOR_SIZE | Specifies the executor size for the Azure Synapse cluster. Currently the options are `Small`, `Medium`, `Large`. | Required if using Azure Synapse | | SPARK_CONFIG__AZURE_SYNAPSE__EXECUTOR_NUM | Specifies the number of executors for the Azure Synapse cluster | Required if using Azure Synapse | | SPARK_CONFIG__AZURE_SYNAPSE__FEATHR_RUNTIME_LOCATION | Specifies the Feathr runtime location. Support local paths, path start with `http(s)://`, and paths start with `abfss:/`. If not set, will use the [Feathr package published in Maven](https://search.maven.org/artifact/com.linkedin.feathr/feathr_2.12). | Required if using Azure Synapse | From 4b9b4940bf3e8a40d9fc720a8af297a9ea8adafc Mon Sep 17 00:00:00 2001 From: Yuqing Wei Date: Wed, 19 Oct 2022 13:28:50 +0800 Subject: [PATCH 29/68] Refine spark runtime error message (#755) Signed-off-by: Yuqing Wei --- feathr_project/feathr/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/feathr_project/feathr/client.py b/feathr_project/feathr/client.py index 1f77d61e0..b21c15549 100644 --- a/feathr_project/feathr/client.py +++ b/feathr_project/feathr/client.py @@ -111,7 +111,7 @@ def __init__(self, config_path:str = "./feathr_config.yaml", local_workspace_dir self.credential = credential if self.spark_runtime not in {'azure_synapse', 'databricks', 'local'}: raise RuntimeError( - 'Only \'azure_synapse\' and \'databricks\' are currently supported.') + f'{self.spark_runtime} is not supported. Only \'azure_synapse\', \'databricks\' and \'local\' are currently supported.') elif self.spark_runtime == 'azure_synapse': # Feathr is a spark-based application so the feathr jar compiled from source code will be used in the # Spark job submission. The feathr jar hosted in cloud saves the time users needed to upload the jar from From b8e3b27189a969744d636d0be8945559430ab2ef Mon Sep 17 00:00:00 2001 From: Richin Jain Date: Tue, 18 Oct 2022 22:58:46 -0700 Subject: [PATCH 30/68] Serialization bug due to version incompatibility between azure-core and msrest (#763) * Adding DevSkim linter to Github actions * Fix in ARM template to pull latest tagged release image from dockerhub * Removing dev skim file from this branch * Fixing linkedin org reference * Removing the docker index url from dockerhub image name as it seems to cause problem with the update * Adding to the right file, had a dockerhub workflow file with different name * Adding debug statements to test udf issue on Synapse * Adding more print statements * Pinning msrest version to work with pinned version of azure-core * Removing debug code from previous branch --- feathr_project/setup.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/feathr_project/setup.py b/feathr_project/setup.py index f36ba18b0..7ecd24ecb 100644 --- a/feathr_project/setup.py +++ b/feathr_project/setup.py @@ -51,6 +51,10 @@ # https://github.com/Azure/azure-sdk-for-python/pull/22891 # using a version lower than that to workaround this issue. "azure-core<=1.22.1", + # azure-core 1.22.1 is dependent on msrest==0.6.21, if an environment(AML) has a different version of azure-core (say 1.24.0), + # it brings a different version of msrest(0.7.0) which is incompatible with azure-core==1.22.1. Hence we need to pin it. + # See this for more details: https://github.com/Azure/azure-sdk-for-python/issues/24765 + "msrest<=0.6.21", "typing_extensions>=4.2.0" ], tests_require=[ # TODO: This has been depricated From fa10e721c44c6aeb2e8679614dbf28b1c033b8d7 Mon Sep 17 00:00:00 2001 From: Yuqing Wei Date: Wed, 19 Oct 2022 18:49:30 +0800 Subject: [PATCH 31/68] Unify Python SDK Build Version and decouple Feathr Maven Version (#746) * unify python package version and enable env setting for scala version Signed-off-by: Yuqing Wei * update docs and decouple maven version Signed-off-by: Yuqing Wei * change version back to 0.8.0 to avoid conflicts Signed-off-by: Yuqing Wei * fix typo Signed-off-by: Yuqing Wei Signed-off-by: Yuqing Wei --- docs/dev_guide/feathr_overall_release_guide.md | 6 +++--- feathr_project/feathr/__init__.py | 7 ++----- feathr_project/feathr/constants.py | 6 +++++- feathr_project/feathr/version.py | 1 + feathr_project/setup.py | 14 +++++++++++++- 5 files changed, 24 insertions(+), 10 deletions(-) create mode 100644 feathr_project/feathr/version.py diff --git a/docs/dev_guide/feathr_overall_release_guide.md b/docs/dev_guide/feathr_overall_release_guide.md index 069f6edf4..0174c8dae 100644 --- a/docs/dev_guide/feathr_overall_release_guide.md +++ b/docs/dev_guide/feathr_overall_release_guide.md @@ -31,11 +31,11 @@ Read through the [commit log](https://github.com/feathr-ai/feathr/commits/main) ## Code Changes Before the release is made, the version needs to be updated in following places - [build.sbt](https://github.com/feathr-ai/feathr/blob/main/build.sbt#L3) - For Maven release version -- [setup.py](https://github.com/feathr-ai/feathr/blob/main/feathr_project/setup.py#L10) - For PyPi release version +- [version.py](https://github.com/feathr-ai/feathr/blob/main/feathr_project/feathr/version.py#L1) - For Feathr version - [conf.py](https://github.com/feathr-ai/feathr/blob/main/feathr_project/docs/conf.py#L27) - For documentation version -- [feathr_config.yaml](https://github.com/feathr-ai/feathr/blob/main/feathr_project/test/test_user_workspace/feathr_config.yaml#L84) - To set the spark runtime location for Azure Synapse and Azure Databricks used by test suite. -- [constants.py](https://github.com/feathr-ai/feathr/blob/73656fe4a57219e99ff6fede10d51a000ae90fa1/feathr_project/feathr/constants.py#L31) - To set the default maven artifact version +- [feathr_config.yaml](https://github.com/feathr-ai/feathr/blob/main/feathr_project/test/test_user_workspace/feathr_config.yaml#L84) - To set the spark runtime location for Azure Synapse and Azure Databricks used by test suite. Please update all .yaml files under this path. - [azure_resource_provision.json](https://github.com/feathr-ai/feathr/blob/main/docs/how-to-guides/azure_resource_provision.json#L114) - To set the deployment template to pull the latest release image. +- [constants.py](https://github.com/feathr-ai/feathr/blob/main/feathr_project/feathr/constants.py#L31) - To set the default maven artifact version (Only needed when maven version is **NOT** the same as python sdk version) ## Triggering automated release pipelines Our goal is to automate the release process as much as possible. So far, we have automated the following steps diff --git a/feathr_project/feathr/__init__.py b/feathr_project/feathr/__init__.py index 9b0cf0a49..74809fd81 100644 --- a/feathr_project/feathr/__init__.py +++ b/feathr_project/feathr/__init__.py @@ -1,5 +1,3 @@ -import pkg_resources - from .client import FeathrClient from .spark_provider.feathr_configurations import SparkExecutionConfiguration from .definition.feature_derivations import * @@ -19,6 +17,7 @@ from .definition.settings import * from .utils.job_utils import * from .utils.feature_printer import * +from .version import __version__ # skipped class as they are internal methods: # RepoDefinitions, HoconConvertible, @@ -75,7 +74,5 @@ 'ObservationSettings', 'FeaturePrinter', 'SparkExecutionConfiguration', + __version__, ] - - -__version__ = pkg_resources.require("feathr")[0].version diff --git a/feathr_project/feathr/constants.py b/feathr_project/feathr/constants.py index c4cbad7ff..b2222e2b6 100644 --- a/feathr_project/feathr/constants.py +++ b/feathr_project/feathr/constants.py @@ -28,7 +28,11 @@ TYPEDEF_ARRAY_DERIVED_FEATURE=f"array" TYPEDEF_ARRAY_ANCHOR_FEATURE=f"array" -FEATHR_MAVEN_ARTIFACT="com.linkedin.feathr:feathr_2.12:0.8.2" +# Decouple Feathr MAVEN Version from Feathr Python SDK Version +import os +from feathr.version import __version__ +FEATHR_MAVEN_VERSION = os.environ.get("FEATHR_MAVEN_VERSION", __version__) +FEATHR_MAVEN_ARTIFACT=f"com.linkedin.feathr:feathr_2.12:{FEATHR_MAVEN_VERSION}" JOIN_CLASS_NAME="com.linkedin.feathr.offline.job.FeatureJoinJob" GEN_CLASS_NAME="com.linkedin.feathr.offline.job.FeatureGenJob" \ No newline at end of file diff --git a/feathr_project/feathr/version.py b/feathr_project/feathr/version.py new file mode 100644 index 000000000..807119de6 --- /dev/null +++ b/feathr_project/feathr/version.py @@ -0,0 +1 @@ +__version__ = "0.8.0" \ No newline at end of file diff --git a/feathr_project/setup.py b/feathr_project/setup.py index 7ecd24ecb..4f766b4ec 100644 --- a/feathr_project/setup.py +++ b/feathr_project/setup.py @@ -1,3 +1,5 @@ +import sys +import os from setuptools import setup, find_packages from pathlib import Path @@ -5,9 +7,19 @@ root_path = Path(__file__).resolve().parent.parent long_description = (root_path / "docs/README.md").read_text(encoding="utf8") +try: + exec(open("feathr/version.py").read()) +except IOError: + print("Failed to load Feathr version file for packaging.", + file=sys.stderr) + sys.exit(-1) + +VERSION = __version__ # noqa +os.environ["FEATHR_VERSION"] = VERSION + setup( name='feathr', - version='0.8.2', + version=VERSION, long_description=long_description, long_description_content_type="text/markdown", author_email="feathr-technical-discuss@lists.lfaidata.foundation", From c0e8bc8ae017d7003210ad96e755f8abb4781543 Mon Sep 17 00:00:00 2001 From: Yuqing Wei Date: Wed, 19 Oct 2022 18:50:39 +0800 Subject: [PATCH 32/68] replace hard code string in notebook and align with others (#765) Signed-off-by: Yuqing Wei Signed-off-by: Yuqing Wei --- docs/samples/product_recommendation_demo_advanced.ipynb | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/samples/product_recommendation_demo_advanced.ipynb b/docs/samples/product_recommendation_demo_advanced.ipynb index e4c5917a5..b03dccac6 100644 --- a/docs/samples/product_recommendation_demo_advanced.ipynb +++ b/docs/samples/product_recommendation_demo_advanced.ipynb @@ -116,7 +116,7 @@ }, "outputs": [], "source": [ - "resource_prefix = \"ckim2\"" + "resource_prefix = \"feathr_resource_prefix\"" ] }, { @@ -1214,7 +1214,7 @@ "widgets": {} }, "kernelspec": { - "display_name": "Python 3.9.5 ('base')", + "display_name": "Python 3.9.13 64-bit ('3.9.13')", "language": "python", "name": "python3" }, @@ -1228,11 +1228,11 @@ "name": "python", "nbconvert_exporter": "python", "pygments_lexer": "ipython3", - "version": "3.9.5" + "version": "3.9.13" }, "vscode": { "interpreter": { - "hash": "3d597f4c481aa0f25dceb95d2a0067e73c0966dcbd003d741d821a7208527ecf" + "hash": "c5d1b88564ea095927319e95d120a01ba9530a1c584720276480e541fd6461c7" } } }, From 143ff898b6c61666289981948f000012fda822d1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BE=90=E8=BE=B0?= Date: Thu, 20 Oct 2022 18:39:37 +0800 Subject: [PATCH 33/68] Add flag to enable generation non-agg features (#719) * Add flag to enable generation non-agg features * Typo * Resolve comments --- feathr_project/feathr/client.py | 19 +++++++++- .../test/test_azure_snowflake_e2e.py | 2 +- feathr_project/test/test_azure_spark_e2e.py | 2 +- .../test/test_azure_spark_maven_e2e.py | 37 ++++++++++++++++++- .../test/test_feature_materialization.py | 2 +- .../test/test_pyduf_preprocessing_e2e.py | 2 +- 6 files changed, 56 insertions(+), 8 deletions(-) diff --git a/feathr_project/feathr/client.py b/feathr_project/feathr/client.py index b21c15549..b14bf868e 100644 --- a/feathr_project/feathr/client.py +++ b/feathr_project/feathr/client.py @@ -6,6 +6,7 @@ from typing import Dict, List, Union from azure.identity import DefaultAzureCredential +from feathr.definition.transformation import WindowAggTransformation from jinja2 import Template from pyhocon import ConfigFactory import redis @@ -608,17 +609,31 @@ def _valid_materialize_keys(self, features: List[str], allow_empty_key=False): self.logger.error(f"Inconsistent feature keys. Current keys are {str(keys)}") return False return True - - def materialize_features(self, settings: MaterializationSettings, execution_configurations: Union[SparkExecutionConfiguration ,Dict[str,str]] = {}, verbose: bool = False): + + def materialize_features(self, settings: MaterializationSettings, execution_configurations: Union[SparkExecutionConfiguration ,Dict[str,str]] = {}, verbose: bool = False, allow_materialize_non_agg_feature: bool = False): """Materialize feature data Args: settings: Feature materialization settings execution_configurations: a dict that will be passed to spark job when the job starts up, i.e. the "spark configurations". Note that not all of the configuration will be honored since some of the configurations are managed by the Spark platform, such as Databricks or Azure Synapse. Refer to the [spark documentation](https://spark.apache.org/docs/latest/configuration.html) for a complete list of spark configurations. + allow_materialize_non_agg_feature: Materializing non-aggregated features (the features without WindowAggTransformation) doesn't output meaningful results so it's by default set to False, but if you really want to materialize non-aggregated features, set this to True. """ feature_list = settings.feature_names if len(feature_list) > 0 and not self._valid_materialize_keys(feature_list): raise RuntimeError(f"Invalid materialization features: {feature_list}, since they have different keys. Currently Feathr only supports materializing features of the same keys.") + + if not allow_materialize_non_agg_feature: + # Check if there are non-aggregation features in the list + for fn in feature_list: + # Check over anchor features + for anchor in self.anchor_list: + for feature in anchor.features: + if feature.name == fn and not isinstance(feature.transform, WindowAggTransformation): + raise RuntimeError(f"Feature {fn} is not an aggregation feature. Currently Feathr only supports materializing aggregation features. If you want to materialize {fn}, please set allow_materialize_non_agg_feature to True.") + # Check over derived features + for feature in self.derived_feature_list: + if feature.name == fn and not isinstance(feature.transform, WindowAggTransformation): + raise RuntimeError(f"Feature {fn} is not an aggregation feature. Currently Feathr only supports materializing aggregation features. If you want to materialize {fn}, please set allow_materialize_non_agg_feature to True.") # Collect secrets from sinks secrets = [] diff --git a/feathr_project/test/test_azure_snowflake_e2e.py b/feathr_project/test/test_azure_snowflake_e2e.py index c84aa9153..17474ab1b 100644 --- a/feathr_project/test/test_azure_snowflake_e2e.py +++ b/feathr_project/test/test_azure_snowflake_e2e.py @@ -30,7 +30,7 @@ def test_feathr_online_store_agg_features(): feature_names=['f_snowflake_call_center_division_name', 'f_snowflake_call_center_zipcode'], backfill_time=backfill_time) - client.materialize_features(settings) + client.materialize_features(settings, allow_materialize_non_agg_feature=True) # just assume the job is successful without validating the actual result in Redis. Might need to consolidate # this part with the test_feathr_online_store test case client.wait_job_to_finish(timeout_sec=Constants.SPARK_JOB_TIMEOUT_SECONDS) diff --git a/feathr_project/test/test_azure_spark_e2e.py b/feathr_project/test/test_azure_spark_e2e.py index 9e452d2d0..e82e0efe9 100644 --- a/feathr_project/test/test_azure_spark_e2e.py +++ b/feathr_project/test/test_azure_spark_e2e.py @@ -153,7 +153,7 @@ def test_feathr_online_store_non_agg_features(): feature_names=["f_gen_trip_distance", "f_gen_is_long_trip_distance", "f1", "f2", "f3", "f4", "f5", "f6"], backfill_time=backfill_time) - client.materialize_features(settings) + client.materialize_features(settings, allow_materialize_non_agg_feature=True) # just assume the job is successful without validating the actual result in Redis. Might need to consolidate # this part with the test_feathr_online_store test case client.wait_job_to_finish(timeout_sec=Constants.SPARK_JOB_TIMEOUT_SECONDS) diff --git a/feathr_project/test/test_azure_spark_maven_e2e.py b/feathr_project/test/test_azure_spark_maven_e2e.py index b8e7cefb0..6b93bb7a8 100644 --- a/feathr_project/test/test_azure_spark_maven_e2e.py +++ b/feathr_project/test/test_azure_spark_maven_e2e.py @@ -3,8 +3,12 @@ from pathlib import Path from feathr import (BackfillTime, MaterializationSettings) -from feathr import RedisSink +# from feathr import * from feathr.client import FeathrClient +from feathr.definition.dtype import ValueType +from feathr.definition.query_feature_list import FeatureQuery +from feathr.definition.settings import ObservationSettings +from feathr.definition.typed_key import TypedKey from test_fixture import (basic_test_setup, get_online_test_table_name) from test_utils.constants import Constants @@ -22,6 +26,35 @@ def test_feathr_online_store_agg_features(): # Maven package as the dependency and `noop.jar` as the main file client: FeathrClient = basic_test_setup(os.path.join(test_workspace_dir, "feathr_config_maven.yaml")) + + + location_id = TypedKey(key_column="DOLocationID", + key_column_type=ValueType.INT32, + description="location id in NYC", + full_name="nyc_taxi.location_id") + + feature_query = FeatureQuery( + feature_list=["f_location_avg_fare"], key=location_id) + settings = ObservationSettings( + observation_path="wasbs://public@azurefeathrstorage.blob.core.windows.net/sample_data/green_tripdata_2020-04.csv", + event_timestamp_column="lpep_dropoff_datetime", + timestamp_format="yyyy-MM-dd HH:mm:ss") + + now = datetime.now() + # set output folder based on different runtime + if client.spark_runtime == 'databricks': + output_path = ''.join(['dbfs:/feathrazure_cijob','_', str(now.minute), '_', str(now.second), ".avro"]) + else: + output_path = ''.join(['abfss://xchfeathrtest4fs@xchfeathrtest4sto.dfs.core.windows.net/demo_data/output','_', str(now.minute), '_', str(now.second), ".avro"]) + + + client.get_offline_features(observation_settings=settings, + feature_query=feature_query, + output_path=output_path) + + # assuming the job can successfully run; otherwise it will throw exception + client.wait_job_to_finish(timeout_sec=Constants.SPARK_JOB_TIMEOUT_SECONDS) + return backfill_time = BackfillTime(start=datetime( 2020, 5, 20), end=datetime(2020, 5, 20), step=timedelta(days=1)) redisSink = RedisSink(table_name=online_test_table) @@ -51,4 +84,4 @@ def test_feathr_online_store_agg_features(): assert res['239'][0] != None assert res['239'][1] != None assert res['265'][0] != None - assert res['265'][1] != None \ No newline at end of file + assert res['265'][1] != None diff --git a/feathr_project/test/test_feature_materialization.py b/feathr_project/test/test_feature_materialization.py index edd9bb537..e8100578c 100644 --- a/feathr_project/test/test_feature_materialization.py +++ b/feathr_project/test/test_feature_materialization.py @@ -236,7 +236,7 @@ def test_delete_feature_from_redis(): "f_day_of_week" ], backfill_time=backfill_time) - client.materialize_features(settings) + client.materialize_features(settings, allow_materialize_non_agg_feature=True) client.wait_job_to_finish(timeout_sec=Constants.SPARK_JOB_TIMEOUT_SECONDS) diff --git a/feathr_project/test/test_pyduf_preprocessing_e2e.py b/feathr_project/test/test_pyduf_preprocessing_e2e.py index 9ac9c1917..83ace12ea 100644 --- a/feathr_project/test/test_pyduf_preprocessing_e2e.py +++ b/feathr_project/test/test_pyduf_preprocessing_e2e.py @@ -103,7 +103,7 @@ def test_non_swa_feature_gen_with_offline_preprocessing(): "f_day_of_week" ], backfill_time=backfill_time) - client.materialize_features(settings) + client.materialize_features(settings, allow_materialize_non_agg_feature=True) # just assume the job is successful without validating the actual result in Redis. Might need to consolidate # this part with the test_feathr_online_store test case client.wait_job_to_finish(timeout_sec=Constants.SPARK_JOB_TIMEOUT_SECONDS) From 59e4ccf0d7a902a96faf592921620de5486769d6 Mon Sep 17 00:00:00 2001 From: Yuqing Wei Date: Mon, 24 Oct 2022 10:25:34 +0800 Subject: [PATCH 34/68] rollback 0.8.2 version bump PR (#771) Signed-off-by: Yuqing Wei --- build.sbt | 2 +- docs/how-to-guides/azure_resource_provision.json | 2 +- feathr_project/test/test_user_workspace/feathr_config.yaml | 4 ++-- .../test_user_workspace/feathr_config_registry_purview.yaml | 4 ++-- .../feathr_config_registry_purview_rbac.yaml | 4 ++-- .../test/test_user_workspace/feathr_config_registry_sql.yaml | 4 ++-- .../test_user_workspace/feathr_config_registry_sql_rbac.yaml | 4 ++-- 7 files changed, 12 insertions(+), 12 deletions(-) diff --git a/build.sbt b/build.sbt index d85f99016..2919ddae6 100644 --- a/build.sbt +++ b/build.sbt @@ -2,7 +2,7 @@ import sbt.Keys.publishLocalConfiguration ThisBuild / resolvers += Resolver.mavenLocal ThisBuild / scalaVersion := "2.12.15" -ThisBuild / version := "0.8.2" +ThisBuild / version := "0.8.0" ThisBuild / organization := "com.linkedin.feathr" ThisBuild / organizationName := "linkedin" val sparkVersion = "3.1.3" diff --git a/docs/how-to-guides/azure_resource_provision.json b/docs/how-to-guides/azure_resource_provision.json index 6ab77a419..03d175052 100644 --- a/docs/how-to-guides/azure_resource_provision.json +++ b/docs/how-to-guides/azure_resource_provision.json @@ -111,7 +111,7 @@ "destinationBacpacBlobUrl": "[concat('https://',variables('dlsName'),'.blob.core.windows.net/',variables('dlsFsName'),'/',variables('bacpacBlobName'))]", "bacpacDeploymentScriptName": "CopyBacpacFile", "bacpacDbExtensionName": "registryRbacDbImport", - "preBuiltdockerImage": "feathrfeaturestore/feathr-registry:releases-v0.8.2" + "preBuiltdockerImage": "feathrfeaturestore/feathr-registry:releases-v0.8.0" }, "functions": [], "resources": [ diff --git a/feathr_project/test/test_user_workspace/feathr_config.yaml b/feathr_project/test/test_user_workspace/feathr_config.yaml index b0f2b259c..e67c803ef 100644 --- a/feathr_project/test/test_user_workspace/feathr_config.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config.yaml @@ -82,7 +82,7 @@ spark_config: # Feathr Job configuration. Support local paths, path start with http(s)://, and paths start with abfs(s):// # this is the default location so end users don't have to compile the runtime again. # feathr_runtime_location: wasbs://public@azurefeathrstorage.blob.core.windows.net/feathr-assembly-LATEST.jar - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" databricks: # workspace instance workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' @@ -93,7 +93,7 @@ spark_config: # Feathr Job location. Support local paths, path start with http(s)://, and paths start with dbfs:/ work_dir: 'dbfs:/feathr_getting_started' # this is the default location so end users don't have to compile the runtime again. - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" online_store: redis: diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml index 003e424eb..f716da0b4 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_purview.yaml @@ -25,13 +25,13 @@ spark_config: workspace_dir: 'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_test_workspace' executor_size: 'Small' executor_num: 1 - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" online_store: redis: diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml index 0722a34d8..c842bc702 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_purview_rbac.yaml @@ -25,13 +25,13 @@ spark_config: workspace_dir: 'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_test_workspace' executor_size: 'Small' executor_num: 1 - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" online_store: redis: diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml index 507b096cb..dcb73d827 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_sql.yaml @@ -25,13 +25,13 @@ spark_config: workspace_dir: 'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_test_workspace' executor_size: 'Small' executor_num: 1 - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" online_store: redis: diff --git a/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml b/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml index 9403cb190..29c6889e8 100644 --- a/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml +++ b/feathr_project/test/test_user_workspace/feathr_config_registry_sql_rbac.yaml @@ -25,13 +25,13 @@ spark_config: workspace_dir: 'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_test_workspace' executor_size: 'Small' executor_num: 1 - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" databricks: workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net/' workspace_token_value: '' config_template: {"run_name":"FEATHR_FILL_IN","new_cluster":{"spark_version":"9.1.x-scala2.12","num_workers":1,"spark_conf":{"FEATHR_FILL_IN":"FEATHR_FILL_IN"},"instance_pool_id":"0403-214809-inlet434-pool-l9dj3kwz"},"libraries":[{"jar":"FEATHR_FILL_IN"}],"spark_jar_task":{"main_class_name":"FEATHR_FILL_IN","parameters":["FEATHR_FILL_IN"]}} work_dir: 'dbfs:/feathr_getting_started' - feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.2.jar" + feathr_runtime_location: "../../target/scala-2.12/feathr-assembly-0.8.0.jar" online_store: redis: From 6a3a0443336b28a2548fed4c2664ed78337e82a3 Mon Sep 17 00:00:00 2001 From: Richin Jain Date: Mon, 24 Oct 2022 08:07:54 +0530 Subject: [PATCH 35/68] Refactor Product Recommendation sample notebook (#743) * Adding DevSkim linter to Github actions * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Update docker-publish.yml * Removing devskim file * Restructuring the Prod Reco sample * Adjusting headings * Minor changes * Removing changes to docker publish file * Addressing PR comments, moving Product recommendation notebook sample to Synapse folder since it is strongly tied to Synapse * Addressing PR comments * Fixing images * Removing the need to pass email id as we could directly compute object Id using az command, also making CLI instructions clearer that it is for advance users --- .../product_recommendation_demo.ipynb | 316 ++++++++---------- 1 file changed, 140 insertions(+), 176 deletions(-) rename docs/samples/{ => azure_synapse}/product_recommendation_demo.ipynb (69%) diff --git a/docs/samples/product_recommendation_demo.ipynb b/docs/samples/azure_synapse/product_recommendation_demo.ipynb similarity index 69% rename from docs/samples/product_recommendation_demo.ipynb rename to docs/samples/azure_synapse/product_recommendation_demo.ipynb index 44febb062..e93860269 100644 --- a/docs/samples/product_recommendation_demo.ipynb +++ b/docs/samples/azure_synapse/product_recommendation_demo.ipynb @@ -4,26 +4,32 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "# Demo Notebook: Feathr Feature Store on Azure\n", + "# Product Recommendation with Feathr on Azure\n", "\n", "This notebook demonstrates how Feathr Feature Store can simplify and empower your model training and inference. You will learn:\n", "\n", "1. Define sharable features using Feathr API\n", - "2. Create a training dataset via point-in-time feature join with Feathr API\n", - "3. Materialize features to online store and then retrieve them with Feathr API" + "2. Register features with register API.\n", + "3. Create a training dataset via point-in-time feature join with Feathr API\n", + "4. Materialize features to online store and then retrieve them with Feathr API\n", + "\n", + "In this tutorial, we use Feathr to create a model that predicts users' product rating. " ] }, { "cell_type": "markdown", "metadata": {}, "source": [ - "## Prerequisite: Use Quick Start Template to Provision Azure Resources\n", + "## 1. Prerequisite: Use Azure Resource Manager(ARM) to Provision Azure Resources\n", "\n", "First step is to provision required cloud resources if you want to use Feathr. Feathr provides a python based client to interact with cloud resources.\n", "\n", - "Please follow the steps [here](https://feathr-ai.github.io/feathr/how-to-guides/azure-deployment-arm.html) to provision required cloud resources. Due to the complexity of the possible cloud environment, it is almost impossible to create a script that works for all the use cases. Because of this, [azure_resource_provision.sh](https://github.com/feathr-ai/feathr/blob/main/docs/how-to-guides/azure_resource_provision.sh) is a full end to end command line to create all the required resources, and you can tailor the script as needed, while [the companion documentation](https://feathr-ai.github.io/feathr/how-to-guides/azure-deployment-cli.html) can be used as a complete guide for using that shell script. \n", + "Please follow the steps [here](https://feathr-ai.github.io/feathr/how-to-guides/azure-deployment-arm.html) to provision required cloud resources. This will create a new resource group and deploy the needed Azure resources in it. \n", "\n", + "If you already have an existing resource group and only want to install few resources manually you can refer to the cli documentation [here](https://feathr-ai.github.io/feathr/how-to-guides/azure-deployment-cli.html). It provides CLI commands to install the needed resources. \n", + "**Please Note: CLI documentation is for advance users since there are lot of configurations and role assignment that would have to be done manually so it won't work out of box and should just be used for reference. ARM template is the preferred way to deploy.**\n", "\n", + "The below architecture diagram represents how different resources interact with each other\n", "![Architecture](https://github.com/feathr-ai/feathr/blob/main/docs/images/architecture.png?raw=true)" ] }, @@ -31,11 +37,9 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "## Prerequisite: Install Feathr and Import Dependencies\n", - "\n", - "Install Feathr using pip:\n", + "## 2. Prerequisite: Login to Azure and Install Feathr\n", "\n", - "`pip install -U feathr pandavro scikit-learn`" + "Login to Azure with a device code (You will see instructions in the output once you execute the cell):" ] }, { @@ -44,53 +48,14 @@ "metadata": {}, "outputs": [], "source": [ - "# Import Dependencies\n", - "import glob\n", - "import os\n", - "import tempfile\n", - "from datetime import datetime, timedelta\n", - "from math import sqrt\n", - "\n", - "import pandas as pd\n", - "import pandavro as pdx\n", - "from feathr import FeathrClient\n", - "from feathr import BOOLEAN, FLOAT, INT32, ValueType\n", - "from feathr import Feature, DerivedFeature, FeatureAnchor\n", - "from feathr import BackfillTime, MaterializationSettings\n", - "from feathr import FeatureQuery, ObservationSettings\n", - "from feathr import RedisSink\n", - "from feathr import INPUT_CONTEXT, HdfsSource\n", - "from feathr import WindowAggTransformation\n", - "from feathr import TypedKey\n", - "from sklearn.metrics import mean_squared_error\n", - "from sklearn.model_selection import train_test_split\n", - "from azure.identity import DefaultAzureCredential\n", - "from azure.keyvault.secrets import SecretClient" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Prerequisite: Configure the required environment with Feathr Quick Start Template\n", - "\n", - "In the first step (Provision cloud resources), you should have provisioned all the required cloud resources. Run the code below to install Feathr, login to Azure to get the required credentials to access more cloud resources." + "! az login --use-device-code" ] }, { "cell_type": "markdown", "metadata": {}, "source": [ - "**REQUIRED STEP: Fill in the resource prefix when provisioning the resources**" - ] - }, - { - "cell_type": "code", - "execution_count": null, - "metadata": {}, - "outputs": [], - "source": [ - "resource_prefix = \"feathr_resource_prefix\"" + "Install Feathr and dependencies to run this notebook." ] }, { @@ -99,14 +64,14 @@ "metadata": {}, "outputs": [], "source": [ - "! pip install feathr azure-cli pandavro scikit-learn" + "%pip install -U feathr pandavro scikit-learn" ] }, { "cell_type": "markdown", "metadata": {}, "source": [ - "Login to Azure with a device code (You will see instructions in the output):" + "Import Dependencies to make sure everything is installed correctly" ] }, { @@ -115,7 +80,27 @@ "metadata": {}, "outputs": [], "source": [ - "! az login --use-device-code" + "import glob\n", + "import os\n", + "import tempfile\n", + "from datetime import datetime, timedelta\n", + "from math import sqrt\n", + "\n", + "import pandas as pd\n", + "import pandavro as pdx\n", + "from feathr import FeathrClient\n", + "from feathr import BOOLEAN, FLOAT, INT32, ValueType\n", + "from feathr import Feature, DerivedFeature, FeatureAnchor\n", + "from feathr import BackfillTime, MaterializationSettings\n", + "from feathr import FeatureQuery, ObservationSettings\n", + "from feathr import RedisSink\n", + "from feathr import INPUT_CONTEXT, HdfsSource\n", + "from feathr import WindowAggTransformation\n", + "from feathr import TypedKey\n", + "from sklearn.metrics import mean_squared_error\n", + "from sklearn.model_selection import train_test_split\n", + "from azure.identity import AzureCliCredential\n", + "from azure.keyvault.secrets import SecretClient" ] }, { @@ -123,20 +108,19 @@ "metadata": {}, "source": [ "\n", - "**Permission**\n", + "## 3. Prerequisite: Set the required permissions\n", "\n", - "To proceed with the following steps, you may need additional permission: permission to access the keyvault, permission to access the Storage Blob as a Contributor and permission to submit jobs to Synapse cluster. Skip this step if you have already given yourself the access. Otherwise, run the following lines of command in the Cloud Shell before running the cell below.\n", + "Before you proceed further, you would need additional permissions: permission to access the keyvault, permission to access the Storage Blob as a Contributor and permission to submit jobs to Synapse cluster. Run the following lines of command in the [Cloud Shell](https://shell.azure.com) before running the cells below. Please replace the resource_prefix with the prefix you used in ARM template deployment.\n", "\n", "```\n", - "userId=\n", - "resource_prefix=\n", - "synapse_workspace_name=\"${resource_prefix}syws\"\n", - "keyvault_name=\"${resource_prefix}kv\"\n", - "objectId=$(az ad user show --id $userId --query id -o tsv)\n", - "az keyvault update --name $keyvault_name --enable-rbac-authorization false\n", - "az keyvault set-policy -n $keyvault_name --secret-permissions get list --object-id $objectId\n", - "az role assignment create --assignee $userId --role \"Storage Blob Data Contributor\"\n", - "az synapse role assignment create --workspace-name $synapse_workspace_name --role \"Synapse Contributor\" --assignee $userId\n", + " resource_prefix=\"YOUR_RESOURCE_PREFIX\"\n", + " synapse_workspace_name=\"${resource_prefix}syws\"\n", + " keyvault_name=\"${resource_prefix}kv\"\n", + " objectId=$(az ad signed-in-user show --query id -o tsv)\n", + " az keyvault update --name $keyvault_name --enable-rbac-authorization false\n", + " az keyvault set-policy -n $keyvault_name --secret-permissions get list --object-id $objectId\n", + " az role assignment create --assignee $userId --role \"Storage Blob Data Contributor\"\n", + " az synapse role assignment create --workspace-name $synapse_workspace_name --role \"Synapse Contributor\" --assignee $userId\n", "```\n" ] }, @@ -144,7 +128,12 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "Get all the required credentials from Azure KeyVault" + "# 4. Prerequisite: Feathr Configuration\n", + "\n", + "### Setting the environment variables\n", + "Set the environment variables that will be used by Feathr as configuration. Feathr supports configuration via enviroment variables and yaml, you can read more about it [here](https://feathr-ai.github.io/feathr/how-to-guides/feathr-configuration-and-env.html).\n", + "\n", + "**Fill in the `resource_prefix` that you used while provisioning the resources in Step 1 using ARM.**" ] }, { @@ -153,44 +142,49 @@ "metadata": {}, "outputs": [], "source": [ - "# Get all the required credentials from Azure Key Vault\n", - "key_vault_name=resource_prefix+\"kv\"\n", - "synapse_workspace_url=resource_prefix+\"syws\"\n", - "adls_account=resource_prefix+\"dls\"\n", - "adls_fs_name=resource_prefix+\"fs\"\n", - "purview_name=resource_prefix+\"purview\"\n", - "key_vault_uri = f\"https://{key_vault_name}.vault.azure.net\"\n", - "credential = DefaultAzureCredential(exclude_interactive_browser_credential=False)\n", - "client = SecretClient(vault_url=key_vault_uri, credential=credential)\n", - "secretName = \"FEATHR-ONLINE-STORE-CONN\"\n", - "retrieved_secret = client.get_secret(secretName).value\n", - "\n", - "# Get redis credentials; This is to parse Redis connection string.\n", - "redis_port=retrieved_secret.split(',')[0].split(\":\")[1]\n", - "redis_host=retrieved_secret.split(',')[0].split(\":\")[0]\n", - "redis_password=retrieved_secret.split(',')[1].split(\"password=\",1)[1]\n", - "redis_ssl=retrieved_secret.split(',')[2].split(\"ssl=\",1)[1]\n", - "\n", - "# Set the resource link\n", - "os.environ['spark_config__azure_synapse__dev_url'] = f'https://{synapse_workspace_url}.dev.azuresynapse.net'\n", - "os.environ['spark_config__azure_synapse__pool_name'] = 'spark31'\n", - "os.environ['spark_config__azure_synapse__workspace_dir'] = f'abfss://{adls_fs_name}@{adls_account}.dfs.core.windows.net/feathr_project'\n", - "os.environ['online_store__redis__host'] = redis_host\n", - "os.environ['online_store__redis__port'] = redis_port\n", - "os.environ['online_store__redis__ssl_enabled'] = redis_ssl\n", - "os.environ['REDIS_PASSWORD']=redis_password\n", - "feathr_output_path = f'abfss://{adls_fs_name}@{adls_account}.dfs.core.windows.net/feathr_output'" + "RESOURCE_PREFIX = \"YOUR_RESOURCE_PREFIX\" # from ARM deployment in Step 1\n", + "FEATHR_PROJECT_NAME=\"YOUR_PROJECT_NAME\" # provide a unique name" ] }, { - "cell_type": "markdown", + "cell_type": "code", + "execution_count": null, "metadata": {}, + "outputs": [], "source": [ - "## Prerequisite: Configure the required environment (Skip this step if using the above Quick Start Template)\n", "\n", - "In the first step (Provision cloud resources), you should have provisioned all the required cloud resources. If you use Feathr CLI to create a workspace, you should have a folder with a file called `feathr_config.yaml` in it with all the required configurations. Otherwise, update the configuration below.\n", + "# Get name for deployed resources using the resource prefix\n", + "KEY_VAULT_NAME=f\"{RESOURCE_PREFIX}kv\"\n", + "SYNAPSE_WORKSPACE_NAME=f\"{RESOURCE_PREFIX}syws\"\n", + "ADLS_ACCOUNT=f\"{RESOURCE_PREFIX}dls\"\n", + "ADLS_FS_NAME=f\"{RESOURCE_PREFIX}fs\"\n", + "KEY_VAULT_URI = f\"https://{KEY_VAULT_NAME}.vault.azure.net\"\n", + "FEATHR_API_APP = f\"{RESOURCE_PREFIX}webapp\"\n", + "\n", "\n", - "The code below will write this configuration string to a temporary location and load it to Feathr. Please still refer to [feathr_config.yaml](https://github.com/feathr-ai/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml) and use that as the source of truth. It should also have more explanations on the meaning of each variable." + "# Getting the credential object for Key Vault client\n", + "credential = AzureCliCredential()\n", + "client = SecretClient(vault_url=KEY_VAULT_URI, credential=credential)\n", + "\n", + "# Getting Redis store's connection string.\n", + "retrieved_secret = client.get_secret(\"FEATHR-ONLINE-STORE-CONN\").value\n", + "\n", + "# Parse Redis connection string\n", + "REDIS_PORT=retrieved_secret.split(',')[0].split(\":\")[1]\n", + "REDIS_HOST=retrieved_secret.split(',')[0].split(\":\")[0]\n", + "REDIS_PASSWORD=retrieved_secret.split(',')[1].split(\"password=\",1)[1]\n", + "REDIS_SSL=retrieved_secret.split(',')[2].split(\"ssl=\",1)[1]\n", + "# Set password as environment variable.\n", + "os.environ['REDIS_PASSWORD']=REDIS_PASSWORD" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "### Write the configuration as yaml file.\n", + "\n", + "The code below will write this configuration string to a temporary location and load it to Feathr. Please refer to [feathr_config.yaml](https://github.com/feathr-ai/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml) for full list of configuration options and details about them." ] }, { @@ -200,68 +194,38 @@ "outputs": [], "source": [ "import tempfile\n", - "yaml_config = \"\"\"\n", - "# Please refer to https://github.com/feathr-ai/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml for explanations on the meaning of each field.\n", + "yaml_config = f\"\"\"\n", "api_version: 1\n", "project_config:\n", - " project_name: 'feathr_getting_started'\n", - " required_environment_variables:\n", - " - 'REDIS_PASSWORD'\n", + " project_name: '{FEATHR_PROJECT_NAME}'\n", "offline_store:\n", "# Please set 'enabled' flags as true (false by default) if any of items under the same paths are expected to be visited\n", " adls:\n", " adls_enabled: true\n", " wasb:\n", " wasb_enabled: true\n", - " s3:\n", - " s3_enabled: false\n", - " s3_endpoint: 's3.amazonaws.com'\n", - " jdbc:\n", - " jdbc_enabled: false\n", - " jdbc_database: 'feathrtestdb'\n", - " jdbc_table: 'feathrtesttable'\n", - " snowflake:\n", - " snowflake_enabled: false\n", - " url: \".snowflakecomputing.com\"\n", - " user: \"\"\n", - " role: \"\"\n", "spark_config:\n", " spark_cluster: 'azure_synapse'\n", " spark_result_output_parts: '1'\n", " azure_synapse:\n", - " dev_url: 'https://feathrazuretest3synapse.dev.azuresynapse.net'\n", - " pool_name: 'spark3'\n", - " workspace_dir: 'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_getting_started'\n", + " dev_url: 'https://{SYNAPSE_WORKSPACE_NAME}.dev.azuresynapse.net'\n", + " pool_name: 'spark31'\n", + " workspace_dir: 'abfss://{ADLS_FS_NAME}@{ADLS_ACCOUNT}.dfs.core.windows.net/feathr_project'\n", " executor_size: 'Small'\n", " executor_num: 1\n", - " databricks:\n", - " workspace_instance_url: 'https://adb-2474129336842816.16.azuredatabricks.net'\n", - " config_template: {'run_name':'','new_cluster':{'spark_version':'9.1.x-scala2.12','node_type_id':'Standard_D3_v2','num_workers':2,'spark_conf':{}},'libraries':[{'jar':''}],'spark_jar_task':{'main_class_name':'','parameters':['']}}\n", - " work_dir: 'dbfs:/feathr_getting_started'\n", "online_store:\n", " redis:\n", - " host: 'feathrazuretest3redis.redis.cache.windows.net'\n", - " port: 6380\n", - " ssl_enabled: True\n", + " host: '{REDIS_HOST}'\n", + " port: {REDIS_PORT}\n", + " ssl_enabled: {REDIS_SSL}\n", "feature_registry:\n", - " api_endpoint: \"https://feathr-sql-registry.azurewebsites.net/api/v1\"\n", + " api_endpoint: 'https://{FEATHR_API_APP}.azurewebsites.net/api/v1'\n", "\"\"\"\n", + "\n", "tmp = tempfile.NamedTemporaryFile(mode='w', delete=False)\n", "with open(tmp.name, \"w\") as text_file:\n", " text_file.write(yaml_config)\n", - "feathr_output_path = f'abfss://feathrazuretest3fs@feathrazuretest3storage.dfs.core.windows.net/feathr_output'" - ] - }, - { - "cell_type": "markdown", - "metadata": {}, - "source": [ - "## Prerequisite: Setup necessary environment variables (Skip this step if using the above Quick Start Template)\n", - "\n", - "You should setup the environment variables in order to run this sample. More environment variables can be set by referring to [feathr_config.yaml](https://github.com/feathr-ai/feathr/blob/main/feathr_project/feathrcli/data/feathr_user_workspace/feathr_config.yaml) and use that as the source of truth. It also has more explanations on the meaning of each variable.\n", - "\n", - "To run this notebook, for Azure users, you need REDIS_PASSWORD.\n", - "To run this notebook, for Databricks useres, you need DATABRICKS_WORKSPACE_TOKEN_VALUE and REDIS_PASSWORD." + "feathr_output_path = f'abfss://{ADLS_FS_NAME}@{ADLS_ACCOUNT}.dfs.core.windows.net/feathr_output'" ] }, { @@ -270,10 +234,10 @@ "source": [ "# Define sharable features using Feathr API\n", "\n", - "In this tutorial, we use Feathr Feature Store to help create a model that predicts users product rating. To make it simple, let's just predict users' rating for ONE product for an e-commerce website. (We have an [advanced demo](./product_recommendation_demo_advanced.ipynb) that predicts ratings for arbitrary products.)\n", + "In this tutorial, we use Feathr Feature Store and create a model that predicts users' product rating. To make it simple, let's just predict users' rating for ONE product for an e-commerce website. (We have an [advanced demo](../product_recommendation_demo_advanced.ipynb) that predicts ratings for arbitrary products.)\n", "\n", "\n", - "## Initialize Feathr Client\n", + "### Initialize Feathr Client\n", "\n", "Let's initialize a Feathr client first. The Feathr client provides all the APIs we need to interact with Feathr Feature Store." ] @@ -291,7 +255,7 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "## Understand the Raw Datasets\n", + "### Understand the Raw Datasets\n", "We have 3 raw datasets to work with: one observation dataset(a.k.a. label dataset) and two raw datasets to generate features." ] }, @@ -305,6 +269,7 @@ "# Observation dataset usually comes with a event_timestamp to denote when the observation happened.\n", "# The label here is product_rating. Our model objective is to predict a user's rating for this product.\n", "import pandas as pd\n", + "# Public URL hosting mock data\n", "pd.read_csv(\"https://azurefeathrstorage.blob.core.windows.net/public/sample_data/product_recommendation_sample/user_observation_mock_data.csv\")" ] }, @@ -346,7 +311,7 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "## What's a Feature in Feathr\n", + "### What's a Feature in Feathr\n", "A feature is an individual measurable property or characteristic of a phenomenon which is sometimes time-sensitive. \n", "\n", "In Feathr, feature can be defined by the following characteristics:\n", @@ -360,7 +325,7 @@ "1. Feature source: what source data that this feature is based on\n", "2. Transformation: what transformation is used to transform the source data into feature. Transformation can be optional when you just want to take a column out from the source data.\n", "\n", - "(For more details on feature definition, please refer to the [Feathr Feature Definition Guide](https://github.com/feathr-ai/feathr/blob/main/docs/concepts/feature-definition.md))" + "(For more details on feature definition, please refer to the [Feathr Feature Definition Guide](https://feathr-ai.github.io/feathr/concepts/feature-definition.html))" ] }, { @@ -456,9 +421,7 @@ "source": [ "### Window aggregation features\n", "\n", - "Using [window aggregations](https://en.wikipedia.org/wiki/Window_function_%28SQL%29) can help us create more powerful features. A window aggregation feature compress large amount of information into one single feature value. Using our raw data as an example, we have the users' purchase history data that might be quite some rows, we want to create a window aggregation feature that represents their last 90 days of average purcahse amount.\n", - "\n", - "Feathr provides a nice API to help us create such window aggregation features.\n", + "Using [window aggregations](https://en.wikipedia.org/wiki/Window_function_%28SQL%29) can help us create more powerful features. A window aggregation feature compresses large amount of information into one single feature value. Using our raw data as an example, we have the user's purchase history data that might be quite some rows, we want to create a window aggregation feature that represents their last 90 days of average purchase amount.\n", "\n", "To create this window aggregation feature via Feathr, we just need to define the following parameters with `WindowAggTransformation` API:\n", "1. `agg_expr`: the field/column you want to aggregate. It can be a ANSI SQL expression. So we just write `cast_float(purchase_amount)`(the raw data might be in string form, let's cast_float).\n", @@ -509,9 +472,7 @@ "### Derived Features Section\n", "Derived features are features that are computed from other Feathr features. They could be computed from anchored features, or other derived features.\n", "\n", - "Typical usage includes feature cross(f1 * f2), or computing cosine similarity between two features.\n", - "\n", - "The syntax works in a similar way." + "Typical usage includes feature cross(f1 * f2), or computing cosine similarity between two features. The syntax works in a similar way." ] }, { @@ -532,7 +493,7 @@ "metadata": {}, "source": [ "### Build Features\n", - "Lastly, we need to build those features so that it can be consumed later. Note that we have to build both the \"anchor\" and the \"derived\" features." + "Lastly, we need to build these features so that they can be consumed later. Note that we have to build both the \"anchor\" and the \"derived\" features." ] }, { @@ -550,12 +511,11 @@ "metadata": {}, "source": [ "### Optional: A Special Type of Feature: Request Feature\n", - "For advanced user cases, in some cases, features defined on top of request data(a.k.a. observation data) may have no entity key or timestamp.\n", - "It is merely a function/transformation executing against request data at runtime.\n", - "For example, the day of week of the request, which is calculated by converting the request UNIX timestamp.\n", - "In this case, the `source` section should be `INPUT_CONTEXT` to indicate the source of those defined anchors.\n", + "Sometimes features defined on top of request data(a.k.a. observation data) may have no entity key or timestamp. It is merely a function/transformation executing against request data at runtime.\n", + "\n", + "For example, the day of the week of the request, which is calculated by converting the request UNIX timestamp. In this case, the `source` section should be `INPUT_CONTEXT` to indicate the source of those defined anchors.\n", "\n", - "We won't cover the details it in this notebook." + "We won't cover the details of it in this notebook." ] }, { @@ -564,12 +524,11 @@ "source": [ "## Create training data using point-in-time correct feature join\n", "\n", - "A training dataset usually contains entity id column(s), multiple feature columns, event timestamp column and label/target column. \n", + "A training dataset usually contains `entity id` column(s), multiple `feature` columns, event timestamp column and `label/target` column. \n", "\n", - "To create a training dataset using Feathr, we need to provide a feature join settings to specify\n", - "what features and how these features should be joined to the observation data. \n", + "To create a training dataset using Feathr, we need to provide a feature join settings to specify what features and how these features should be joined to the observation data. \n", "\n", - "(To learn more on this topic, please refer to [Point-in-time Correctness](https://github.com/feathr-ai/feathr/blob/main/docs/concepts/point-in-time-join.md))" + "(To learn more on this topic, please refer to [Point-in-time Correctness](https://feathr-ai.github.io/feathr/concepts/point-in-time-join.html))." ] }, { @@ -578,12 +537,7 @@ "metadata": {}, "outputs": [], "source": [ - "# Synapse and Databricks have different output path format\n", - "if feathr_client.spark_runtime == 'databricks':\n", - " output_path = 'dbfs:/feathrazure_test.avro'\n", - "else:\n", - " output_path = feathr_output_path\n", - "\n", + "output_path = feathr_output_path\n", "# Features that we want to request\n", "feature_query = FeatureQuery(feature_list=[\"feature_user_age\", \n", " \"feature_user_tax_rate\", \n", @@ -606,7 +560,7 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "## Download the result and show the result\n", + "### Download the result and show the result\n", "\n", "Let's use the helper function `get_result_df` to download the result and view it:" ] @@ -639,7 +593,7 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "## Train a machine learning model\n", + "### Train a machine learning model\n", "After getting all the features, let's train a machine learning model with the converted feature by Feathr:" ] }, @@ -694,16 +648,16 @@ "\n", "In the previous section, we demonstrated how Feathr can compute feature value to generate training dataset from feature definition on-they-fly.\n", "\n", - "Now let's talk about how we can use the trained models. We can use the trained models for offline inference as well as online inference. In both cases, we need features to be feed into the models. For offline inference, you can compute and get the features on-demand; or you can store the computed features to some offline database for later offline inference.\n", + "Now let's talk about how we can use the trained models. We can use the trained models for both online and offline inference. In both cases, we need features to be fed into the models. For offline inference, you can compute and get the features on-demand; or you can store the computed features to some offline database for later offline inference.\n", "\n", "For online inference, we can use Feathr to compute and store the features in the online database. Then use it for online inference when the request comes.\n", "\n", - "![img](../images/online_inference.jpg)\n", + "![img](../../images/online_inference.jpg)\n", "\n", "\n", - "In this section, we will focus on materialize features to online store. For materialization to offline store, you can check out our [user guide](https://github.com/feathr-ai/feathr/blob/main/docs/concepts/materializing-features.md#materializing-features-to-offline-store).\n", + "In this section, we will focus on materialize features to online store. For materialization to offline store, you can check out our [user guide](https://feathr-ai.github.io/feathr/concepts/materializing-features.html#materializing-features-to-offline-store).\n", "\n", - "We can push the computed features to the online store like below:" + "We can push the computed features to the online store(Redis) like below:" ] }, { @@ -729,7 +683,7 @@ "cell_type": "markdown", "metadata": {}, "source": [ - "## Fetch feature value from online store\n", + "### Fetch feature value from online store\n", "We can then get the features from the online store (Redis) via the client's `get_online_features` or `multi_get_online_features` API." ] }, @@ -761,7 +715,7 @@ "source": [ "### Registering and Fetching features\n", "\n", - "We can also register the features with an Apache Atlas compatible service, such as Azure Purview, and share the registered features across teams:" + "We can also register the features and share them across teams:" ] }, { @@ -771,13 +725,23 @@ "outputs": [], "source": [ "feathr_client.register_features()\n", - "feathr_client.list_registered_features(project_name=\"feathr_getting_started\")" + "feathr_client.list_registered_features(project_name=f\"{FEATHR_PROJECT_NAME}\")" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Summary\n", + "In this notebook you learnt how to set up Feathr and use it to create features, register features and use those features for model training and inferencing.\n", + "\n", + "We hope this example gave you a good sense of Feathr's capabilities and how you could leverage it within your organization's MLOps workflow." ] } ], "metadata": { "kernelspec": { - "display_name": "Python 3.9.12 ('ifelse_bug_env': venv)", + "display_name": "Python 3.8.13 ('feathrtest')", "language": "python", "name": "python3" }, @@ -791,11 +755,11 @@ "name": "python", "nbconvert_exporter": "python", "pygments_lexer": "ipython3", - "version": "3.9.12" + "version": "3.8.13" }, "vscode": { "interpreter": { - "hash": "6a6c366ec8f33a88299a9f856c1a3e4312616abcb6fcf46b22c3da0a923e63af" + "hash": "96bbbb728c64ae5eda27ed1c89d74908bf0652fd45caa45cd0ade6bdc0df4d48" } } }, From eb6b9b86c75cf1c7d6e7f35261ded118eb5a8587 Mon Sep 17 00:00:00 2001 From: Boli Guan Date: Tue, 25 Oct 2022 11:48:25 +0800 Subject: [PATCH 36/68] Update role-management page in UI (#751) (#764) * Update role-management page in UI (#751) * fix home.css LF file * fix RoleForm eslint warning * remove import dayjs Signed-off-by: Boli Guan * Change components to arrow function. Signed-off-by: Boli Guan Signed-off-by: Boli Guan --- ui/package-lock.json | 10 +- ui/package.json | 1 + ui/src/pages/home/home.css | 46 ++--- .../management/components/RoleForm/index.tsx | 126 ++++++++++++ .../management/components/SearchBar/index.tsx | 71 +++++++ .../components/UserRolesTable/index.tsx | 190 ++++++++++++++++++ ui/src/pages/management/management.tsx | 28 ++- ui/src/pages/management/roleManagement.tsx | 25 ++- 8 files changed, 461 insertions(+), 36 deletions(-) create mode 100644 ui/src/pages/management/components/RoleForm/index.tsx create mode 100644 ui/src/pages/management/components/SearchBar/index.tsx create mode 100644 ui/src/pages/management/components/UserRolesTable/index.tsx diff --git a/ui/package-lock.json b/ui/package-lock.json index b3a0d27d8..28bd6553c 100644 --- a/ui/package-lock.json +++ b/ui/package-lock.json @@ -13,6 +13,7 @@ "antd": "^4.20.2", "axios": "^0.27.2", "dagre": "^0.8.5", + "dayjs": "^1.11.5", "react": "^17.0.2", "react-dom": "^17.0.2", "react-flow-renderer": "^9.7.4", @@ -6353,8 +6354,9 @@ } }, "node_modules/dayjs": { - "version": "1.11.1", - "license": "MIT" + "version": "1.11.5", + "resolved": "https://registry.npmjs.org/dayjs/-/dayjs-1.11.5.tgz", + "integrity": "sha512-CAdX5Q3YW3Gclyo5Vpqkgpj8fSdLQcRuzfX6mC6Phy0nfJ0eGYOeS7m4mt2plDWLAtA4TqTakvbboHvUxfe4iA==" }, "node_modules/debug": { "version": "4.3.4", @@ -20783,7 +20785,9 @@ "version": "2.28.0" }, "dayjs": { - "version": "1.11.1" + "version": "1.11.5", + "resolved": "https://registry.npmjs.org/dayjs/-/dayjs-1.11.5.tgz", + "integrity": "sha512-CAdX5Q3YW3Gclyo5Vpqkgpj8fSdLQcRuzfX6mC6Phy0nfJ0eGYOeS7m4mt2plDWLAtA4TqTakvbboHvUxfe4iA==" }, "debug": { "version": "4.3.4", diff --git a/ui/package.json b/ui/package.json index 3a211df1f..dc8ee5e7e 100644 --- a/ui/package.json +++ b/ui/package.json @@ -8,6 +8,7 @@ "antd": "^4.20.2", "axios": "^0.27.2", "dagre": "^0.8.5", + "dayjs": "^1.11.5", "react": "^17.0.2", "react-dom": "^17.0.2", "react-flow-renderer": "^9.7.4", diff --git a/ui/src/pages/home/home.css b/ui/src/pages/home/home.css index 5c4a3b8a8..308e45367 100644 --- a/ui/src/pages/home/home.css +++ b/ui/src/pages/home/home.css @@ -1,23 +1,23 @@ -.home .ant-card { - box-shadow: 5px 8px 15px 5px rgba(208, 216, 243, 0.6); - border-radius: 8px; -} - -.home .card-meta { - display: flex; -} - -.home .card-meta .ant-card-meta-avatar { - max-width: 80px; - flex-basis: 30%; - box-sizing: border-box; -} - -.home .card-meta .ant-card-meta-avatar > span { - width: 100%; -} - -.home .card-meta .ant-card-meta-avatar svg { - width: 100%; - height: auto; -} +.home .ant-card { + box-shadow: 5px 8px 15px 5px rgba(208, 216, 243, 0.6); + border-radius: 8px; +} + +.home .card-meta { + display: flex; +} + +.home .card-meta .ant-card-meta-avatar { + max-width: 80px; + flex-basis: 30%; + box-sizing: border-box; +} + +.home .card-meta .ant-card-meta-avatar > span { + width: 100%; +} + +.home .card-meta .ant-card-meta-avatar svg { + width: 100%; + height: auto; +} diff --git a/ui/src/pages/management/components/RoleForm/index.tsx b/ui/src/pages/management/components/RoleForm/index.tsx new file mode 100644 index 000000000..9e073abd8 --- /dev/null +++ b/ui/src/pages/management/components/RoleForm/index.tsx @@ -0,0 +1,126 @@ +import React, { forwardRef, useCallback, useEffect, useState } from "react"; +import { Form, Select, Input, Button, message } from "antd"; +import { listUserRole, addUserRole } from "../../../../api"; + +export interface RoleFormProps { + getRole?: (isAdmin: boolean) => void; +} + +const { Item } = Form; +const { TextArea } = Input; + +const RoleOptions = [ + { label: "Admin", value: "admin" }, + { label: "Producer", value: "producer" }, + { label: "Consumer", value: "consumer" }, +]; + +const ValidateRule = { + scope: [{ required: true, message: "Please select scope!" }], + userName: [{ required: true, message: "Please input user name!" }], + roleName: [{ required: true, message: "Please select role name!" }], + reason: [{ required: true, message: "Please input reason!" }], +}; + +const RoleForm = (props: RoleFormProps, ref: any) => { + const [form] = Form.useForm(); + const { getRole } = props; + const [loading, setLoading] = useState(false); + + const [scopeOptions, setScopeOptions] = useState< + { label: string; value: string }[] + >([]); + + const handleFinish = useCallback( + async (values) => { + try { + setLoading(true); + await addUserRole(values); + form.resetFields(); + message.success("User role is created successfully."); + } catch { + message.error("Failed to create user role."); + } finally { + setLoading(false); + } + }, + [form] + ); + + const handleInit = useCallback(async () => { + try { + const result = await listUserRole(); + if (result.length) { + const dataset = new Set( + result.reduce( + (list: string[], item) => { + list.push(item.scope); + return list; + }, + ["global"] + ) + ); + const options = Array.from(dataset).map((item) => { + return { + label: item, + value: item, + }; + }); + setScopeOptions(options); + return true; + } else { + return false; + } + } catch { + return false; + } + }, []); + + useEffect(() => { + handleInit().then((isAdmin: boolean) => { + getRole?.(isAdmin); + }); + }, [handleInit, getRole]); + + return ( +
    + + + + +