You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by "CamilleTeruel (via GitHub)" <gi...@apache.org> on 2023/05/25 16:52:30 UTC

[GitHub] [incubator-devlake] CamilleTeruel opened a new pull request, #5286: 5250 define migration script on python side

CamilleTeruel opened a new pull request, #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286

   ### Summary
   Let python plugin define migration scripts that are transmitted to go side like regular `MigrationScript`s.
   That is to say, remote plugins now implement `PluginMigration`
   
   ### Does this close any open issues?
   Closes #5250


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] CamilleTeruel commented on a diff in pull request #5286: 5250 define migration script on python side

Posted by "CamilleTeruel (via GitHub)" <gi...@apache.org>.
CamilleTeruel commented on code in PR #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286#discussion_r1206693473


##########
backend/core/utils/json.go:
##########
@@ -0,0 +1,67 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package utils
+
+import (
+	"fmt"
+	"reflect"
+
+	"github.com/apache/incubator-devlake/core/errors"
+)
+
+type JsonObject = map[string]any
+type JsonArray = []any
+
+func GetProperty[T any](object JsonObject, key string) (T, errors.Error) {
+	property, ok := object[key]
+	if !ok {
+		return *new(T), errors.Default.New(fmt.Sprintf("Missing property %s", key))
+	}
+	return convert[T](property)
+}
+
+func GetItem[T any](array JsonArray, index int) (T, errors.Error) {
+	if index < 0 || index >= len(array) {
+		return *new(T), errors.Default.New(fmt.Sprintf("Index %d out of range", index))
+	}
+	return convert[T](array[index])
+}
+
+func convert[T any](value any) (T, errors.Error) {

Review Comment:
   Done.



##########
backend/python/pydevlake/pydevlake/migration.py:
##########
@@ -0,0 +1,113 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+
+#     http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+from typing import List, Literal, Optional, Union, Annotated
+from enum import Enum
+from datetime import datetime
+
+from pydantic import BaseModel, Field
+
+
+MIGRATION_SCRIPTS = []
+
+class Dialect(Enum):
+    MYSQL = "mysql"
+    POSTGRESQL = "postgres"
+
+
+class Execute(BaseModel):
+    type: Literal["execute"] = "execute"
+    sql: str
+    dialect: Optional[Dialect] = None
+
+
+class DropColumn(BaseModel):
+    type: Literal["drop_column"] = "drop_column"
+    table: str
+    column: str
+
+
+class DropTable(BaseModel):
+    type: Literal["drop_table"] = "drop_table"
+    table: str
+
+
+Operation = Annotated[
+    Union[Execute, DropColumn, DropTable],
+    Field(discriminator="type")
+]
+
+
+class MigrationScript(BaseModel):
+    operations: List[Operation]
+    version: int
+    name: str
+
+
+class MigrationScriptBuilder:
+    def __init__(self):
+        self.operations = []
+
+    def execute(self, sql: str, dialect: Optional[Dialect] = None):
+        """
+        Executes a raw SQL statement.
+        If dialect is specified the statement will be executed only if the db dialect matches.
+        """
+        self.operations.append(Execute(sql=sql, dialect=dialect))
+
+    def drop_column(self, table: str, column: str):
+        """
+        Drops a column from a table.
+        """
+        self.operations.append(DropColumn(table=table, column=column))
+
+    def drop_table(self, table: str):
+        """
+        Drops a table.
+        """
+        self.operations.append(DropTable(table=table))
+
+
+def migration(version: int):

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] keon94 commented on a diff in pull request #5286: 5250 define migration script on python side

Posted by "keon94 (via GitHub)" <gi...@apache.org>.
keon94 commented on code in PR #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286#discussion_r1205952102


