Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-5498

Connect validation API stops returning recommendations for some fields after the right sequence of requests

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 0.11.0.0
    • connect
    • None

    Description

      If you issue the right sequence of requests against this API, it starts behaving differently, omitting certain fields (at a minimum recommended values, which is how I noticed this). If you start with

      $ curl -X PUT -H "Content-Type: application/json" --data '{"connector.class": "org.apache.kafka.connect.file.FileStreamSourceConnector", "name": "file", "transforms": "foo"}' http://localhost:8083/connector-plugins/org.apache.kafka.connect.file.FileStreamSourceConnector/config/validate  | jq
        % Total    % Received % Xferd  Average Speed   Time    Time     Time  Current
                                       Dload  Upload   Total   Spent    Left  Speed
      100  5845  100  5730  100   115  36642    735 --:--:-- --:--:-- --:--:-- 36496
      {
        "name": "org.apache.kafka.connect.file.FileStreamSourceConnector",
        "error_count": 4,
        "groups": [
          "Common",
          "Transforms",
          "Transforms: foo"
        ],
        "configs": [
          {
            "definition": {
              "name": "name",
              "type": "STRING",
              "required": true,
              "default_value": null,
              "importance": "HIGH",
              "documentation": "Globally unique name to use for this connector.",
              "group": "Common",
              "width": "MEDIUM",
              "display_name": "Connector name",
              "dependents": [],
              "order": 1
            },
            "value": {
              "name": "name",
              "value": "file",
              "recommended_values": [],
              "errors": [],
              "visible": true
            }
          },
          {
            "definition": {
              "name": "connector.class",
              "type": "STRING",
              "required": true,
              "default_value": null,
              "importance": "HIGH",
              "documentation": "Name or alias of the class for this connector. Must be a subclass of org.apache.kafka.connect.connector.Connector. If the connector is org.apache.kafka.connect.file.FileStreamSinkConnector, you can either specify this full name,  or use \"FileStreamSink\" or \"FileStreamSinkConnector\" to make the configuration a bit shorter",
              "group": "Common",
              "width": "LONG",
              "display_name": "Connector class",
              "dependents": [],
              "order": 2
            },
            "value": {
              "name": "connector.class",
              "value": "org.apache.kafka.connect.file.FileStreamSourceConnector",
              "recommended_values": [],
              "errors": [],
              "visible": true
            }
          },
          {
            "definition": {
              "name": "tasks.max",
              "type": "INT",
              "required": false,
              "default_value": "1",
              "importance": "HIGH",
              "documentation": "Maximum number of tasks to use for this connector.",
              "group": "Common",
              "width": "SHORT",
              "display_name": "Tasks max",
              "dependents": [],
              "order": 3
            },
            "value": {
              "name": "tasks.max",
              "value": "1",
              "recommended_values": [],
              "errors": [],
              "visible": true
            }
          },
          {
            "definition": {
              "name": "key.converter",
              "type": "CLASS",
              "required": false,
              "default_value": null,
              "importance": "LOW",
              "documentation": "Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the keys in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.",
              "group": "Common",
              "width": "SHORT",
              "display_name": "Key converter class",
              "dependents": [],
              "order": 4
            },
            "value": {
              "name": "key.converter",
              "value": null,
              "recommended_values": [],
              "errors": [],
              "visible": true
            }
          },
          {
            "definition": {
              "name": "value.converter",
              "type": "CLASS",
              "required": false,
              "default_value": null,
              "importance": "LOW",
              "documentation": "Converter class used to convert between Kafka Connect format and the serialized form that is written to Kafka. This controls the format of the values in messages written to or read from Kafka, and since this is independent of connectors it allows any connector to work with any serialization format. Examples of common formats include JSON and Avro.",
              "group": "Common",
              "width": "SHORT",
              "display_name": "Value converter class",
              "dependents": [],
              "order": 5
            },
            "value": {
              "name": "value.converter",
              "value": null,
              "recommended_values": [],
              "errors": [],
              "visible": true
            }
          },
          {
            "definition": {
              "name": "transforms",
              "type": "LIST",
              "required": false,
              "default_value": null,
              "importance": "LOW",
              "documentation": "Aliases for the transformations to be applied to records.",
              "group": "Transforms",
              "width": "LONG",
              "display_name": "Transforms",
              "dependents": [],
              "order": 6
            },
            "value": {
              "name": "transforms",
              "value": "foo",
              "recommended_values": [],
              "errors": [],
              "visible": true
            }
          },
          {
            "definition": {
              "name": "transforms.foo.type",
              "type": "CLASS",
              "required": true,
              "default_value": null,
              "importance": "HIGH",
              "documentation": "Class for the 'foo' transformation.",
              "group": "Transforms: foo",
              "width": "LONG",
              "display_name": "Transformation type for foo",
              "dependents": [],
              "order": 0
            },
            "value": {
              "name": "transforms.foo.type",
              "value": null,
              "recommended_values": [
                "org.apache.kafka.connect.transforms.Cast$Key",
                "org.apache.kafka.connect.transforms.Cast$Value",
                "org.apache.kafka.connect.transforms.ExtractField$Key",
                "org.apache.kafka.connect.transforms.ExtractField$Value",
                "org.apache.kafka.connect.transforms.Flatten$Key",
                "org.apache.kafka.connect.transforms.Flatten$Value",
                "org.apache.kafka.connect.transforms.HoistField$Key",
                "org.apache.kafka.connect.transforms.HoistField$Value",
                "org.apache.kafka.connect.transforms.InsertField$Key",
                "org.apache.kafka.connect.transforms.InsertField$Value",
                "org.apache.kafka.connect.transforms.MaskField$Key",
                "org.apache.kafka.connect.transforms.MaskField$Value",
                "org.apache.kafka.connect.transforms.RegexRouter",
                "org.apache.kafka.connect.transforms.ReplaceField$Key",
                "org.apache.kafka.connect.transforms.ReplaceField$Value",
                "org.apache.kafka.connect.transforms.SetSchemaMetadata$Key",
                "org.apache.kafka.connect.transforms.SetSchemaMetadata$Value",
                "org.apache.kafka.connect.transforms.TimestampConverter$Key",
                "org.apache.kafka.connect.transforms.TimestampConverter$Value",
                "org.apache.kafka.connect.transforms.TimestampRouter",
                "org.apache.kafka.connect.transforms.ValueToKey"
              ],
              "errors": [
                "Missing required configuration \"transforms.foo.type\" which has no default value.",
                "Invalid value null for configuration transforms.foo.type: Not a Transformation"
              ],
              "visible": true
            }
          },
          {
            "definition": {
              "name": "file",
              "type": "STRING",
              "required": true,
              "default_value": null,
              "importance": "HIGH",
              "documentation": "Source filename.",
              "group": null,
              "width": "NONE",
              "display_name": "file",
              "dependents": [],
              "order": -1
            },
            "value": {
              "name": "file",
              "value": null,
              "recommended_values": [],
              "errors": [
                "Missing required configuration \"file\" which has no default value."
              ],
              "visible": true
            }
          },
          {
            "definition": {
              "name": "topic",
              "type": "STRING",
              "required": true,
              "default_value": null,
              "importance": "HIGH",
              "documentation": "The topic to publish data to",
              "group": null,
              "width": "NONE",
              "display_name": "topic",
              "dependents": [],
              "order": -1
            },
            "value": {
              "name": "topic",
              "value": null,
              "recommended_values": [],
              "errors": [
                "Missing required configuration \"topic\" which has no default value."
              ],
              "visible": true
            }
          }
        ]
      }
      

      You can see that the recommended values for transformations are there. However, if you do

      curl -X PUT -H "Content-Type: application/json" --data '{"connector.class": "org.apache.kafka.connect.file.FileStreamSourceConnector", "name": "file"}' http://localhost:8083/connector-plugins/org.apache.kafka.connect.file.FileStreamSourceConnector/config/validate  | jq
      

      before that command, the recommended values will not appear.

      Attachments

        Issue Links

          Activity

            People

              ewencp Ewen Cheslack-Postava
              ewencp Ewen Cheslack-Postava
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - 24h
                  24h
                  Remaining:
                  Remaining Estimate - 24h
                  24h
                  Logged:
                  Time Spent - Not Specified
                  Not Specified