You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hop.apache.org by ha...@apache.org on 2021/10/05 09:27:39 UTC

[incubator-hop] branch master updated: HOP-3057 Add MDI support to Dynamic SQL row

This is an automated email from the ASF dual-hosted git repository.

hansva pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-hop.git


The following commit(s) were added to refs/heads/master by this push:
     new 5bedcf4  HOP-3057 Add MDI support to Dynamic SQL row
     new 0b50a87  Merge pull request #1106 from nadment/HOP-3057
5bedcf4 is described below

commit 5bedcf4e53d1842c4ac53f50cbfaa41d1a96633d
Author: Nicolas Adment <na...@gmail.com>
AuthorDate: Sun Oct 3 12:56:44 2021 +0200

    HOP-3057 Add MDI support to Dynamic SQL row
---
 .../transforms/dynamicsqlrow/DynamicSqlRow.java    |  10 +-
 .../dynamicsqlrow/DynamicSqlRowData.java           |   4 -
 .../dynamicsqlrow/DynamicSqlRowDialog.java         |  23 +--
 .../dynamicsqlrow/DynamicSqlRowMeta.java           | 154 +++++++++------------
 .../messages/messages_en_US.properties             |  11 +-
 .../messages/messages_fr_FR.properties             |   6 +-
 .../messages/messages_ja_JP.properties             |   4 +-
 .../messages/messages_zh_CN.properties             |   2 +-
 .../dynamicsqlrow/DynamicSqlRowMetaTest.java       |   5 +-
 9 files changed, 104 insertions(+), 115 deletions(-)