##########
backend/server/services/remote/models/conversion_test.go:
##########
@@ -0,0 +1,153 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestGenerateSimpleField(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "integer",
+	}
+	field, err := generateStructField("i", schema, true)
+	assert.NoError(t, err)
+	assert.Equal(t, int64Type, field.Type)
+	assert.Equal(t, "I", field.Name)
+	json, ok := field.Tag.Lookup("json")
+	assert.True(t, ok)
+	assert.Equal(t, "i", json)
+	validate, ok := field.Tag.Lookup("validate")
+	assert.True(t, ok)
+	assert.Equal(t, "required", validate)
+	_, ok = field.Tag.Lookup("gorm")
+	assert.False(t, ok)
+}
+
+func TestGetGoTypeInt64(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "integer",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, int64Type, typ)
+}
+
+func TestGetGoTypeFloat64(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "number",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, float64Type, typ)
+}
+
+func TestGetGoTypeBool(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "boolean",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, boolType, typ)
+}
+
+func TestGetGoTypeString(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "string",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, stringType, typ)
+}
+
+func TestGetGoTypeTime(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":   "string",
+		"format": "date-time",
+	}
+	typ, err := getGoType(schema, true)
+	assert.NoError(t, err)
+	assert.Equal(t, timeType, typ)
+}
+
+func TestGetGoTypeTimePointer(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":   "string",
+		"format": "date-time",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, reflect.PtrTo(timeType), typ)
+}
+
+func TestGetGoTypeJsonMap(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "object",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, jsonMapType, typ)
+}
+
+func TestGetGormTagPrimaryKey(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":       "integer",
+		"primaryKey": true,
+	}
+	tag := getGormTag(schema, int64Type)
+	assert.Equal(t, "gorm:\"primaryKey\"", tag)
+}
+
+func TestGetGormTagVarChar(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":      "string",
+		"maxLength": float64(100),
+	}
+	tag := getGormTag(schema, stringType)
+	assert.Equal(t, "gorm:\"type:varchar(100)\"", tag)
+}
+
+func TestGetGormTagText(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":      "string",
+		"maxLength": float64(300),
+	}
+	tag := getGormTag(schema, stringType)
+	assert.Equal(t, "gorm:\"type:text\"", tag)
+}
+
+func TestGetGormTagStringPrimaryKey(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":       "string",
+		"primaryKey": true,
+	}
+	tag := getGormTag(schema, stringType)
+	assert.Equal(t, "gorm:\"primaryKey;type:varchar(255)\"", tag)
+}
+
+func TestGetGormTagEncDec(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":   "string",
+		"format": "password",

Review Comment:
   oh, nevermind. this is a pydantic thing. I reckon it can't be changed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] keon94 commented on a diff in pull request #5286: 5250 define migration script on python side

Posted by "keon94 (via GitHub)" <gi...@apache.org>.
keon94 commented on code in PR #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286#discussion_r1205917974


##########
backend/core/utils/json.go:
##########
@@ -0,0 +1,67 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package utils
+
+import (
+	"fmt"
+	"reflect"
+
+	"github.com/apache/incubator-devlake/core/errors"
+)
+
+type JsonObject = map[string]any
+type JsonArray = []any
+
+func GetProperty[T any](object JsonObject, key string) (T, errors.Error) {
+	property, ok := object[key]
+	if !ok {
+		return *new(T), errors.Default.New(fmt.Sprintf("Missing property %s", key))
+	}
+	return convert[T](property)
+}
+
+func GetItem[T any](array JsonArray, index int) (T, errors.Error) {
+	if index < 0 || index >= len(array) {
+		return *new(T), errors.Default.New(fmt.Sprintf("Index %d out of range", index))
+	}
+	return convert[T](array[index])
+}
+
+func convert[T any](value any) (T, errors.Error) {

Review Comment:
   I think this function can be useful outside of this file as well. I suggest you export it (e.g. call it Convert) and add a few unit tests for it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] keon94 commented on a diff in pull request #5286: 5250 define migration script on python side

Posted by "keon94 (via GitHub)" <gi...@apache.org>.
keon94 commented on code in PR #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286#discussion_r1205917974


##########
backend/core/utils/json.go:
##########
@@ -0,0 +1,67 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package utils
+
+import (
+	"fmt"
+	"reflect"
+
+	"github.com/apache/incubator-devlake/core/errors"
+)
+
+type JsonObject = map[string]any
+type JsonArray = []any
+
+func GetProperty[T any](object JsonObject, key string) (T, errors.Error) {
+	property, ok := object[key]
+	if !ok {
+		return *new(T), errors.Default.New(fmt.Sprintf("Missing property %s", key))
+	}
+	return convert[T](property)
+}
+
+func GetItem[T any](array JsonArray, index int) (T, errors.Error) {
+	if index < 0 || index >= len(array) {
+		return *new(T), errors.Default.New(fmt.Sprintf("Index %d out of range", index))
+	}
+	return convert[T](array[index])
+}
+
+func convert[T any](value any) (T, errors.Error) {

Review Comment:
   I think this function can be useful outside of this file. I suggest you export it (e.g. call it Convert) and add a few unit tests for it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] hezyin merged pull request #5286: 5250 define migration script on python side

Posted by "hezyin (via GitHub)" <gi...@apache.org>.
hezyin merged PR #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] keon94 commented on a diff in pull request #5286: 5250 define migration script on python side

Posted by "keon94 (via GitHub)" <gi...@apache.org>.
keon94 commented on code in PR #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286#discussion_r1205925745


##########
backend/python/pydevlake/pydevlake/migration.py:
##########
@@ -0,0 +1,113 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+
+#     http://www.apache.org/licenses/LICENSE-2.0
+
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+
+from typing import List, Literal, Optional, Union, Annotated
+from enum import Enum
+from datetime import datetime
+
+from pydantic import BaseModel, Field
+
+
+MIGRATION_SCRIPTS = []
+
+class Dialect(Enum):
+    MYSQL = "mysql"
+    POSTGRESQL = "postgres"
+
+
+class Execute(BaseModel):
+    type: Literal["execute"] = "execute"
+    sql: str
+    dialect: Optional[Dialect] = None
+
+
+class DropColumn(BaseModel):
+    type: Literal["drop_column"] = "drop_column"
+    table: str
+    column: str
+
+
+class DropTable(BaseModel):
+    type: Literal["drop_table"] = "drop_table"
+    table: str
+
+
+Operation = Annotated[
+    Union[Execute, DropColumn, DropTable],
+    Field(discriminator="type")
+]
+
+
+class MigrationScript(BaseModel):
+    operations: List[Operation]
+    version: int
+    name: str
+
+
+class MigrationScriptBuilder:
+    def __init__(self):
+        self.operations = []
+
+    def execute(self, sql: str, dialect: Optional[Dialect] = None):
+        """
+        Executes a raw SQL statement.
+        If dialect is specified the statement will be executed only if the db dialect matches.
+        """
+        self.operations.append(Execute(sql=sql, dialect=dialect))
+
+    def drop_column(self, table: str, column: str):
+        """
+        Drops a column from a table.
+        """
+        self.operations.append(DropColumn(table=table, column=column))
+
+    def drop_table(self, table: str):
+        """
+        Drops a table.
+        """
+        self.operations.append(DropTable(table=table))
+
+
+def migration(version: int):

Review Comment:
   could we add a second optional param for the script name? We can default it to fn.__name__ like in L99 if not provided.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] keon94 commented on a diff in pull request #5286: 5250 define migration script on python side

Posted by "keon94 (via GitHub)" <gi...@apache.org>.
keon94 commented on code in PR #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286#discussion_r1205942931


##########
backend/server/services/remote/models/conversion_test.go:
##########
@@ -0,0 +1,153 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestGenerateSimpleField(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "integer",
+	}
+	field, err := generateStructField("i", schema, true)
+	assert.NoError(t, err)
+	assert.Equal(t, int64Type, field.Type)
+	assert.Equal(t, "I", field.Name)
+	json, ok := field.Tag.Lookup("json")
+	assert.True(t, ok)
+	assert.Equal(t, "i", json)
+	validate, ok := field.Tag.Lookup("validate")
+	assert.True(t, ok)
+	assert.Equal(t, "required", validate)
+	_, ok = field.Tag.Lookup("gorm")
+	assert.False(t, ok)
+}
+
+func TestGetGoTypeInt64(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "integer",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, int64Type, typ)
+}
+
+func TestGetGoTypeFloat64(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "number",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, float64Type, typ)
+}
+
+func TestGetGoTypeBool(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "boolean",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, boolType, typ)
+}
+
+func TestGetGoTypeString(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "string",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, stringType, typ)
+}
+
+func TestGetGoTypeTime(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":   "string",
+		"format": "date-time",
+	}
+	typ, err := getGoType(schema, true)
+	assert.NoError(t, err)
+	assert.Equal(t, timeType, typ)
+}
+
+func TestGetGoTypeTimePointer(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":   "string",
+		"format": "date-time",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, reflect.PtrTo(timeType), typ)
+}
+
+func TestGetGoTypeJsonMap(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "object",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, jsonMapType, typ)
+}
+
+func TestGetGormTagPrimaryKey(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":       "integer",
+		"primaryKey": true,
+	}
+	tag := getGormTag(schema, int64Type)
+	assert.Equal(t, "gorm:\"primaryKey\"", tag)
+}
+
+func TestGetGormTagVarChar(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":      "string",
+		"maxLength": float64(100),
+	}
+	tag := getGormTag(schema, stringType)
+	assert.Equal(t, "gorm:\"type:varchar(100)\"", tag)
+}
+
+func TestGetGormTagText(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":      "string",
+		"maxLength": float64(300),
+	}
+	tag := getGormTag(schema, stringType)
+	assert.Equal(t, "gorm:\"type:text\"", tag)
+}
+
+func TestGetGormTagStringPrimaryKey(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":       "string",
+		"primaryKey": true,
+	}
+	tag := getGormTag(schema, stringType)
+	assert.Equal(t, "gorm:\"primaryKey;type:varchar(255)\"", tag)
+}
+
+func TestGetGormTagEncDec(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":   "string",
+		"format": "password",

Review Comment:
   It's only semantics, but would calling the format "encrypted" or "secret" be more general than "password"?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] keon94 commented on pull request #5286: 5250 define migration script on python side

Posted by "keon94 (via GitHub)" <gi...@apache.org>.
keon94 commented on PR #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286#issuecomment-1563437543

   Looks good overall. Left a couple of comments here and there.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [incubator-devlake] CamilleTeruel commented on a diff in pull request #5286: 5250 define migration script on python side

Posted by "CamilleTeruel (via GitHub)" <gi...@apache.org>.
CamilleTeruel commented on code in PR #5286:
URL: https://github.com/apache/incubator-devlake/pull/5286#discussion_r1206698812


##########
backend/server/services/remote/models/conversion_test.go:
##########
@@ -0,0 +1,153 @@
+/*
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to You under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+*/
+
+package models
+
+import (
+	"reflect"
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestGenerateSimpleField(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "integer",
+	}
+	field, err := generateStructField("i", schema, true)
+	assert.NoError(t, err)
+	assert.Equal(t, int64Type, field.Type)
+	assert.Equal(t, "I", field.Name)
+	json, ok := field.Tag.Lookup("json")
+	assert.True(t, ok)
+	assert.Equal(t, "i", json)
+	validate, ok := field.Tag.Lookup("validate")
+	assert.True(t, ok)
+	assert.Equal(t, "required", validate)
+	_, ok = field.Tag.Lookup("gorm")
+	assert.False(t, ok)
+}
+
+func TestGetGoTypeInt64(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "integer",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, int64Type, typ)
+}
+
+func TestGetGoTypeFloat64(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "number",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, float64Type, typ)
+}
+
+func TestGetGoTypeBool(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "boolean",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, boolType, typ)
+}
+
+func TestGetGoTypeString(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "string",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, stringType, typ)
+}
+
+func TestGetGoTypeTime(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":   "string",
+		"format": "date-time",
+	}
+	typ, err := getGoType(schema, true)
+	assert.NoError(t, err)
+	assert.Equal(t, timeType, typ)
+}
+
+func TestGetGoTypeTimePointer(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":   "string",
+		"format": "date-time",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, reflect.PtrTo(timeType), typ)
+}
+
+func TestGetGoTypeJsonMap(t *testing.T) {
+	schema := map[string]interface{}{
+		"type": "object",
+	}
+	typ, err := getGoType(schema, false)
+	assert.NoError(t, err)
+	assert.Equal(t, jsonMapType, typ)
+}
+
+func TestGetGormTagPrimaryKey(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":       "integer",
+		"primaryKey": true,
+	}
+	tag := getGormTag(schema, int64Type)
+	assert.Equal(t, "gorm:\"primaryKey\"", tag)
+}
+
+func TestGetGormTagVarChar(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":      "string",
+		"maxLength": float64(100),
+	}
+	tag := getGormTag(schema, stringType)
+	assert.Equal(t, "gorm:\"type:varchar(100)\"", tag)
+}
+
+func TestGetGormTagText(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":      "string",
+		"maxLength": float64(300),
+	}
+	tag := getGormTag(schema, stringType)
+	assert.Equal(t, "gorm:\"type:text\"", tag)
+}
+
+func TestGetGormTagStringPrimaryKey(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":       "string",
+		"primaryKey": true,
+	}
+	tag := getGormTag(schema, stringType)
+	assert.Equal(t, "gorm:\"primaryKey;type:varchar(255)\"", tag)
+}
+
+func TestGetGormTagEncDec(t *testing.T) {
+	schema := map[string]interface{}{
+		"type":   "string",
+		"format": "password",

Review Comment:
   What we can do is to provide a new type `EncStr` that has json schema prop `"encrypted": true` instead of `"format": "password"`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@devlake.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org