diff --git a/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRow.java b/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRow.java
index 0509c24..f76e9f3 100644
--- a/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRow.java
+++ b/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRow.java
@@ -35,9 +35,6 @@ import java.sql.ResultSet;
 
 /**
  * Run dynamic SQL. SQL is defined in a field.
- *
- * @author Samatar
- * @since 13-10-2008
  */
 public class DynamicSqlRow extends BaseTransform<DynamicSqlRowMeta, DynamicSqlRowData>
     implements ITransform<DynamicSqlRowMeta, DynamicSqlRowData> {
@@ -81,7 +78,7 @@ public class DynamicSqlRow extends BaseTransform<DynamicSqlRowMeta, DynamicSqlRo
     // get dynamic SQL statement
     String sqlTemp = getInputRowMeta().getString(rowData, data.indexOfSqlField);
     String sql = null;
-    if (meta.isVariableReplace()) {
+    if (meta.isReplaceVariables()) {
       sql = resolve(sqlTemp);
     } else {
       sql = sqlTemp;
@@ -311,6 +308,11 @@ public class DynamicSqlRow extends BaseTransform<DynamicSqlRowMeta, DynamicSqlRo
   public boolean init() {
 
     if (super.init()) {
+      
+      if (meta.getConnection() != null) {
+        meta.setDatabaseMeta(getPipelineMeta().findDatabase(meta.getConnection(), variables));
+      }
+      
       if (meta.getDatabaseMeta() == null) {
         logError(
             BaseMessages.getString(
diff --git a/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowData.java b/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowData.java
index 54c0b09..45950ee 100644
--- a/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowData.java
+++ b/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowData.java
@@ -24,10 +24,6 @@ import org.apache.hop.pipeline.transform.ITransformData;
 
 import java.util.ArrayList;
 
-/**
- * @author Matt
- * @since 24-jan-2005
- */
 public class DynamicSqlRowData extends BaseTransformData implements ITransformData {
   IRowMeta outputRowMeta;
   IRowMeta lookupRowMeta;
diff --git a/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowDialog.java b/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowDialog.java
index 23de5d1..75603b7 100644
--- a/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowDialog.java
+++ b/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowDialog.java
@@ -112,7 +112,8 @@ public class DynamicSqlRowDialog extends BaseTransformDialog implements ITransfo
     wTransformName.setLayoutData(fdTransformName);
 
     // Connection line
-    wConnection = addConnectionLine(shell, wTransformName, input.getDatabaseMeta(), lsMod);
+    DatabaseMeta databaseMeta = pipelineMeta.findDatabase(input.getConnection(), variables);
+    wConnection = addConnectionLine(shell, wTransformName, databaseMeta, lsMod);
     if (input.getDatabaseMeta() == null && pipelineMeta.nrDatabases() == 1) {
       wConnection.select(0);
     }
@@ -134,7 +135,7 @@ public class DynamicSqlRowDialog extends BaseTransformDialog implements ITransfo
     FormData fdSqlFieldName = new FormData();
     fdSqlFieldName.left = new FormAttachment(middle, 0);
     fdSqlFieldName.top = new FormAttachment(wConnection, 2 * margin);
-    fdSqlFieldName.right = new FormAttachment(100, -margin);
+    fdSqlFieldName.right = new FormAttachment(100, 0);
     wSqlFieldName.setLayoutData(fdSqlFieldName);
     wSqlFieldName.addFocusListener(
         new FocusListener() {
@@ -281,7 +282,7 @@ public class DynamicSqlRowDialog extends BaseTransformDialog implements ITransfo
     FormData fdSql = new FormData();
     fdSql.left = new FormAttachment(0, 0);
     fdSql.top = new FormAttachment(wlSql, margin);
-    fdSql.right = new FormAttachment(100, -2 * margin);
+    fdSql.right = new FormAttachment(100, 0);
     fdSql.bottom = new FormAttachment(wlPosition, -margin);
     wSql.setLayoutData(fdSql);
 
@@ -355,13 +356,15 @@ public class DynamicSqlRowDialog extends BaseTransformDialog implements ITransfo
     wSql.setText(Const.NVL(input.getSql(), ""));
     wLimit.setText("" + input.getRowLimit());
     wOuter.setSelection(input.isOuterJoin());
-    wuseVars.setSelection(input.isVariableReplace());
+    wuseVars.setSelection(input.isReplaceVariables());
     if (input.getSqlFieldName() != null) {
       wSqlFieldName.setText(input.getSqlFieldName());
     }
     wqueryOnlyOnChange.setSelection(input.isQueryOnlyOnChange());
-    if (input.getDatabaseMeta() != null) {
-      wConnection.setText(input.getDatabaseMeta().getName());
+
+    if (input.getConnection() != null) {
+      wConnection.setText(input.getConnection());
+      input.setDatabaseMeta(pipelineMeta.findDatabase(input.getConnection(), variables));
     }
 
     wTransformName.selectAll();
@@ -378,18 +381,18 @@ public class DynamicSqlRowDialog extends BaseTransformDialog implements ITransfo
     if (Utils.isEmpty(wTransformName.getText())) {
       return;
     }
-
+    
+    input.setConnection(wConnection.getText());
     input.setRowLimit(Const.toInt(wLimit.getText(), 0));
     input.setSql(wSql.getText());
     input.setSqlFieldName(wSqlFieldName.getText());
     input.setOuterJoin(wOuter.getSelection());
-    input.setVariableReplace(wuseVars.getSelection());
+    input.setReplaceVariables(wuseVars.getSelection());
     input.setQueryOnlyOnChange(wqueryOnlyOnChange.getSelection());
-    input.setDatabaseMeta(pipelineMeta.findDatabase(wConnection.getText()));
 
     transformName = wTransformName.getText(); // return value
 
-    if (pipelineMeta.findDatabase(wConnection.getText()) == null) {
+    if (pipelineMeta.findDatabase(wConnection.getText(),variables) == null) {
       MessageBox mb = new MessageBox(shell, SWT.OK | SWT.ICON_ERROR);
       mb.setMessage(
           BaseMessages.getString(PKG, "DynamicSQLRowDialog.InvalidConnection.DialogMessage"));
diff --git a/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowMeta.java b/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowMeta.java
index 9077f37..59c0324 100644
--- a/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowMeta.java
+++ b/plugins/transforms/dynamicsqlrow/src/main/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowMeta.java
@@ -26,13 +26,12 @@ import org.apache.hop.core.database.DatabaseMeta;
 import org.apache.hop.core.exception.HopDatabaseException;
 import org.apache.hop.core.exception.HopException;
 import org.apache.hop.core.exception.HopTransformException;
-import org.apache.hop.core.exception.HopXmlException;
 import org.apache.hop.core.row.IRowMeta;
 import org.apache.hop.core.row.IValueMeta;
 import org.apache.hop.core.util.Utils;
 import org.apache.hop.core.variables.IVariables;
-import org.apache.hop.core.xml.XmlHandler;
 import org.apache.hop.i18n.BaseMessages;
+import org.apache.hop.metadata.api.HopMetadataProperty;
 import org.apache.hop.metadata.api.IHopMetadataProvider;
 import org.apache.hop.pipeline.DatabaseImpact;
 import org.apache.hop.pipeline.Pipeline;
@@ -40,15 +39,14 @@ import org.apache.hop.pipeline.PipelineMeta;
 import org.apache.hop.pipeline.transform.BaseTransformMeta;
 import org.apache.hop.pipeline.transform.ITransformMeta;
 import org.apache.hop.pipeline.transform.TransformMeta;
-import org.w3c.dom.Node;
 
 import java.util.List;
 
 @Transform(
     id = "DynamicSqlRow",
     image = "dynamicsqlrow.svg",
-    name = "i18n::BaseTransform.TypeLongDesc.DynamicSQLRow",
-    description = "i18n::BaseTransform.TypeTooltipDesc.DynamicSQLRow",
+    name = "i18n::DynamicSQLRow.Name",
+    description = "i18n::DynamicSQLRow.Description",
     categoryDescription = "i18n:org.apache.hop.pipeline.transform:BaseTransform.Category.Lookup",
     documentationUrl = "/pipeline/transforms/dynamicsqlrow.html")
 public class DynamicSqlRowMeta extends BaseTransformMeta
@@ -56,30 +54,61 @@ public class DynamicSqlRowMeta extends BaseTransformMeta
   private static final Class<?> PKG = DynamicSqlRowMeta.class; // For Translator
 
   /** database connection */
+  @HopMetadataProperty(
+      key = "connection",
+      injectionKeyDescription = "DynamicSQLRow.Injection.Connection")
+  private String connection;
+
   private DatabaseMeta databaseMeta;
 
   /** SQL Statement */
+  @HopMetadataProperty(key = "sql", injectionKeyDescription = "DynamicSQLRow.Injection.Sql")
   private String sql;
 
-  private String sqlfieldname;
+  @HopMetadataProperty(key = "sql_fieldname", injectionKeyDescription = "DynamicSQLRow.Injection.SqlFieldName")
+  private String sqlFieldName;
 
   /** Number of rows to return (0=ALL) */
+  @HopMetadataProperty(key = "rowlimit", injectionKeyDescription = "DynamicSQLRow.Injection.RowLimit")
   private int rowLimit;
 
   /**
    * false: don't return rows where nothing is found true: at least return one source row, the rest
    * is NULL
    */
+  @HopMetadataProperty(key = "outer_join", injectionKeyDescription = "DynamicSQLRow.Injection.OuterJoin")
   private boolean outerJoin;
 
-  private boolean replacevars;
+  @HopMetadataProperty(key = "replace_vars", injectionKeyDescription = "DynamicSQLRow.Injection.ReplaceVariables")
+  private boolean replaceVariables;
 
-  public boolean queryonlyonchange;
+  @HopMetadataProperty(key = "query_only_on_change", injectionKeyDescription = "DynamicSQLRow.Injection.QueryOnlyOnChange")
+  private boolean queryOnlyOnChange;
 
   public DynamicSqlRowMeta() {
     super(); // allocate BaseTransformMeta
   }
 
+  public DynamicSqlRowMeta(DynamicSqlRowMeta meta) {
+    super(); 
+    this.connection = meta.connection;
+    this.sql = meta.sql;
+    this.sqlFieldName = meta.sqlFieldName;
+    this.replaceVariables = meta.replaceVariables;
+    this.rowLimit =  meta.rowLimit;
+    this.connection = meta.connection;
+    this.outerJoin = meta.outerJoin;
+    this.queryOnlyOnChange = meta.queryOnlyOnChange;
+  }
+
+  public String getConnection() {
+    return connection;
+  }
+
+  public void setConnection(String connection) {
+    this.connection = connection;
+  }
+  
   /** @return Returns the database. */
   public DatabaseMeta getDatabaseMeta() {
     return databaseMeta;
@@ -101,23 +130,23 @@ public class DynamicSqlRowMeta extends BaseTransformMeta
   }
 
   /** @return Returns the replacevars. */
-  public boolean isVariableReplace() {
-    return replacevars;
+  public boolean isReplaceVariables() {
+    return replaceVariables;
   }
 
   /** @param replacevars The replacevars to set. */
-  public void setVariableReplace(boolean replacevars) {
-    this.replacevars = replacevars;
+  public void setReplaceVariables(boolean replaceVariables) {
+    this.replaceVariables = replaceVariables;
   }
 
   /** @return Returns the queryonlyonchange. */
   public boolean isQueryOnlyOnChange() {
-    return queryonlyonchange;
+    return queryOnlyOnChange;
   }
 
   /** @param queryonlyonchange The queryonlyonchange to set. */
   public void setQueryOnlyOnChange(boolean queryonlyonchange) {
-    this.queryonlyonchange = queryonlyonchange;
+    this.queryOnlyOnChange = queryonlyonchange;
   }
 
   /** @return Returns the rowLimit. */
@@ -142,56 +171,29 @@ public class DynamicSqlRowMeta extends BaseTransformMeta
 
   /** @return Returns the sqlfieldname. */
   public String getSqlFieldName() {
-    return sqlfieldname;
+    return sqlFieldName;
   }
 
   /** @param sqlfieldname The sqlfieldname to set. */
   public void setSqlFieldName(String sqlfieldname) {
-    this.sqlfieldname = sqlfieldname;
-  }
-
-  @Override
-  public void loadXml(Node transformNode, IHopMetadataProvider metadataProvider)
-      throws HopXmlException {
-    readData(transformNode, metadataProvider);
+    this.sqlFieldName = sqlfieldname;
   }
 
   @Override
   public Object clone() {
-    DynamicSqlRowMeta retval = (DynamicSqlRowMeta) super.clone();
-
-    return retval;
-  }
-
-  private void readData(Node transformNode, IHopMetadataProvider metadataProvider)
-      throws HopXmlException {
-    try {
-      String con = XmlHandler.getTagValue(transformNode, "connection");
-      databaseMeta = DatabaseMeta.loadDatabase(metadataProvider, con);
-      sql = XmlHandler.getTagValue(transformNode, "sql");
-      outerJoin = "Y".equalsIgnoreCase(XmlHandler.getTagValue(transformNode, "outer_join"));
-      replacevars = "Y".equalsIgnoreCase(XmlHandler.getTagValue(transformNode, "replace_vars"));
-      queryonlyonchange =
-          "Y".equalsIgnoreCase(XmlHandler.getTagValue(transformNode, "query_only_on_change"));
-
-      rowLimit = Const.toInt(XmlHandler.getTagValue(transformNode, "rowlimit"), 0);
-      sqlfieldname = XmlHandler.getTagValue(transformNode, "sql_fieldname");
-
-    } catch (Exception e) {
-      throw new HopXmlException(
-          BaseMessages.getString(PKG, "DynamicSQLRowMeta.Exception.UnableToLoadTransformMeta"), e);
-    }
-  }
+    return new DynamicSqlRowMeta(this);
+  }  
 
   @Override
   public void setDefault() {
-    databaseMeta = null;
-    rowLimit = 0;
-    sql = "";
-    outerJoin = false;
-    replacevars = false;
-    sqlfieldname = null;
-    queryonlyonchange = false;
+    this.connection = null;
+    this.databaseMeta = null;
+    this.rowLimit = 0;
+    this.sql = "";
+    this.outerJoin = false;
+    this.replaceVariables = false;
+    this.sqlFieldName = null;
+    this.queryOnlyOnChange = false;
   }
 
   @Override
@@ -215,7 +217,7 @@ public class DynamicSqlRowMeta extends BaseTransformMeta
     // See if it's in the cache...
     IRowMeta add = null;
     String realSql = sql;
-    if (replacevars) {
+    if (replaceVariables) {
       realSql = variables.resolve(realSql);
     }
     try {
@@ -253,24 +255,6 @@ public class DynamicSqlRowMeta extends BaseTransformMeta
   }
 
   @Override
-  public String getXml() {
-    StringBuilder retval = new StringBuilder();
-
-    retval.append(
-        "    "
-            + XmlHandler.addTagValue(
-                "connection", databaseMeta == null ? "" : databaseMeta.getName()));
-    retval.append("    " + XmlHandler.addTagValue("rowlimit", rowLimit));
-    retval.append("    " + XmlHandler.addTagValue("sql", sql));
-    retval.append("    " + XmlHandler.addTagValue("outer_join", outerJoin));
-    retval.append("    " + XmlHandler.addTagValue("replace_vars", replacevars));
-    retval.append("    " + XmlHandler.addTagValue("sql_fieldname", sqlfieldname));
-    retval.append("    " + XmlHandler.addTagValue("query_only_on_change", queryonlyonchange));
-
-    return retval.toString();
-  }
-
-  @Override
   public void check(
       List<ICheckResult> remarks,
       PipelineMeta pipelineMeta,
@@ -288,44 +272,44 @@ public class DynamicSqlRowMeta extends BaseTransformMeta
     if (input.length > 0) {
       cr =
           new CheckResult(
-              CheckResult.TYPE_RESULT_OK,
+              ICheckResult.TYPE_RESULT_OK,
               BaseMessages.getString(PKG, "DynamicSQLRowMeta.CheckResult.ReceivingInfo"),
               transformMeta);
       remarks.add(cr);
     } else {
       cr =
           new CheckResult(
-              CheckResult.TYPE_RESULT_ERROR,
+              ICheckResult.TYPE_RESULT_ERROR,
               BaseMessages.getString(PKG, "DynamicSQLRowMeta.CheckResult.NoInputReceived"),
               transformMeta);
       remarks.add(cr);
     }
 
     // Check for SQL field
-    if (Utils.isEmpty(sqlfieldname)) {
+    if (Utils.isEmpty(sqlFieldName)) {
       cr =
           new CheckResult(
-              CheckResult.TYPE_RESULT_ERROR,
+              ICheckResult.TYPE_RESULT_ERROR,
               BaseMessages.getString(PKG, "DynamicSQLRowMeta.CheckResult.SQLFieldNameMissing"),
               transformMeta);
       remarks.add(cr);
     } else {
-      IValueMeta vfield = prev.searchValueMeta(sqlfieldname);
+      IValueMeta vfield = prev.searchValueMeta(sqlFieldName);
       if (vfield == null) {
         cr =
             new CheckResult(
-                CheckResult.TYPE_RESULT_ERROR,
+                ICheckResult.TYPE_RESULT_ERROR,
                 BaseMessages.getString(
-                    PKG, "DynamicSQLRowMeta.CheckResult.SQLFieldNotFound", sqlfieldname),
+                    PKG, "DynamicSQLRowMeta.CheckResult.SQLFieldNotFound", sqlFieldName),
                 transformMeta);
       } else {
         cr =
             new CheckResult(
-                CheckResult.TYPE_RESULT_OK,
+                ICheckResult.TYPE_RESULT_OK,
                 BaseMessages.getString(
                     PKG,
                     "DynamicSQLRowMeta.CheckResult.SQLFieldFound",
-                    sqlfieldname,
+                    sqlFieldName,
                     vfield.getOrigin()),
                 transformMeta);
       }
@@ -346,14 +330,14 @@ public class DynamicSqlRowMeta extends BaseTransformMeta
           if (r != null) {
             cr =
                 new CheckResult(
-                    CheckResult.TYPE_RESULT_OK,
+                    ICheckResult.TYPE_RESULT_OK,
                     BaseMessages.getString(PKG, "DynamicSQLRowMeta.CheckResult.QueryOK"),
                     transformMeta);
             remarks.add(cr);
           } else {
             errorMessage =
                 BaseMessages.getString(PKG, "DynamicSQLRowMeta.CheckResult.InvalidDBQuery");
-            cr = new CheckResult(CheckResult.TYPE_RESULT_ERROR, errorMessage, transformMeta);
+            cr = new CheckResult(ICheckResult.TYPE_RESULT_ERROR, errorMessage, transformMeta);
             remarks.add(cr);
           }
         }
@@ -361,14 +345,14 @@ public class DynamicSqlRowMeta extends BaseTransformMeta
         errorMessage =
             BaseMessages.getString(PKG, "DynamicSQLRowMeta.CheckResult.ErrorOccurred")
                 + e.getMessage();
-        cr = new CheckResult(CheckResult.TYPE_RESULT_ERROR, errorMessage, transformMeta);
+        cr = new CheckResult(ICheckResult.TYPE_RESULT_ERROR, errorMessage, transformMeta);
         remarks.add(cr);
       } finally {
         db.disconnect();
       }
     } else {
       errorMessage = BaseMessages.getString(PKG, "DynamicSQLRowMeta.CheckResult.InvalidConnection");
-      cr = new CheckResult(CheckResult.TYPE_RESULT_ERROR, errorMessage, transformMeta);
+      cr = new CheckResult(ICheckResult.TYPE_RESULT_ERROR, errorMessage, transformMeta);
       remarks.add(cr);
     }
   }
diff --git a/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_en_US.properties b/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_en_US.properties
index 8adf4b0..02bfc9b 100644
--- a/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_en_US.properties
+++ b/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_en_US.properties
@@ -19,14 +19,14 @@ DynamicSQLRowDialog.Log.ParametersFound=Found
 DynamicSQLRowDialog.Shell.Title=Dynamic SQL row
 DynamicSQLRow.Log.ConnectedToDB=Connected to database...
 DynamicSQLRowDialog.SQL.Label=Template SQL (to retrieve Meta data) 
-BaseTransform.TypeLongDesc.DynamicSQLRow=Dynamic SQL row
+DynamicSQLRow.Name=Dynamic SQL row
 DynamicSQLRow.Exception.SQLEmpty=Template SQL statement is empty\!
 DynamicSQLRowDialog.queryOnlyOnChangejoin.Label=Query only on parameters change
 DynamicSQLRowMeta.CheckResult.NoInputReceived=No input received from other transforms\!
 DynamicSQLRow.Exception.IncorrectNrTemplateFields=The number of fields returned by the specified template ({0}) is not the same as those from the input SQL ({1}) \: ''{2}''
 DynamicSqlRowDialog.Position.Label=Position
 DynamicSQLRowDialog.FailedToGetFields.DialogMessage=Unable to get fields from previous transforms because of an error
-BaseTransform.TypeTooltipDesc.DynamicSQLRow=Execute dynamic SQL statement build in a previous field
+DynamicSQLRow.Description=Execute dynamic SQL statement build in a previous field
 DynamicSQLRowMeta.CheckResult.ReceivingInfo=Transform is receiving info from other transforms.
 DynamicSQLRow.Exception.TemplateReturnDataTypeError=The data type of template SQL result field ''{0}'' does not match with the input SQL field ''{1}''
 DynamicSQLRowDialog.useVarsjoin.Label=Replace variables
@@ -64,3 +64,10 @@ DynamicSQLRowMeta.Log.DatabaseErrorOccurred=A database error occurred\:
 DynamicSQLRowMeta.CheckResult.QueryOK=The query is specified and working correctly.
 DynamicSQLRowDialog.Outerjoin.Tooltip=Return al least the input row and add NULLs for the lookup values
 DynamicSQLRowDialog.InvalidConnection.DialogTitle=ERROR
+DynamicSQLRow.Injection.Connection=Connection name
+DynamicSQLRow.Injection.Sql=Template SQL (to retrieve Metadata)
+DynamicSQLRow.Injection.SqlFieldName=SQL field name
+DynamicSQLRow.Injection.RowLimit=Number of rows to return
+DynamicSQLRow.Injection.OuterJoin=Outer join?
+DynamicSQLRow.Injection.ReplaceVariables=Replace variables
+DynamicSQLRow.Injection.QueryOnlyOnChange=Query only on parameters change
\ No newline at end of file
diff --git a/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_fr_FR.properties b/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_fr_FR.properties
index dd0df66..4981f14 100644
--- a/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_fr_FR.properties
+++ b/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_fr_FR.properties
@@ -17,8 +17,8 @@
 #
 #
 #
-BaseTransform.TypeLongDesc.DynamicSQLRow=Interrogation dynamique d''une BdD
-BaseTransform.TypeTooltipDesc.DynamicSQLRow=Interrogation dynamique d''une Base de donn\u00E9es
+DynamicSQLRow.Name=Interrogation dynamique d''une BdD
+DynamicSQLRow.Description=Interrogation dynamique d''une Base de donn\u00E9es
 DynamicSQLRowDialog.Position.Label=Ligne \: {0} Colonne \: {1}
 DynamicSQLRowDialog.Shell.Title=Interrogation dynamique d''une BdD
 DynamicSQLRowMeta.Exception.UnableToLoadTransformMeta=Impossible de charger les infos de la transformation depuis le fichier XML
@@ -38,7 +38,7 @@ DynamicSQLRowMeta.CheckResult.InvalidDBQuery=Impossible de v\u00E9rifier la requ
 DynamicSQLRowDialog.Log.GettingKeyInfo=r\u00E9cup\u00E9ration infos cl\u00E9...
 DynamicSQLRow.Log.StartingToRun=D\u00E9marrage...
 DynamicSQLRowMeta.CheckResult.InvalidConnection=Veuillez svp s\u00E9lectionner ou le cas \u00E9ch\u00E9ant cr\u00E9er une connexion\!
-DynamicSQLRowDialog.TransformName.Label=Nom de la transformation 
+DynamicSQLRowDialog.TransformName.Label=Nom de la transformation
 DynamicSQLRowMeta.DatabaseImpact.Title=Interroger une ou plusiers base de donn\u00E9es via une requ\u00EAte SQL
 DynamicSQLRow.Exception.FieldNotFound=Le champ [{0}] est introuvable dans le flux d''entr\u00E9e\!
 DynmaicSQLRow.Init.ConnectionMissing=La connexion \u00E0 la base de donn\u00E9es n''a pas \u00E9t\u00E9 d\u00E9finie ou est introuvable pour la transformation [{0}]\!
diff --git a/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_ja_JP.properties b/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_ja_JP.properties
index aaca72e..90e5a65 100644
--- a/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_ja_JP.properties
+++ b/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_ja_JP.properties
@@ -17,8 +17,8 @@
 #
 #
 #
-BaseTransform.TypeLongDesc.DynamicSQLRow=\u52D5\u7684SQL
-BaseTransform.TypeTooltipDesc.DynamicSQLRow=Dynamic SQL row\n\u30C7\u30FC\u30BF\u30D9\u30FC\u30B9\u30D5\u30A3\u30FC\u30EB\u30C9\u3067\u4F5C\u6210\u3055\u308C\u305FSQL\u6587\u3092\u5B9F\u884C\u3057\u307E\u3059\u3002
+DynamicSQLRow.Name=\u52D5\u7684SQL
+DynamicSQLRow.Description=Dynamic SQL row\n\u30C7\u30FC\u30BF\u30D9\u30FC\u30B9\u30D5\u30A3\u30FC\u30EB\u30C9\u3067\u4F5C\u6210\u3055\u308C\u305FSQL\u6587\u3092\u5B9F\u884C\u3057\u307E\u3059\u3002
 DynamicSQLRowDialog.Position.Label=\u30e9\u30a4\u30f3: {0} \u30ab\u30e9\u30e0 : {1}
 DynamicSQLRowDialog.Shell.Title=\u52d5\u7684SQL
 DynamicSQLRowDialog.Log.ParametersFound=Found
diff --git a/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_zh_CN.properties b/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_zh_CN.properties
index 1d377e7..0281744 100644
--- a/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_zh_CN.properties
+++ b/plugins/transforms/dynamicsqlrow/src/main/resources/org/apache/hop/pipeline/transforms/dynamicsqlrow/messages/messages_zh_CN.properties
@@ -17,4 +17,4 @@
 #
 #
 #
-BaseTransform.TypeLongDesc.DynamicSQLRow=\u6267\u884CDynamic SQL
\ No newline at end of file
+DynamicSQLRow.Name=\u6267\u884CDynamic SQL
\ No newline at end of file
diff --git a/plugins/transforms/dynamicsqlrow/src/test/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowMetaTest.java b/plugins/transforms/dynamicsqlrow/src/test/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowMetaTest.java
index b73ef6d..6ef3583 100644
--- a/plugins/transforms/dynamicsqlrow/src/test/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowMetaTest.java
+++ b/plugins/transforms/dynamicsqlrow/src/test/java/org/apache/hop/pipeline/transforms/dynamicsqlrow/DynamicSqlRowMetaTest.java
@@ -21,7 +21,6 @@ import org.apache.hop.core.exception.HopException;
 import org.apache.hop.core.plugins.PluginRegistry;
 import org.apache.hop.junit.rules.RestoreHopEngineEnvironment;
 import org.apache.hop.pipeline.transforms.loadsave.LoadSaveTester;
-import org.apache.hop.pipeline.transforms.loadsave.validator.DatabaseMetaLoadSaveValidator;
 import org.apache.hop.pipeline.transforms.loadsave.validator.IFieldLoadSaveValidator;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -43,13 +42,11 @@ public class DynamicSqlRowMetaTest {
     PluginRegistry.init(false);
     List<String> attributes =
         Arrays.asList(
-            "sql", "sqlFieldName", "rowLimit", "outerJoin", "variableReplace", "databaseMeta");
+            "sql", "sqlFieldName", "rowLimit", "outerJoin", "replaceVariables", "connection");
 
     Map<String, String> getterMap = new HashMap<>();
     Map<String, String> setterMap = new HashMap<>();
     Map<String, IFieldLoadSaveValidator<?>> attrValidatorMap = new HashMap<>();
-    attrValidatorMap.put("databaseMeta", new DatabaseMetaLoadSaveValidator());
-
     Map<String, IFieldLoadSaveValidator<?>> typeValidatorMap = new HashMap<>();
 
     loadSaveTester =